You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cp...@apache.org on 2017/05/25 17:39:37 UTC

[1/6] lucene-solr:jira/solr-8668: SOLR-10233: Add support for replica types

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-8668 e4cbe0ed8 -> ab9aad20a


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
index 48f7670..9b8f707 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
@@ -25,9 +25,11 @@ import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
@@ -74,11 +76,16 @@ import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.Diagnostics;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.ReplicationHandler;
+import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.update.SolrCmdDistributor;
+import org.apache.solr.update.SolrIndexWriter;
 import org.apache.solr.util.RTimer;
+import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.noggit.CharArr;
@@ -232,6 +239,11 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
             CreateMode.PERSISTENT, true);
       }
     }
+    if (useTlogReplicas()) {
+      log.info("Will use {} replicas unless explicitly asked otherwise", Replica.Type.TLOG);
+    } else {
+      log.info("Will use {} replicas unless explicitly asked otherwise", Replica.Type.NRT);
+    }
   }
 
   @BeforeClass
@@ -272,8 +284,8 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
         shardToJetty, shardToLeaderJetty);
   }
 
-  protected int getRealtimeReplicas() {
-    return -1;
+  protected boolean useTlogReplicas() {
+    return false;
   }
   
   protected CloudSolrClient createCloudClient(String defaultCollection) {
@@ -379,17 +391,22 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
     StringBuilder sb = new StringBuilder();
 
     if ("2".equals(getStateFormat())) {
-      log.info("Creating collection1 with stateFormat=2");
+      log.info("Creating " + DEFAULT_COLLECTION + " with stateFormat=2");
       SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(),
           AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT);
       Overseer.getStateUpdateQueue(zkClient).offer(
           Utils.toJSON(Utils.makeMap(Overseer.QUEUE_OPERATION,
-              CollectionParams.CollectionAction.CREATE.toLower(), "name",
-              DEFAULT_COLLECTION, "numShards", String.valueOf(sliceCount),
+              CollectionParams.CollectionAction.CREATE.toLower(), 
+              "name", DEFAULT_COLLECTION, 
+              "numShards", String.valueOf(sliceCount),
               DocCollection.STATE_FORMAT, getStateFormat(),
-              ZkStateReader.REALTIME_REPLICAS, getRealtimeReplicas())));
+              ZkStateReader.NRT_REPLICAS, useTlogReplicas()?"0":"1",
+              ZkStateReader.TLOG_REPLICAS, useTlogReplicas()?"1":"0",
+              ZkStateReader.PULL_REPLICAS, String.valueOf(getPullReplicaCount()))));
       zkClient.close();
     }
+    
+    int numPullReplicas = getPullReplicaCount() * sliceCount;
 
     for (int i = 1; i <= numJettys; i++) {
       if (sb.length() > 0) sb.append(',');
@@ -399,9 +416,22 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
 
       jettyDir.mkdirs();
       setupJettySolrHome(jettyDir);
-      log.info("create jetty {} in directory {}", i, jettyDir);
-      JettySolrRunner j = createJetty(jettyDir, useJettyDataDir ? getDataDir(testDir + "/jetty"
-          + cnt) : null, null, "solrconfig.xml", null);
+      JettySolrRunner j;
+      
+      if (numPullReplicas > 0) {
+        numPullReplicas--;
+        log.info("create jetty {} in directory {} of type {}", i, jettyDir, Replica.Type.PULL);
+        j = createJetty(jettyDir, useJettyDataDir ? getDataDir(testDir + "/jetty"
+            + cnt) : null, null, "solrconfig.xml", null, Replica.Type.PULL);
+      } else if (useTlogReplicas()) {
+        log.info("create jetty {} in directory {} of type {}", i, jettyDir, Replica.Type.TLOG);
+        j = createJetty(jettyDir, useJettyDataDir ? getDataDir(testDir + "/jetty"
+            + cnt) : null, null, "solrconfig.xml", null, Replica.Type.TLOG);
+      } else {
+        log.info("create jetty {} in directory {} of type {}", i, jettyDir, Replica.Type.NRT);
+        j = createJetty(jettyDir, useJettyDataDir ? getDataDir(testDir + "/jetty"
+            + cnt) : null, null, "solrconfig.xml", null, null);
+      }
       jettys.add(j);
       SolrClient client = createNewSolrClient(j.getLocalPort());
       clients.add(client);
@@ -410,17 +440,18 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
     this.jettys.addAll(jettys);
     this.clients.addAll(clients);
 
-    int numShards = getTotalReplicas(DEFAULT_COLLECTION);
+    int numReplicas = getTotalReplicas(DEFAULT_COLLECTION);
+    int expectedNumReplicas = numJettys;
 
     // now wait until we see that the number of shards in the cluster state
     // matches what we expect
     int retries = 0;
-    while (numShards != getShardCount()) {
-      numShards = getTotalReplicas(DEFAULT_COLLECTION);
-      if (numShards == getShardCount()) break;
+    while (numReplicas != expectedNumReplicas) {
+      numReplicas = getTotalReplicas(DEFAULT_COLLECTION);
+      if (numReplicas == expectedNumReplicas) break;
       if (retries++ == 60) {
         printLayoutOnTearDown = true;
-        fail("Shards in the state does not match what we set:" + numShards + " vs " + getShardCount());
+        fail("Number of replicas in the state does not match what we set:" + numReplicas + " vs " + expectedNumReplicas);
       }
       Thread.sleep(500);
     }
@@ -431,7 +462,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
       zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + i, 10000);
     }
 
-    if (numShards > 0) {
+    if (numReplicas > 0) {
       updateMappingsFromZk(this.jettys, this.clients);
     }
 
@@ -449,6 +480,10 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
   }
 
 
+  protected int getPullReplicaCount() {
+    return 0;
+  }
+
   /* Total number of replicas (number of cores serving an index to the collection) shown by the cluster state */
   protected int getTotalReplicas(String collection) {
     ZkStateReader zkStateReader = cloudClient.getZkStateReader();
@@ -484,8 +519,12 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
 
     return jetty;
   }
+  
+  public final JettySolrRunner createJetty(File solrHome, String dataDir, String shardList, String solrConfigOverride, String schemaOverride) throws Exception {
+    return createJetty(solrHome, dataDir, shardList, solrConfigOverride, schemaOverride, null);
+  }
 
-  public JettySolrRunner createJetty(File solrHome, String dataDir, String shardList, String solrConfigOverride, String schemaOverride) throws Exception {
+  public JettySolrRunner createJetty(File solrHome, String dataDir, String shardList, String solrConfigOverride, String schemaOverride, Replica.Type replicaType) throws Exception {
     // randomly test a relative solr.home path
     if (random().nextBoolean()) {
       solrHome = getRelativeSolrHomePath(solrHome);
@@ -508,6 +547,11 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
       props.setProperty("shards", shardList);
     if (dataDir != null)
       props.setProperty("solr.data.dir", getDataDir(dataDir));
+    if (replicaType != null) {
+      props.setProperty("replicaType", replicaType.toString());
+    } else if (random().nextBoolean()) {
+      props.setProperty("replicaType", Replica.Type.NRT.toString());
+    }
     props.setProperty("coreRootDirectory", solrHome.toPath().resolve("cores").toAbsolutePath().toString());
     
     JettySolrRunner jetty = new JettySolrRunner(solrHome.getPath(), props, jettyconfig);
@@ -522,7 +566,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
    * with IPTables.
    */
   public JettySolrRunner createProxiedJetty(File solrHome, String dataDir,
-                                     String shardList, String solrConfigOverride, String schemaOverride)
+                                     String shardList, String solrConfigOverride, String schemaOverride, Replica.Type replicaType)
       throws Exception {
 
     JettyConfig jettyconfig = JettyConfig.builder()
@@ -542,6 +586,11 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
       props.setProperty("shards", shardList);
     if (dataDir != null)
       props.setProperty("solr.data.dir", getDataDir(dataDir));
+    if (replicaType != null) {
+      props.setProperty("replicaType", replicaType.toString());
+    } else if (random().nextBoolean()) {
+      props.setProperty("replicaType", Replica.Type.NRT.toString());
+    }
     props.setProperty("coreRootDirectory", solrHome.toPath().resolve("cores").toAbsolutePath().toString());
 
     JettySolrRunner jetty = new JettySolrRunner(solrHome.getPath(), props, jettyconfig);
@@ -1570,11 +1619,24 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
       String shardNames = (String) collectionProps.get(SHARDS_PROP);
       numShards = StrUtils.splitSmart(shardNames,',').size();
     }
-    Integer replicationFactor = (Integer) collectionProps.get(ZkStateReader.REPLICATION_FACTOR);
-    if(replicationFactor==null){
-      replicationFactor = (Integer) OverseerCollectionMessageHandler.COLL_PROPS.get(ZkStateReader.REPLICATION_FACTOR);
+    Integer numNrtReplicas = (Integer) collectionProps.get(ZkStateReader.NRT_REPLICAS);
+    if (numNrtReplicas == null) {
+      numNrtReplicas = (Integer) collectionProps.get(ZkStateReader.REPLICATION_FACTOR);
+    }
+    if(numNrtReplicas == null){
+      numNrtReplicas = (Integer) OverseerCollectionMessageHandler.COLL_PROPS.get(ZkStateReader.REPLICATION_FACTOR);
+    }
+    if (numNrtReplicas == null) {
+      numNrtReplicas = Integer.valueOf(0);
+    }
+    Integer numTlogReplicas = (Integer) collectionProps.get(ZkStateReader.TLOG_REPLICAS);
+    if (numTlogReplicas == null) {
+      numTlogReplicas = Integer.valueOf(0);
+    }
+    Integer numPullReplicas = (Integer) collectionProps.get(ZkStateReader.PULL_REPLICAS);
+    if (numPullReplicas == null) {
+      numPullReplicas = Integer.valueOf(0);
     }
-
     if (confSetName != null) {
       params.set("collection.configName", confSetName);
     }
@@ -1582,7 +1644,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
     int clientIndex = random().nextInt(2);
     List<Integer> list = new ArrayList<>();
     list.add(numShards);
-    list.add(replicationFactor);
+    list.add(numNrtReplicas + numTlogReplicas + numPullReplicas);
     if (collectionInfos != null) {
       collectionInfos.put(collectionName, list);
     }
@@ -1610,26 +1672,32 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
   protected CollectionAdminResponse createCollection(Map<String,List<Integer>> collectionInfos,
       String collectionName, int numShards, int replicationFactor, int maxShardsPerNode, SolrClient client, String createNodeSetStr) throws SolrServerException, IOException {
 
+    int numNrtReplicas = useTlogReplicas()?0:replicationFactor;
+    int numTlogReplicas = useTlogReplicas()?replicationFactor:0;
     return createCollection(collectionInfos, collectionName,
         Utils.makeMap(
         NUM_SLICES, numShards,
-        ZkStateReader.REPLICATION_FACTOR, replicationFactor,
+        ZkStateReader.NRT_REPLICAS, numNrtReplicas,
+        ZkStateReader.TLOG_REPLICAS, numTlogReplicas,
+        ZkStateReader.PULL_REPLICAS, getPullReplicaCount(),
         CREATE_NODE_SET, createNodeSetStr,
-        ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode,
-        ZkStateReader.REALTIME_REPLICAS, getRealtimeReplicas()),
+        ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode),
         client);
   }
 
   protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
                                                      String collectionName, int numShards, int replicationFactor, int maxShardsPerNode, SolrClient client, String createNodeSetStr, String configName) throws SolrServerException, IOException {
 
+    int numNrtReplicas = useTlogReplicas()?0:replicationFactor;
+    int numTlogReplicas = useTlogReplicas()?replicationFactor:0;
     return createCollection(collectionInfos, collectionName,
         Utils.makeMap(
         NUM_SLICES, numShards,
-        ZkStateReader.REPLICATION_FACTOR, replicationFactor,
+        ZkStateReader.NRT_REPLICAS, numNrtReplicas,
+        ZkStateReader.TLOG_REPLICAS, numTlogReplicas,
+        ZkStateReader.PULL_REPLICAS, getPullReplicaCount(),
         CREATE_NODE_SET, createNodeSetStr,
-        ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode,
-        ZkStateReader.REALTIME_REPLICAS, getRealtimeReplicas()),
+        ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode),
         client, configName);
   }
 
@@ -1666,7 +1734,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
             - DEFAULT_COLLECTION.length() - 1);
   }
 
-  protected SolrInputDocument getDoc(Object... fields) throws Exception {
+  public static SolrInputDocument getDoc(Object... fields) throws Exception {
     SolrInputDocument doc = new SolrInputDocument();
     addFields(doc, fields);
     return doc;
@@ -1808,11 +1876,13 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
                                   int numShards ) throws Exception {
     int maxShardsPerNode = ((((numShards+1) * replicationFactor) / getCommonCloudSolrClient()
         .getZkStateReader().getClusterState().getLiveNodes().size())) + 1;
-
+    int numNrtReplicas = useTlogReplicas()?0:replicationFactor;
+    int numTlogReplicas = useTlogReplicas()?replicationFactor:0;
     Map<String, Object> props = makeMap(
-        ZkStateReader.REPLICATION_FACTOR, replicationFactor,
         ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode,
-        ZkStateReader.REALTIME_REPLICAS, getRealtimeReplicas(),
+        ZkStateReader.NRT_REPLICAS, numNrtReplicas,
+        ZkStateReader.TLOG_REPLICAS, numTlogReplicas,
+        ZkStateReader.PULL_REPLICAS, getPullReplicaCount(),
         NUM_SLICES, numShards);
     Map<String,List<Integer>> collectionInfos = new HashMap<>();
     createCollection(collectionInfos, collName, props, client);
@@ -1971,6 +2041,136 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
     }
     return reloadedOk;
   }
+  
+
+  protected void logReplicaTypesReplicationInfo(String collectionName, ZkStateReader zkStateReader) throws KeeperException, InterruptedException, IOException {
+    log.info("## Collecting extra Replica.Type information of the cluster");
+    zkStateReader.updateLiveNodes();
+    StringBuilder builder = new StringBuilder();
+    zkStateReader.forceUpdateCollection(collectionName);
+    DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
+    for(Slice s:collection.getSlices()) {
+      Replica leader = s.getLeader();
+      for (Replica r:s.getReplicas()) {
+        if (!r.isActive(zkStateReader.getClusterState().getLiveNodes())) {
+          builder.append(String.format(Locale.ROOT, "Replica %s not in liveNodes or is not active%s", r.getName(), System.lineSeparator()));
+          continue;
+        }
+        if (r.equals(leader)) {
+          builder.append(String.format(Locale.ROOT, "Replica %s is leader%s", r.getName(), System.lineSeparator()));
+        }
+        logReplicationDetails(r, builder);
+      }
+    }
+    log.info("Summary of the cluster: " + builder.toString());
+  }
+
+  protected void waitForReplicationFromReplicas(String collectionName, ZkStateReader zkStateReader, TimeOut timeout) throws KeeperException, InterruptedException, IOException {
+    zkStateReader.forceUpdateCollection(collectionName);
+    DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
+    Map<String, CoreContainer> containers = new HashMap<>();
+    for (JettySolrRunner runner:jettys) {
+      if (!runner.isRunning()) {
+        continue;
+      }
+      containers.put(runner.getNodeName(), runner.getCoreContainer());
+    }
+    for(Slice s:collection.getSlices()) {
+      Replica leader = s.getLeader();
+      long leaderIndexVersion = -1;
+      while (!timeout.hasTimedOut()) {
+        leaderIndexVersion = getIndexVersion(leader);
+        if (leaderIndexVersion >= 0) {
+          break;
+        }
+        Thread.sleep(1000);
+      }
+      if (timeout.hasTimedOut()) {
+        fail("Unable to get leader indexVersion");
+      }
+      for (Replica pullReplica:s.getReplicas(EnumSet.of(Replica.Type.PULL,Replica.Type.TLOG))) {
+        if (!zkStateReader.getClusterState().liveNodesContain(pullReplica.getNodeName())) {
+          continue;
+        }
+        while (true) {
+          long replicaIndexVersion = getIndexVersion(pullReplica); 
+          if (leaderIndexVersion == replicaIndexVersion) {
+            log.debug("Leader replica's version ({}) in sync with replica({}): {} == {}", leader.getName(), pullReplica.getName(), leaderIndexVersion, replicaIndexVersion);
+            
+            // Make sure the host is serving the correct version
+            try (SolrCore core = containers.get(pullReplica.getNodeName()).getCore(pullReplica.getCoreName())) {
+              RefCounted<SolrIndexSearcher> ref = core.getRegisteredSearcher();
+              try {
+                SolrIndexSearcher searcher = ref.get();
+                String servingVersion = searcher.getIndexReader().getIndexCommit().getUserData().get(SolrIndexWriter.COMMIT_TIME_MSEC_KEY);
+                if (Long.parseLong(servingVersion) == replicaIndexVersion) {
+                  break;
+                } else {
+                  log.debug("Replica {} has the correct version replicated, but the searcher is not ready yet. Replicated version: {}, Serving version: {}", pullReplica.getName(), replicaIndexVersion, servingVersion);
+                }
+              } finally {
+                if (ref != null) ref.decref();
+              }
+            }
+          } else {
+            if (timeout.hasTimedOut()) {
+              logReplicaTypesReplicationInfo(collectionName, zkStateReader);
+              fail(String.format(Locale.ROOT, "Timed out waiting for replica %s (%d) to replicate from leader %s (%d)", pullReplica.getName(), replicaIndexVersion, leader.getName(), leaderIndexVersion));
+            }
+            if (leaderIndexVersion > replicaIndexVersion) {
+              log.debug("{} version is {} and leader's is {}, will wait for replication", pullReplica.getName(), replicaIndexVersion, leaderIndexVersion);
+            } else {
+              log.debug("Leader replica's version ({}) is lower than pull replica({}): {} < {}", leader.getName(), pullReplica.getName(), leaderIndexVersion, replicaIndexVersion);
+            }
+          }
+          Thread.sleep(1000);
+        }
+      }
+    }
+  }
+  
+  protected void waitForAllWarmingSearchers() throws InterruptedException {
+    for (JettySolrRunner jetty:jettys) {
+      if (!jetty.isRunning()) {
+        continue;
+      }
+      for (SolrCore core:jetty.getCoreContainer().getCores()) {
+        waitForWarming(core);
+      }
+    }
+  }
+
+  protected long getIndexVersion(Replica replica) throws IOException {
+    try (HttpSolrClient client = new HttpSolrClient.Builder(replica.getCoreUrl()).build()) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set("qt", "/replication");
+      params.set(ReplicationHandler.COMMAND, ReplicationHandler.CMD_SHOW_COMMITS);
+      try {
+        QueryResponse response = client.query(params);
+        @SuppressWarnings("unchecked")
+        List<NamedList<Object>> commits = (List<NamedList<Object>>)response.getResponse().get(ReplicationHandler.CMD_SHOW_COMMITS);
+        Collections.max(commits, (a,b)->((Long)a.get("indexVersion")).compareTo((Long)b.get("indexVersion")));
+        return (long) Collections.max(commits, (a,b)->((Long)a.get("indexVersion")).compareTo((Long)b.get("indexVersion"))).get("indexVersion");
+      } catch (SolrServerException e) {
+        log.warn("Exception getting version from {}, will return an invalid version to retry.", replica.getName(), e);
+        return -1;
+      }
+    }
+  }
+  
+  protected void logReplicationDetails(Replica replica, StringBuilder builder) throws IOException {
+    try (HttpSolrClient client = new HttpSolrClient.Builder(replica.getCoreUrl()).build()) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set("qt", "/replication");
+      params.set(ReplicationHandler.COMMAND, ReplicationHandler.CMD_DETAILS);
+      try {
+        QueryResponse response = client.query(params);
+        builder.append(String.format(Locale.ROOT, "%s: %s%s", replica.getName(), response.getResponse(), System.lineSeparator()));
+      } catch (SolrServerException e) {
+        log.warn("Unable to ger replication details for replica {}", replica.getName(), e);
+      }
+    }
+  }
 
   static RequestStatusState getRequestStateAfterCompletion(String requestId, int waitForSeconds, SolrClient client)
       throws IOException, SolrServerException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
index 5cae356..bdbbdd2 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
@@ -21,15 +21,20 @@ import java.lang.invoke.MethodHandles;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Pattern;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.cloud.AbstractFullDistribZkTestBase.CloudJettyRunner;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Replica.Type;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -39,6 +44,7 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.servlet.SolrDispatchFilter;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.util.RTimer;
+import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -61,8 +67,9 @@ public class ChaosMonkey {
   private Map<String,List<CloudJettyRunner>> shardToJetty;
   
   private static final Boolean MONKEY_ENABLED = Boolean.valueOf(System.getProperty("solr.tests.cloud.cm.enabled", "true"));
-  private static final Boolean CONN_LOSS = Boolean.valueOf(System.getProperty("solr.tests.cloud.cm.connloss", null));
-  private static final Boolean EXP = Boolean.valueOf(System.getProperty("solr.tests.cloud.cm.exp", null));
+  // NOTE: CONN_LOSS and EXP are currently being set to "false" intentionally here. Remove the default value once we know tests pass reliably under those conditions
+  private static final String CONN_LOSS = System.getProperty("solr.tests.cloud.cm.connloss", "false");
+  private static final String EXP = System.getProperty("solr.tests.cloud.cm.exp", "false");
   
   private ZkTestServer zkServer;
   private ZkStateReader zkStateReader;
@@ -106,12 +113,12 @@ public class ChaosMonkey {
     }
     
     if (EXP != null) {
-      expireSessions = EXP; 
+      expireSessions = Boolean.parseBoolean(EXP); 
     } else {
       expireSessions = chaosRandom.nextBoolean();
     }
     if (CONN_LOSS != null) {
-      causeConnectionLoss = CONN_LOSS;
+      causeConnectionLoss = Boolean.parseBoolean(CONN_LOSS);
     } else {
       causeConnectionLoss = chaosRandom.nextBoolean();
     }
@@ -123,10 +130,9 @@ public class ChaosMonkey {
   
   // TODO: expire all clients at once?
   public void expireSession(final JettySolrRunner jetty) {
-    monkeyLog("expire session for " + jetty.getLocalPort() + " !");
-
     CoreContainer cores = jetty.getCoreContainer();
     if (cores != null) {
+      monkeyLog("expire session for " + jetty.getLocalPort() + " !");
       causeConnectionLoss(jetty);
       long sessionId = cores.getZkController().getZkClient()
           .getSolrZooKeeper().getSessionId();
@@ -146,7 +152,7 @@ public class ChaosMonkey {
   }
   
   public void randomConnectionLoss() throws KeeperException, InterruptedException {
-    monkeyLog("cause connection loss!");
+    monkeyLog("Will cause connection loss!");
     
     String sliceName = getRandomSlice();
     CloudJettyRunner jetty = getRandomJetty(sliceName, aggressivelyKillLeaders);
@@ -159,6 +165,7 @@ public class ChaosMonkey {
   public static void causeConnectionLoss(JettySolrRunner jetty) {
     CoreContainer cores = jetty.getCoreContainer();
     if (cores != null) {
+      monkeyLog("Will cause connection loss on " + jetty.getLocalPort());
       SolrZkClient zkClient = cores.getZkController().getZkClient();
       zkClient.getSolrZooKeeper().closeCnxn();
     }
@@ -187,7 +194,7 @@ public class ChaosMonkey {
   
   private static void stopJettySolrRunner(JettySolrRunner jetty) throws Exception {
     assert(jetty != null);
-    monkeyLog("stop shard! " + jetty.getLocalPort());
+    monkeyLog("stop jetty! " + jetty.getLocalPort());
     SolrDispatchFilter sdf = jetty.getSolrDispatchFilter();
     if (sdf != null) {
       try {
@@ -231,7 +238,7 @@ public class ChaosMonkey {
 
     IpTables.blockPort(jetty.getLocalPort());
     
-    monkeyLog("kill shard! " + jetty.getLocalPort());
+    monkeyLog("kill jetty! " + jetty.getLocalPort());
     
     jetty.stop();
     
@@ -369,16 +376,32 @@ public class ChaosMonkey {
       return null;
     }
     
+    boolean canKillIndexer = canKillIndexer(slice);
+    
+    if (!canKillIndexer) {
+      monkeyLog("Number of indexer nodes (nrt or tlog replicas) is not enough to kill one of them, Will only choose a pull replica to kill");
+    }
+    
     int chance = chaosRandom.nextInt(10);
-    CloudJettyRunner cjetty;
-    if (chance <= 5 && aggressivelyKillLeaders) {
+    CloudJettyRunner cjetty = null;
+    if (chance <= 5 && aggressivelyKillLeaders && canKillIndexer) {
       // if killLeader, really aggressively go after leaders
       cjetty = shardToLeaderJetty.get(slice);
     } else {
-      // get random shard
       List<CloudJettyRunner> jetties = shardToJetty.get(slice);
-      int index = chaosRandom.nextInt(jetties.size());
-      cjetty = jetties.get(index);
+      // get random node
+      int attempt = 0;
+      while (true) {
+        attempt++;
+        int index = chaosRandom.nextInt(jetties.size());
+        cjetty = jetties.get(index);
+        if (canKillIndexer || getTypeForJetty(slice, cjetty) == Replica.Type.PULL) {
+          break;
+        } else if (attempt > 20) {
+          monkeyLog("Can't kill indexer nodes (nrt or tlog replicas) and couldn't find a random pull node after 20 attempts - monkey cannot kill :(");
+          return null;
+        }
+      }
       
       ZkNodeProps leader = null;
       try {
@@ -403,7 +426,7 @@ public class ChaosMonkey {
         return null;
       }
 
-      boolean isLeader = leader.getStr(ZkStateReader.NODE_NAME_PROP).equals(jetties.get(index).nodeName)
+      boolean isLeader = leader.getStr(ZkStateReader.NODE_NAME_PROP).equals(cjetty.nodeName)
           || rtIsLeader;
       if (!aggressivelyKillLeaders && isLeader) {
         // we don't kill leaders...
@@ -424,18 +447,61 @@ public class ChaosMonkey {
     return cjetty;
   }
 
-  private int checkIfKillIsLegal(String slice, int numActive) throws KeeperException, InterruptedException {
-    for (CloudJettyRunner cloudJetty : shardToJetty.get(slice)) {
+  private Type getTypeForJetty(String sliceName, CloudJettyRunner cjetty) {
+    DocCollection docCollection = zkStateReader.getClusterState().getCollection(collection);
+    
+    Slice slice = docCollection.getSlice(sliceName);
+    
+    ZkNodeProps props = slice.getReplicasMap().get(cjetty.coreNodeName);
+    if (props == null) {
+      throw new RuntimeException("shard name " + cjetty.coreNodeName + " not found in " + slice.getReplicasMap().keySet());
+    }
+    return Replica.Type.valueOf(props.getStr(ZkStateReader.REPLICA_TYPE));
+  }
+
+  private boolean canKillIndexer(String sliceName) throws KeeperException, InterruptedException {
+    int numIndexersFoundInShard = 0;
+    for (CloudJettyRunner cloudJetty : shardToJetty.get(sliceName)) {
+      
+      // get latest cloud state
+      zkStateReader.forceUpdateCollection(collection);
+      
+      DocCollection docCollection = zkStateReader.getClusterState().getCollection(collection);
+      
+      Slice slice = docCollection.getSlice(sliceName);
+      
+      ZkNodeProps props = slice.getReplicasMap().get(cloudJetty.coreNodeName);
+      if (props == null) {
+        throw new RuntimeException("shard name " + cloudJetty.coreNodeName + " not found in " + slice.getReplicasMap().keySet());
+      }
+      
+      final Replica.State state = Replica.State.getState(props.getStr(ZkStateReader.STATE_PROP));
+      final Replica.Type replicaType = Replica.Type.valueOf(props.getStr(ZkStateReader.REPLICA_TYPE));
+      final String nodeName = props.getStr(ZkStateReader.NODE_NAME_PROP);
+      
+      if (cloudJetty.jetty.isRunning()
+          && state == Replica.State.ACTIVE
+          && (replicaType == Replica.Type.TLOG || replicaType == Replica.Type.NRT) 
+          && zkStateReader.getClusterState().liveNodesContain(nodeName)) {
+        numIndexersFoundInShard++;
+      }
+    }
+    return numIndexersFoundInShard > 1;
+  }
+
+  private int checkIfKillIsLegal(String sliceName, int numActive) throws KeeperException, InterruptedException {
+    for (CloudJettyRunner cloudJetty : shardToJetty.get(sliceName)) {
       
       // get latest cloud state
       zkStateReader.forceUpdateCollection(collection);
       
-      Slice theShards = zkStateReader.getClusterState().getSlicesMap(collection)
-          .get(slice);
+      DocCollection docCollection = zkStateReader.getClusterState().getCollection(collection);
+      
+      Slice slice = docCollection.getSlice(sliceName);
       
-      ZkNodeProps props = theShards.getReplicasMap().get(cloudJetty.coreNodeName);
+      ZkNodeProps props = slice.getReplicasMap().get(cloudJetty.coreNodeName);
       if (props == null) {
-        throw new RuntimeException("shard name " + cloudJetty.coreNodeName + " not found in " + theShards.getReplicasMap().keySet());
+        throw new RuntimeException("shard name " + cloudJetty.coreNodeName + " not found in " + slice.getReplicasMap().keySet());
       }
       
       final Replica.State state = Replica.State.getState(props.getStr(ZkStateReader.STATE_PROP));
@@ -501,6 +567,10 @@ public class ChaosMonkey {
     log.info("monkey: " + msg);
   }
   
+  public static void monkeyLog(String msg, Object...logParams) {
+    log.info("monkey: " + msg, logParams);
+  }
+  
   public void stopTheMonkey() {
     stop = true;
     try {
@@ -579,7 +649,7 @@ public class ChaosMonkey {
   }
   
   public static boolean start(JettySolrRunner jetty) throws Exception {
-
+    monkeyLog("starting jetty! " + jetty.getLocalPort());
     IpTables.unblockPort(jetty.getLocalPort());
     try {
       jetty.start();
@@ -619,4 +689,49 @@ public class ChaosMonkey {
     return true;
   }
 
+  /**
+   * You can call this method to wait while the ChaosMonkey is running, it waits approximately the specified time, and periodically
+   * logs the status of the collection
+   * @param runLength The time in ms to wait
+   * @param collectionName The main collection being used for the ChaosMonkey
+   * @param zkStateReader current state reader
+   */
+  public static void wait(long runLength, String collectionName, ZkStateReader zkStateReader) throws InterruptedException {
+    TimeOut t = new TimeOut(runLength, TimeUnit.MILLISECONDS);
+    while (!t.hasTimedOut()) {
+      Thread.sleep(Math.min(1000, t.timeLeft(TimeUnit.MILLISECONDS)));
+      logCollectionStateSummary(collectionName, zkStateReader);
+    }
+  }
+
+  private static void logCollectionStateSummary(String collectionName, ZkStateReader zkStateReader) {
+    Pattern portPattern = Pattern.compile(".*:([0-9]*).*");
+    DocCollection docCollection = zkStateReader.getClusterState().getCollection(collectionName);
+    if (docCollection == null) {
+      monkeyLog("Could not find collection {}", collectionName);
+    }
+    StringBuilder builder = new StringBuilder();
+    builder.append("Collection status: {");
+    for (Slice slice:docCollection.getSlices()) {
+      builder.append(slice.getName() + ": {");
+      for (Replica replica:slice.getReplicas()) {
+        log.info(replica.toString());
+        java.util.regex.Matcher m = portPattern.matcher(replica.getBaseUrl());
+        m.find();
+        String jettyPort = m.group(1);
+        builder.append(String.format(Locale.ROOT, "%s(%s): {state: %s, type: %s, leader: %s, Live: %s}, ", 
+            replica.getName(), jettyPort, replica.getState(), replica.getType(), (replica.get("leader")!= null), zkStateReader.getClusterState().liveNodesContain(replica.getNodeName())));
+      }
+      if (slice.getReplicas().size() > 0) {
+        builder.setLength(builder.length() - 2);
+      }
+      builder.append("}, ");
+    }
+    if (docCollection.getSlices().size() > 0) {
+      builder.setLength(builder.length() - 2);
+    }
+    builder.append("}");
+    monkeyLog(builder.toString());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/test-framework/src/java/org/apache/solr/cloud/StoppableCommitThread.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/StoppableCommitThread.java b/solr/test-framework/src/java/org/apache/solr/cloud/StoppableCommitThread.java
new file mode 100644
index 0000000..f87ebb5
--- /dev/null
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/StoppableCommitThread.java
@@ -0,0 +1,69 @@
+/*
+ * 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.concurrent.atomic.AtomicInteger;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.cloud.AbstractFullDistribZkTestBase.StoppableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StoppableCommitThread extends StoppableThread {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  
+  private final SolrClient cloudClient;
+  private final long timeBetweenCommitsMs;
+  private final boolean softCommits;
+  private volatile boolean stop = false;
+  private final AtomicInteger numCommits = new AtomicInteger(0);
+  private final AtomicInteger numFails = new AtomicInteger(0);
+
+  public StoppableCommitThread(SolrClient cloudClient, long timeBetweenCommitsMs, boolean softCommits) {
+    super("StoppableCommitThread");
+    this.cloudClient = cloudClient;
+    this.timeBetweenCommitsMs = timeBetweenCommitsMs;
+    this.softCommits = softCommits;
+  }
+  
+  @Override
+  public void run() {
+    log.debug("StoppableCommitThread started");
+    while (!stop) {
+      try {
+        cloudClient.commit(false, false, softCommits);
+        numCommits.incrementAndGet();
+      } catch (Exception e) {
+        numFails.incrementAndGet();
+      }
+      try {
+        Thread.sleep(timeBetweenCommitsMs);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        break;
+      }
+    }
+    log.debug("StoppableCommitThread finished. Committed {} times. Failed {} times.", numCommits.get(), numFails.get());
+  }
+
+  @Override
+  public void safeStop() {
+    this.stop = true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/test-framework/src/java/org/apache/solr/cloud/StoppableIndexingThread.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/StoppableIndexingThread.java b/solr/test-framework/src/java/org/apache/solr/cloud/StoppableIndexingThread.java
index 7dd88c1..0385d73 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/StoppableIndexingThread.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/StoppableIndexingThread.java
@@ -132,7 +132,7 @@ public class StoppableIndexingThread extends AbstractFullDistribZkTestBase.Stopp
       
       if (docs.size() > 0 && pauseBetweenUpdates) {
         try {
-          Thread.currentThread().sleep(AbstractFullDistribZkTestBase.random().nextInt(500) + 50);
+          Thread.sleep(AbstractFullDistribZkTestBase.random().nextInt(500) + 50);
         } catch (InterruptedException e) {
           Thread.currentThread().interrupt();
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/test-framework/src/java/org/apache/solr/cloud/StoppableSearchThread.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/StoppableSearchThread.java b/solr/test-framework/src/java/org/apache/solr/cloud/StoppableSearchThread.java
index fa916c6..c579f22 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/StoppableSearchThread.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/StoppableSearchThread.java
@@ -47,7 +47,7 @@ class StoppableSearchThread extends AbstractFullDistribZkTestBase.StoppableThrea
     Random random = LuceneTestCase.random();
     int numSearches = 0;
 
-    while (true && !stop) {
+    while (!stop) {
       numSearches++;
       try {
         //to come to the aid of their country.


[5/6] lucene-solr:jira/solr-8668: SOLR-10233: Add support for replica types

Posted by cp...@apache.org.
SOLR-10233: Add support for replica types


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/2fc41d56
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/2fc41d56
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/2fc41d56

Branch: refs/heads/jira/solr-8668
Commit: 2fc41d565a4a0408a09856a37d3be7d87414ba3f
Parents: 1802d24
Author: Tomas Fernandez Lobbe <tf...@apache.org>
Authored: Mon May 22 19:44:01 2017 -0700
Committer: Tomas Fernandez Lobbe <tf...@apache.org>
Committed: Mon May 22 19:58:51 2017 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  14 +
 .../org/apache/solr/cloud/AddReplicaCmd.java    |   9 +-
 .../src/java/org/apache/solr/cloud/Assign.java  |  12 +-
 .../org/apache/solr/cloud/CloudDescriptor.java  |  25 +-
 .../apache/solr/cloud/CreateCollectionCmd.java  |  40 +-
 .../org/apache/solr/cloud/CreateShardCmd.java   |  48 +-
 .../org/apache/solr/cloud/ElectionContext.java  |  13 +-
 .../java/org/apache/solr/cloud/MigrateCmd.java  |   8 +-
 .../org/apache/solr/cloud/MoveReplicaCmd.java   |   4 +-
 .../cloud/OverseerCollectionMessageHandler.java |  91 +-
 .../org/apache/solr/cloud/RecoveryStrategy.java | 166 +++-
 .../org/apache/solr/cloud/ReplaceNodeCmd.java   |   4 +-
 .../apache/solr/cloud/ReplicateFromLeader.java  |  42 +-
 .../java/org/apache/solr/cloud/RestoreCmd.java  |  71 +-
 .../org/apache/solr/cloud/SplitShardCmd.java    |   4 +-
 .../org/apache/solr/cloud/ZkController.java     |  67 +-
 .../solr/cloud/overseer/ReplicaMutator.java     |   7 +-
 .../solr/cloud/overseer/SliceMutator.java       |  24 +-
 .../solr/cloud/overseer/ZkStateWriter.java      |   1 +
 .../apache/solr/cloud/rule/ReplicaAssigner.java |   6 +-
 .../org/apache/solr/core/CoreContainer.java     |  38 +-
 .../org/apache/solr/handler/IndexFetcher.java   |  23 +-
 .../apache/solr/handler/RealTimeGetHandler.java |  20 +-
 .../apache/solr/handler/ReplicationHandler.java |   4 +
 .../solr/handler/admin/CollectionsHandler.java  |  17 +-
 .../solr/handler/admin/CoreAdminHandler.java    |   1 +
 .../solr/handler/admin/PrepRecoveryOp.java      |  30 +-
 .../handler/component/HttpShardHandler.java     |  37 +-
 .../handler/component/RealTimeGetComponent.java |  27 +-
 .../solr/update/DirectUpdateHandler2.java       |  20 +-
 .../org/apache/solr/update/UpdateHandler.java   |   6 +-
 .../java/org/apache/solr/update/UpdateLog.java  |   7 +-
 .../processor/DistributedUpdateProcessor.java   |  51 +-
 .../org/apache/solr/util/TestInjection.java     |   6 +-
 .../solr/collection1/conf/solrconfig.xml        |   8 +-
 .../cloud-minimal/conf/solrconfig.xml           |   2 +-
 .../AbstractCloudBackupRestoreTestCase.java     |  29 +-
 .../test/org/apache/solr/cloud/AssignTest.java  |   6 +
 .../solr/cloud/BasicDistributedZk2Test.java     |   4 +-
 .../solr/cloud/BasicDistributedZkTest.java      |  10 +-
 .../cloud/ChaosMonkeyNothingIsSafeTest.java     | 136 +--
 ...MonkeyNothingIsSafeWithPullReplicasTest.java | 327 +++++++
 ...aosMonkeySafeLeaderWithPullReplicasTest.java | 254 ++++++
 .../cloud/CollectionsAPIDistributedZkTest.java  |   4 +-
 .../solr/cloud/CollectionsAPISolrJTest.java     |  18 +-
 .../org/apache/solr/cloud/DeleteNodeTest.java   |  11 +-
 .../apache/solr/cloud/DeleteReplicaTest.java    |  22 +-
 .../org/apache/solr/cloud/ForceLeaderTest.java  |   4 +-
 .../FullThrottleStoppableIndexingThread.java    | 156 ++++
 .../apache/solr/cloud/HttpPartitionTest.java    |   8 +-
 .../LeaderInitiatedRecoveryOnCommitTest.java    |   8 +-
 .../solr/cloud/OnlyLeaderIndexesTest.java       | 435 ----------
 ...verseerCollectionConfigSetProcessorTest.java |   3 +-
 .../solr/cloud/RecoveryAfterSoftCommitTest.java |   8 +-
 .../org/apache/solr/cloud/ReplaceNodeTest.java  |  21 +-
 .../solr/cloud/ReplicationFactorTest.java       |   4 +-
 .../org/apache/solr/cloud/ShardSplitTest.java   |   6 -
 .../apache/solr/cloud/TestCloudRecovery.java    |   5 +-
 .../apache/solr/cloud/TestCollectionAPI.java    |  10 +-
 .../org/apache/solr/cloud/TestPullReplica.java  | 576 +++++++++++++
 .../cloud/TestPullReplicaErrorHandling.java     | 344 ++++++++
 .../org/apache/solr/cloud/TestTlogReplica.java  | 845 +++++++++++++++++++
 .../cloud/hdfs/HdfsBasicDistributedZkTest.java  |   4 +-
 .../reporters/solr/SolrCloudReportersTest.java  |   2 +-
 .../solr/update/TestInPlaceUpdatesDistrib.java  |  10 +-
 .../solrj/impl/ConcurrentUpdateSolrClient.java  |  23 +-
 .../solrj/request/CollectionAdminRequest.java   | 121 ++-
 .../apache/solr/common/cloud/DocCollection.java |  56 +-
 .../org/apache/solr/common/cloud/Replica.java   |  32 +
 .../org/apache/solr/common/cloud/Slice.java     |  21 +-
 .../apache/solr/common/cloud/ZkStateReader.java |  16 +-
 .../solr/common/params/CoreAdminParams.java     |   5 +
 ...ollectionAdminRequestRequiredParamsTest.java |   9 +
 .../solr/BaseDistributedSearchTestCase.java     |   3 +-
 .../java/org/apache/solr/SolrTestCaseJ4.java    |  31 +-
 .../cloud/AbstractFullDistribZkTestBase.java    | 264 +++++-
 .../java/org/apache/solr/cloud/ChaosMonkey.java | 159 +++-
 .../solr/cloud/StoppableCommitThread.java       |  69 ++
 .../solr/cloud/StoppableIndexingThread.java     |   2 +-
 .../solr/cloud/StoppableSearchThread.java       |   2 +-
 80 files changed, 4026 insertions(+), 1010 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7adac97..799df74 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -95,6 +95,20 @@ New Features
 
 * SOLR-10431: Make it possible to invoke v2 api calls using SolrJ (Cao Manh Dat, Noble Paul, shalin)
 
+* SOLR-10233: Add support for different replica types, that can handle updates differently:
+  - NRT: Writes updates to transaction log and indexes locally. Replicas of type “NRT” support NRT 
+         (soft commits) and RTG. Any NRT replica can become a leader. This was the only type supported 
+         in SolrCloud until now and it’s the default type.
+  - TLOG: Writes to transaction log, but not to index, uses replication to copy segment files from the 
+          shard leader. Any TLOG replica can become leader (by first applying all local transaction log 
+          elements). If a replica is of type TLOG but is also the leader, it will behave as a NRT. This
+          is exactly what was added in SOLR-9835 (non-realtime replicas), just the API and naming changes.
+  - PULL: Doesn’t index or writes to transaction log, just replicates from the shard leader. PULL replicas 
+          can’t become shard leaders (i.e., if there are only PULL replicas in the collection at some point, 
+          updates will fail same as if there is no leaders, queries continue to work), so they don’t even 
+          participate in elections.
+  (Tomás Fernández Löbbe)
+
 Bug Fixes
 ----------------------
 * SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
index 6bb3350..7338d9e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
@@ -72,6 +72,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     String node = message.getStr(CoreAdminParams.NODE);
     String shard = message.getStr(SHARD_ID_PROP);
     String coreName = message.getStr(CoreAdminParams.NAME);
+    Replica.Type replicaType = Replica.Type.valueOf(message.getStr(ZkStateReader.REPLICA_TYPE, Replica.Type.NRT.name()));
     boolean parallel = message.getBool("parallel", false);
     if (StringUtils.isBlank(coreName)) {
       coreName = message.getStr(CoreAdminParams.PROPERTY_PREFIX + CoreAdminParams.NAME);
@@ -93,7 +94,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     // Kind of unnecessary, but it does put the logic of whether to override maxShardsPerNode in one place.
     if (!skipCreateReplicaInClusterState) {
       node = getNodesForNewReplicas(clusterState, collection, shard, 1, node,
-          ocmh.overseer.getZkController().getCoreContainer()).get(0).nodeName;
+          ocmh.overseer.getZkController().getCoreContainer()).get(0).nodeName;// TODO: use replica type in this logic too
     }
     log.info("Node Identified {} for creating new replica", node);
 
@@ -101,7 +102,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Node: " + node + " is not live");
     }
     if (coreName == null) {
-      coreName = Assign.buildCoreName(coll, shard);
+      coreName = Assign.buildCoreName(coll, shard, replicaType);
     } else if (!skipCreateReplicaInClusterState) {
       //Validate that the core name is unique in that collection
       for (Slice slice : coll.getSlices()) {
@@ -126,7 +127,8 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
             ZkStateReader.CORE_NAME_PROP, coreName,
             ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
             ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(node),
-            ZkStateReader.NODE_NAME_PROP, node);
+            ZkStateReader.NODE_NAME_PROP, node,
+            ZkStateReader.REPLICA_TYPE, replicaType.name());
         Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(props));
       }
       params.set(CoreAdminParams.CORE_NODE_NAME,
@@ -142,6 +144,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     params.set(CoreAdminParams.NAME, coreName);
     params.set(COLL_CONF, configName);
     params.set(CoreAdminParams.COLLECTION, collection);
+    params.set(CoreAdminParams.REPLICA_TYPE, replicaType.name());
     if (shard != null) {
       params.set(CoreAdminParams.SHARD, shard);
     } else if (routeKey != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/Assign.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/Assign.java b/solr/core/src/java/org/apache/solr/cloud/Assign.java
index ba03ccd..265e453 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Assign.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Assign.java
@@ -107,12 +107,17 @@ public class Assign {
     returnShardId = shardIdNames.get(0);
     return returnShardId;
   }
+  
+  public static String buildCoreName(String collectionName, String shard, Replica.Type type, int replicaNum) {
+    // TODO: Adding the suffix is great for debugging, but may be an issue if at some point we want to support a way to change replica type
+    return String.format(Locale.ROOT, "%s_%s_replica_%s%s", collectionName, shard, type.name().substring(0,1).toLowerCase(Locale.ROOT), replicaNum);
+  }
 
-  static String buildCoreName(DocCollection collection, String shard) {
+  public static String buildCoreName(DocCollection collection, String shard, Replica.Type type) {
     Slice slice = collection.getSlice(shard);
     int replicaNum = slice.getReplicas().size();
     for (; ; ) {
-      String replicaName = collection.getName() + "_" + shard + "_replica" + replicaNum;
+      String replicaName = buildCoreName(collection.getName(), shard, type, replicaNum);
       boolean exists = false;
       for (Replica replica : slice.getReplicas()) {
         if (replicaName.equals(replica.getStr(CORE_NAME_PROP))) {
@@ -121,9 +126,8 @@ public class Assign {
         }
       }
       if (exists) replicaNum++;
-      else break;
+      else return replicaName;
     }
-    return collection.getName() + "_" + shard + "_replica" + replicaNum;
   }
 
   static class ReplicaCount {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java b/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
index 719b1d1..32cb65b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
@@ -20,12 +20,13 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 
-import com.google.common.base.Strings;
 import org.apache.solr.common.StringUtils;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.util.PropertiesUtil;
 
+import com.google.common.base.Strings;
+
 public class CloudDescriptor {
 
   private final CoreDescriptor cd;
@@ -44,6 +45,13 @@ public class CloudDescriptor {
   volatile Replica.State lastPublished = Replica.State.ACTIVE;
 
   public static final String NUM_SHARDS = "numShards";
+  
+  public static final String REPLICA_TYPE = "replicaType";
+  
+  /**
+   * The type of replica this core hosts
+   */
+  private final Replica.Type replicaType;
 
   public CloudDescriptor(String coreName, Properties props, CoreDescriptor cd) {
     this.cd = cd;
@@ -57,7 +65,12 @@ public class CloudDescriptor {
     if (Strings.isNullOrEmpty(nodeName))
       this.nodeName = null;
     this.numShards = PropertiesUtil.toInteger(props.getProperty(CloudDescriptor.NUM_SHARDS), null);
-
+    String replicaTypeStr = props.getProperty(CloudDescriptor.REPLICA_TYPE);
+    if (Strings.isNullOrEmpty(replicaTypeStr)) {
+      this.replicaType = Replica.Type.NRT;
+    } else {
+      this.replicaType = Replica.Type.valueOf(replicaTypeStr);
+    }
     for (String propName : props.stringPropertyNames()) {
       if (propName.startsWith(ZkController.COLLECTION_PARAM_PREFIX)) {
         collectionParams.put(propName.substring(ZkController.COLLECTION_PARAM_PREFIX.length()), props.getProperty(propName));
@@ -65,6 +78,10 @@ public class CloudDescriptor {
     }
   }
   
+  public boolean requiresTransactionLog() {
+    return this.replicaType != Replica.Type.PULL;
+  }
+  
   public Replica.State getLastPublished() {
     return lastPublished;
   }
@@ -155,4 +172,8 @@ public class CloudDescriptor {
       collectionParams.put(ent.getKey(), ent.getValue());
     }
   }
+
+  public Replica.Type getReplicaType() {
+    return replicaType;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
index a1bb70e..3d1a54e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CreateCollectionCmd.java
@@ -60,8 +60,7 @@ import static org.apache.solr.cloud.OverseerCollectionMessageHandler.COLL_CONF;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.CREATE_NODE_SET;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.RANDOM;
-import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
+import static org.apache.solr.common.cloud.ZkStateReader.*;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 import static org.apache.solr.common.params.CommonParams.NAME;
@@ -96,7 +95,9 @@ public class CreateCollectionCmd implements Cmd {
       // look at the replication factor and see if it matches reality
       // if it does not, find best nodes to create more cores
 
-      int repFactor = message.getInt(REPLICATION_FACTOR, 1);
+      int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, 1));
+      int numPullReplicas = message.getInt(PULL_REPLICAS, 0);
+      int numTlogReplicas = message.getInt(TLOG_REPLICAS, 0);
 
       ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
       final String async = message.getStr(ASYNC);
@@ -116,8 +117,8 @@ public class CreateCollectionCmd implements Cmd {
 
       int maxShardsPerNode = message.getInt(MAX_SHARDS_PER_NODE, 1);
 
-      if (repFactor <= 0) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, REPLICATION_FACTOR + " must be greater than 0");
+      if (numNrtReplicas + numTlogReplicas <= 0) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
       }
 
       if (numSlices <= 0) {
@@ -135,32 +136,33 @@ public class CreateCollectionCmd implements Cmd {
 
         positionVsNodes = new HashMap<>();
       } else {
-        if (repFactor > nodeList.size()) {
-          log.warn("Specified "
-              + REPLICATION_FACTOR
-              + " of "
-              + repFactor
+        int totalNumReplicas = numNrtReplicas + numTlogReplicas + numPullReplicas;
+        if (totalNumReplicas > nodeList.size()) {
+          log.warn("Specified number of replicas of "
+              + totalNumReplicas
               + " on collection "
               + collectionName
-              + " is higher than or equal to the number of Solr instances currently live or live and part of your " + CREATE_NODE_SET + "("
+              + " is higher than the number of Solr instances currently live or live and part of your " + CREATE_NODE_SET + "("
               + nodeList.size()
               + "). It's unusual to run two replica of the same slice on the same Solr-instance.");
         }
 
         int maxShardsAllowedToCreate = maxShardsPerNode * nodeList.size();
-        int requestedShardsToCreate = numSlices * repFactor;
+        int requestedShardsToCreate = numSlices * totalNumReplicas;
         if (maxShardsAllowedToCreate < requestedShardsToCreate) {
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cannot create collection " + collectionName + ". Value of "
               + MAX_SHARDS_PER_NODE + " is " + maxShardsPerNode
               + ", and the number of nodes currently live or live and part of your "+CREATE_NODE_SET+" is " + nodeList.size()
               + ". This allows a maximum of " + maxShardsAllowedToCreate
               + " to be created. Value of " + NUM_SLICES + " is " + numSlices
-              + " and value of " + REPLICATION_FACTOR + " is " + repFactor
+              + ", value of " + NRT_REPLICAS + " is " + numNrtReplicas
+              + ", value of " + TLOG_REPLICAS + " is " + numTlogReplicas
+              + " and value of " + PULL_REPLICAS + " is " + numPullReplicas
               + ". This requires " + requestedShardsToCreate
               + " shards to be created (higher than the allowed number)");
         }
 
-        positionVsNodes = ocmh.identifyNodes(clusterState, nodeList, message, shardNames, repFactor);
+        positionVsNodes = ocmh.identifyNodes(clusterState, nodeList, message, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas);
       }
 
       ZkStateReader zkStateReader = ocmh.zkStateReader;
@@ -200,13 +202,13 @@ public class CreateCollectionCmd implements Cmd {
       Map<String, String> requestMap = new HashMap<>();
 
 
-      log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} , replicationFactor : {2}",
-          collectionName, shardNames, repFactor));
+      log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} , nrtReplicas : {2}, tlogReplicas: {3}, pullReplicas: {4}",
+          collectionName, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas));
       Map<String,ShardRequest> coresToCreate = new LinkedHashMap<>();
       for (Map.Entry<ReplicaAssigner.Position, String> e : positionVsNodes.entrySet()) {
         ReplicaAssigner.Position position = e.getKey();
         String nodeName = e.getValue();
-        String coreName = collectionName + "_" + position.shard + "_replica" + (position.index + 1);
+        String coreName = Assign.buildCoreName(collectionName, position.shard, position.type, position.index + 1);
         log.debug(formatString("Creating core {0} as part of shard {1} of collection {2} on {3}"
             , coreName, position.shard, collectionName, nodeName));
 
@@ -221,7 +223,8 @@ public class CreateCollectionCmd implements Cmd {
               ZkStateReader.SHARD_ID_PROP, position.shard,
               ZkStateReader.CORE_NAME_PROP, coreName,
               ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
-              ZkStateReader.BASE_URL_PROP, baseUrl);
+              ZkStateReader.BASE_URL_PROP, baseUrl, 
+              ZkStateReader.REPLICA_TYPE, position.type.name());
           Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(props));
         }
 
@@ -235,6 +238,7 @@ public class CreateCollectionCmd implements Cmd {
         params.set(CoreAdminParams.SHARD, position.shard);
         params.set(ZkStateReader.NUM_SHARDS_PROP, numSlices);
         params.set(CoreAdminParams.NEW_COLLECTION, "true");
+        params.set(CoreAdminParams.REPLICA_TYPE, position.type.name());
 
         if (async != null) {
           String coreAdminAsyncId = async + Math.abs(System.nanoTime());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
index 52df32b..d3eb828 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CreateShardCmd.java
@@ -28,6 +28,7 @@ import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
@@ -41,7 +42,10 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.cloud.Assign.getNodesForNewReplicas;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.COLL_CONF;
+import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
@@ -67,9 +71,18 @@ public class CreateShardCmd implements Cmd {
 
     ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
     DocCollection collection = clusterState.getCollection(collectionName);
-    int repFactor = message.getInt(REPLICATION_FACTOR, collection.getInt(REPLICATION_FACTOR, 1));
+//    int repFactor = message.getInt(REPLICATION_FACTOR, collection.getInt(REPLICATION_FACTOR, 1));
+    int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, collection.getInt(NRT_REPLICAS, collection.getInt(REPLICATION_FACTOR, 1))));
+    int numPullReplicas = message.getInt(PULL_REPLICAS, collection.getInt(PULL_REPLICAS, 0));
+    int numTlogReplicas = message.getInt(TLOG_REPLICAS, collection.getInt(TLOG_REPLICAS, 0));
+    int totalReplicas = numNrtReplicas + numPullReplicas + numTlogReplicas;
+    
+    if (numNrtReplicas + numTlogReplicas <= 0) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
+    }
+    
     Object createNodeSetStr = message.get(OverseerCollectionMessageHandler.CREATE_NODE_SET);
-    List<Assign.ReplicaCount> sortedNodeList = getNodesForNewReplicas(clusterState, collectionName, sliceName, repFactor,
+    List<Assign.ReplicaCount> sortedNodeList = getNodesForNewReplicas(clusterState, collectionName, sliceName, totalReplicas,
         createNodeSetStr, ocmh.overseer.getZkController().getCoreContainer());
 
     ZkStateReader zkStateReader = ocmh.zkStateReader;
@@ -90,19 +103,38 @@ public class CreateShardCmd implements Cmd {
     String async = message.getStr(ASYNC);
     Map<String, String> requestMap = null;
     if (async != null) {
-      requestMap = new HashMap<>(repFactor, 1.0f);
+      requestMap = new HashMap<>(totalReplicas, 1.0f);
     }
-
-    for (int j = 1; j <= repFactor; j++) {
+    
+    int createdNrtReplicas = 0, createdTlogReplicas = 0, createdPullReplicas = 0;
+
+    for (int j = 1; j <= totalReplicas; j++) {
+      int coreNameNumber;
+      Replica.Type typeToCreate;
+      if (createdNrtReplicas < numNrtReplicas) {
+        createdNrtReplicas++;
+        coreNameNumber = createdNrtReplicas;
+        typeToCreate = Replica.Type.NRT;
+      } else if (createdTlogReplicas < numTlogReplicas) {
+        createdTlogReplicas++;
+        coreNameNumber = createdTlogReplicas;
+        typeToCreate = Replica.Type.TLOG;
+      } else {
+        createdPullReplicas++;
+        coreNameNumber = createdPullReplicas;
+        typeToCreate = Replica.Type.PULL;
+      }
       String nodeName = sortedNodeList.get(((j - 1)) % sortedNodeList.size()).nodeName;
-      String shardName = collectionName + "_" + sliceName + "_replica" + j;
-      log.info("Creating shard " + shardName + " as part of slice " + sliceName + " of collection " + collectionName
+      String coreName = Assign.buildCoreName(collectionName, sliceName, typeToCreate, coreNameNumber);
+//      String coreName = collectionName + "_" + sliceName + "_replica" + j;
+      log.info("Creating replica " + coreName + " as part of slice " + sliceName + " of collection " + collectionName
           + " on " + nodeName);
 
       // Need to create new params for each request
       ModifiableSolrParams params = new ModifiableSolrParams();
       params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.CREATE.toString());
-      params.set(CoreAdminParams.NAME, shardName);
+      params.set(CoreAdminParams.NAME, coreName);
+      params.set(CoreAdminParams.REPLICA_TYPE, typeToCreate.name());
       params.set(COLL_CONF, configName);
       params.set(CoreAdminParams.COLLECTION, collectionName);
       params.set(CoreAdminParams.SHARD, sliceName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
index bdbeca9..588262d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
@@ -20,6 +20,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
@@ -326,6 +327,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
         return;
       }
       
+      Replica.Type replicaType;
+      
       try (SolrCore core = cc.getCore(coreName)) {
         
         if (core == null) {
@@ -338,6 +341,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
           }
         }
         
+        replicaType = core.getCoreDescriptor().getCloudDescriptor().getReplicaType();
+        
         // should I be leader?
         if (weAreReplacement && !shouldIBeLeader(leaderProps, core, weAreReplacement)) {
           rejoinLeaderElection(core);
@@ -423,9 +428,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
         try {
           // we must check LIR before registering as leader
           checkLIR(coreName, allReplicasInLine);
-
-          boolean onlyLeaderIndexes = zkController.getClusterState().getCollection(collection).getRealtimeReplicas() == 1;
-          if (onlyLeaderIndexes) {
+          if (replicaType == Replica.Type.TLOG) {
             // stop replicate from old leader
             zkController.stopReplicationFromLeader(coreName);
             if (weAreReplacement) {
@@ -621,7 +624,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
         }
         
         // on startup and after connection timeout, wait for all known shards
-        if (found >= slices.getReplicasMap().size()) {
+        if (found >= slices.getReplicas(EnumSet.of(Replica.Type.TLOG, Replica.Type.NRT)).size()) {
           log.info("Enough replicas found to continue.");
           return true;
         } else {
@@ -629,7 +632,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
             log.info("Waiting until we see more replicas up for shard {}: total={}"
               + " found={}"
               + " timeoutin={}ms",
-                shardId, slices.getReplicasMap().size(), found,
+                shardId, slices.getReplicas(EnumSet.of(Replica.Type.TLOG, Replica.Type.NRT)).size(), found,
                 TimeUnit.MILLISECONDS.convert(timeoutAt - System.nanoTime(), TimeUnit.NANOSECONDS));
           }
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java b/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java
index 7b1ad2c..0ea5d6e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/MigrateCmd.java
@@ -51,7 +51,7 @@ import static org.apache.solr.cloud.OverseerCollectionMessageHandler.COLL_PROP_P
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.CREATE_NODE_SET;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
+import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
@@ -208,7 +208,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
     Map<String, Object> props = makeMap(
         Overseer.QUEUE_OPERATION, CREATE.toLower(),
         NAME, tempSourceCollectionName,
-        REPLICATION_FACTOR, 1,
+        NRT_REPLICAS, 1,
         NUM_SLICES, 1,
         COLL_CONF, configName,
         CREATE_NODE_SET, sourceLeader.getNodeName());
@@ -224,7 +224,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
     Slice tempSourceSlice = clusterState.getCollection(tempSourceCollectionName).getSlices().iterator().next();
     Replica tempSourceLeader = zkStateReader.getLeaderRetry(tempSourceCollectionName, tempSourceSlice.getName(), 120000);
 
-    String tempCollectionReplica1 = tempSourceCollectionName + "_" + tempSourceSlice.getName() + "_replica1";
+    String tempCollectionReplica1 = Assign.buildCoreName(tempSourceCollectionName, tempSourceSlice.getName(), Replica.Type.NRT, 1);
     String coreNodeName = ocmh.waitForCoreNodeName(tempSourceCollectionName,
         sourceLeader.getNodeName(), tempCollectionReplica1);
     // wait for the replicas to be seen as active on temp source leader
@@ -257,7 +257,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
 
     log.info("Creating a replica of temporary collection: {} on the target leader node: {}",
         tempSourceCollectionName, targetLeader.getNodeName());
-    String tempCollectionReplica2 = tempSourceCollectionName + "_" + tempSourceSlice.getName() + "_replica2";
+    String tempCollectionReplica2 = Assign.buildCoreName(tempSourceCollectionName, tempSourceSlice.getName(), Replica.Type.NRT, 2); 
     props = new HashMap<>();
     props.put(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower());
     props.put(COLLECTION_PROP, tempSourceCollectionName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
index 545989e..fed1398 100644
--- a/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
@@ -111,7 +111,7 @@ public class MoveReplicaCmd implements Cmd{
 
   private void moveHdfsReplica(ClusterState clusterState, NamedList results, String dataDir, String targetNode, String async,
                                  DocCollection coll, Replica replica, Slice slice) throws Exception {
-    String newCoreName = Assign.buildCoreName(coll, slice.getName());
+    String newCoreName = Assign.buildCoreName(coll, slice.getName(), replica.getType());
 
     ZkNodeProps removeReplicasProps = new ZkNodeProps(
         COLLECTION_PROP, coll.getName(),
@@ -155,7 +155,7 @@ public class MoveReplicaCmd implements Cmd{
 
   private void moveNormalReplica(ClusterState clusterState, NamedList results, String targetNode, String async,
                                  DocCollection coll, Replica replica, Slice slice) throws Exception {
-    String newCoreName = Assign.buildCoreName(coll, slice.getName());
+    String newCoreName = Assign.buildCoreName(coll, slice.getName(), replica.getType());
     ZkNodeProps addReplicasProps = new ZkNodeProps(
         COLLECTION_PROP, coll.getName(),
         SHARD_ID_PROP, slice.getName(),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
index 9b83140..1d51df7 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
@@ -16,6 +16,51 @@
  */
 package org.apache.solr.cloud;
 
+import static org.apache.solr.common.cloud.DocCollection.SNITCH;
+import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICAPROP;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDROLE;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.BACKUP;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESHARDUNIQUE;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATEALIAS;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESNAPSHOT;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETE;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEALIAS;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETENODE;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICA;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICAPROP;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESHARD;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESNAPSHOT;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.MIGRATE;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.MIGRATESTATEFORMAT;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOCK_COLL_TASK;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOCK_REPLICA_TASK;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOCK_SHARD_TASK;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.MODIFYCOLLECTION;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.OVERSEERSTATUS;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.REBALANCELEADERS;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.RELOAD;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.REMOVEROLE;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.REPLACENODE;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.RESTORE;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.SPLITSHARD;
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+import static org.apache.solr.common.params.CommonParams.NAME;
+import static org.apache.solr.common.util.Utils.makeMap;
+
 import java.io.Closeable;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
@@ -33,7 +78,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.collect.ImmutableMap;
 import org.apache.commons.lang.StringUtils;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -80,21 +124,7 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.common.cloud.DocCollection.SNITCH;
-import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
-import static org.apache.solr.common.params.CollectionParams.CollectionAction.*;
-import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
-import static org.apache.solr.common.params.CommonParams.NAME;
-import static org.apache.solr.common.util.Utils.makeMap;
+import com.google.common.collect.ImmutableMap;
 
 /**
  * A {@link OverseerMessageHandler} that handles Collections API related
@@ -130,9 +160,11 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
   public static final Map<String, Object> COLL_PROPS = Collections.unmodifiableMap(makeMap(
       ROUTER, DocRouter.DEFAULT_NAME,
       ZkStateReader.REPLICATION_FACTOR, "1",
+      ZkStateReader.NRT_REPLICAS, "1",
+      ZkStateReader.TLOG_REPLICAS, "0",
+      ZkStateReader.PULL_REPLICAS, "0",
       ZkStateReader.MAX_SHARDS_PER_NODE, "1",
       ZkStateReader.AUTO_ADD_REPLICAS, "false",
-      ZkStateReader.REALTIME_REPLICAS, "-1",
       DocCollection.RULE, null,
       SNITCH, null));
 
@@ -702,18 +734,33 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
                                       List<String> nodeList,
                                       ZkNodeProps message,
                                       List<String> shardNames,
-                                      int repFactor) throws IOException {
+                                      int numNrtReplicas, 
+                                      int numTlogReplicas,
+                                      int numPullReplicas) throws IOException {
     List<Map> rulesMap = (List) message.get("rule");
     if (rulesMap == null) {
       int i = 0;
       Map<Position, String> result = new HashMap<>();
       for (String aShard : shardNames) {
-        for (int j = 0; j < repFactor; j++){
-          result.put(new Position(aShard, j), nodeList.get(i % nodeList.size()));
+        for (int j = 0; j < numNrtReplicas; j++){
+          result.put(new Position(aShard, j, Replica.Type.NRT), nodeList.get(i % nodeList.size()));
+          i++;
+        }
+        for (int j = 0; j < numTlogReplicas; j++){
+          result.put(new Position(aShard, j, Replica.Type.TLOG), nodeList.get(i % nodeList.size()));
+          i++;
+        }
+        for (int j = 0; j < numPullReplicas; j++){
+          result.put(new Position(aShard, j, Replica.Type.PULL), nodeList.get(i % nodeList.size()));
           i++;
         }
       }
       return result;
+    } else {
+      if (numTlogReplicas + numPullReplicas != 0) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, 
+            Replica.Type.TLOG + " or " + Replica.Type.PULL + " replica types not supported with placement rules");
+      }
     }
 
     List<Rule> rules = new ArrayList<>();
@@ -721,7 +768,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
 
     Map<String, Integer> sharVsReplicaCount = new HashMap<>();
 
-    for (String shard : shardNames) sharVsReplicaCount.put(shard, repFactor);
+    for (String shard : shardNames) sharVsReplicaCount.put(shard, numNrtReplicas);
     ReplicaAssigner replicaAssigner = new ReplicaAssigner(rules,
         sharVsReplicaCount,
         (List<Map>) message.get(SNITCH),
@@ -752,6 +799,8 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
       
       if (result.size() == coreNames.size()) {
         return result;
+      } else {
+        log.debug("Expecting {} cores but found {}", coreNames.size(), result.size());
       }
       if (timeout.hasTimedOut()) {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Timed out waiting to see all replicas: " + coreNames + " in cluster state.");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index 7599b05..3449935 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -118,7 +118,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
   private boolean recoveringAfterStartup;
   private CoreContainer cc;
   private volatile HttpUriRequest prevSendPreRecoveryHttpUriRequest;
-  private boolean onlyLeaderIndexes;
+  private final Replica.Type replicaType;
 
   protected RecoveryStrategy(CoreContainer cc, CoreDescriptor cd, RecoveryListener recoveryListener) {
     this.cc = cc;
@@ -128,8 +128,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
     zkStateReader = zkController.getZkStateReader();
     baseUrl = zkController.getBaseUrl();
     coreZkNodeName = cd.getCloudDescriptor().getCoreNodeName();
-    String collection = cd.getCloudDescriptor().getCollectionName();
-    onlyLeaderIndexes = zkStateReader.getClusterState().getCollection(collection).getRealtimeReplicas() == 1;
+    replicaType = cd.getCloudDescriptor().getReplicaType();
   }
 
   final public int getWaitForUpdatesWithStaleStatePauseMilliSeconds() {
@@ -262,7 +261,8 @@ public class RecoveryStrategy implements Runnable, Closeable {
       UpdateRequest ureq = new UpdateRequest();
       ureq.setParams(new ModifiableSolrParams());
       ureq.getParams().set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
-      ureq.getParams().set(UpdateParams.OPEN_SEARCHER, onlyLeaderIndexes);
+//      ureq.getParams().set(UpdateParams.OPEN_SEARCHER, onlyLeaderIndexes);// Why do we need to open searcher if "onlyLeaderIndexes"?
+      ureq.getParams().set(UpdateParams.OPEN_SEARCHER, false);
       ureq.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, true).process(
           client);
     }
@@ -296,9 +296,151 @@ public class RecoveryStrategy implements Runnable, Closeable {
       MDCLoggingContext.clear();
     }
   }
+  
+  final public void doRecovery(SolrCore core) throws KeeperException, InterruptedException {
+    if (core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog()) {
+      doSyncOrReplicateRecovery(core);
+    } else {
+      doReplicateOnlyRecovery(core);
+    }
+  }
+
+  final private void doReplicateOnlyRecovery(SolrCore core) throws InterruptedException {
+    boolean successfulRecovery = false;
+
+//  if (core.getUpdateHandler().getUpdateLog() != null) {
+//    SolrException.log(LOG, "'replicate-only' recovery strategy should only be used if no update logs are present, but this core has one: "
+//        + core.getUpdateHandler().getUpdateLog());
+//    return;
+//  }
+  while (!successfulRecovery && !Thread.currentThread().isInterrupted() && !isClosed()) { // don't use interruption or it will close channels though
+    try {
+      CloudDescriptor cloudDesc = core.getCoreDescriptor().getCloudDescriptor();
+      ZkNodeProps leaderprops = zkStateReader.getLeaderRetry(
+          cloudDesc.getCollectionName(), cloudDesc.getShardId());
+      final String leaderBaseUrl = leaderprops.getStr(ZkStateReader.BASE_URL_PROP);
+      final String leaderCoreName = leaderprops.getStr(ZkStateReader.CORE_NAME_PROP);
+
+      String leaderUrl = ZkCoreNodeProps.getCoreUrl(leaderBaseUrl, leaderCoreName);
+
+      String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
+
+      boolean isLeader = leaderUrl.equals(ourUrl); //TODO: We can probably delete most of this code if we say this strategy can only be used for pull replicas
+      if (isLeader && !cloudDesc.isLeader()) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Cloud state still says we are leader.");
+      }
+      if (cloudDesc.isLeader()) {
+        assert cloudDesc.getReplicaType() != Replica.Type.PULL;
+        // we are now the leader - no one else must have been suitable
+        LOG.warn("We have not yet recovered - but we are now the leader!");
+        LOG.info("Finished recovery process.");
+        zkController.publish(core.getCoreDescriptor(), Replica.State.ACTIVE);
+        return;
+      }
+      
+      
+      LOG.info("Publishing state of core [{}] as recovering, leader is [{}] and I am [{}]", core.getName(), leaderUrl,
+          ourUrl);
+      zkController.publish(core.getCoreDescriptor(), Replica.State.RECOVERING);
+      
+      if (isClosed()) {
+        LOG.info("Recovery for core {} has been closed", core.getName());
+        break;
+      }
+      LOG.info("Starting Replication Recovery.");
+
+      try {
+        LOG.info("Stopping background replicate from leader process");
+        zkController.stopReplicationFromLeader(coreName);
+        replicate(zkController.getNodeName(), core, leaderprops);
+
+        if (isClosed()) {
+          LOG.info("Recovery for core {} has been closed", core.getName());
+          break;
+        }
+
+        LOG.info("Replication Recovery was successful.");
+        successfulRecovery = true;
+      } catch (Exception e) {
+        SolrException.log(LOG, "Error while trying to recover", e);
+      }
+
+    } catch (Exception e) {
+      SolrException.log(LOG, "Error while trying to recover. core=" + coreName, e);
+    } finally {
+      if (successfulRecovery) {
+        LOG.info("Restaring background replicate from leader process");
+        zkController.startReplicationFromLeader(coreName, false);
+        LOG.info("Registering as Active after recovery.");
+        try {
+          zkController.publish(core.getCoreDescriptor(), Replica.State.ACTIVE);
+        } catch (Exception e) {
+          LOG.error("Could not publish as ACTIVE after succesful recovery", e);
+          successfulRecovery = false;
+        }
+        
+        if (successfulRecovery) {
+          close = true;
+          recoveryListener.recovered();
+        }
+      }
+    }
+
+    if (!successfulRecovery) {
+      // lets pause for a moment and we need to try again...
+      // TODO: we don't want to retry for some problems?
+      // Or do a fall off retry...
+      try {
+
+        if (isClosed()) {
+          LOG.info("Recovery for core {} has been closed", core.getName());
+          break;
+        }
+        
+        LOG.error("Recovery failed - trying again... (" + retries + ")");
+        
+        retries++;
+        if (retries >= maxRetries) {
+          SolrException.log(LOG, "Recovery failed - max retries exceeded (" + retries + ").");
+          try {
+            recoveryFailed(core, zkController, baseUrl, coreZkNodeName, core.getCoreDescriptor());
+          } catch (Exception e) {
+            SolrException.log(LOG, "Could not publish that recovery failed", e);
+          }
+          break;
+        }
+      } catch (Exception e) {
+        SolrException.log(LOG, "An error has occurred during recovery", e);
+      }
+
+      try {
+        // Wait an exponential interval between retries, start at 5 seconds and work up to a minute.
+        // If we're at attempt >= 4, there's no point computing pow(2, retries) because the result 
+        // will always be the minimum of the two (12). Since we sleep at 5 seconds sub-intervals in
+        // order to check if we were closed, 12 is chosen as the maximum loopCount (5s * 12 = 1m).
+        double loopCount = retries < 4 ? Math.min(Math.pow(2, retries), 12) : 12;
+        LOG.info("Wait [{}] seconds before trying to recover again (attempt={})", loopCount, retries);
+        for (int i = 0; i < loopCount; i++) {
+          if (isClosed()) {
+            LOG.info("Recovery for core {} has been closed", core.getName());
+            break; // check if someone closed us
+          }
+          Thread.sleep(startingRecoveryDelayMilliSeconds);
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        LOG.warn("Recovery was interrupted.", e);
+        close = true;
+      }
+    }
+
+  }
+  // We skip core.seedVersionBuckets(); We don't have a transaction log
+  LOG.info("Finished recovery process, successful=[{}]", Boolean.toString(successfulRecovery));
+}
 
   // TODO: perhaps make this grab a new core each time through the loop to handle core reloads?
-  final public void doRecovery(SolrCore core) throws KeeperException, InterruptedException {
+  final public void doSyncOrReplicateRecovery(SolrCore core) throws KeeperException, InterruptedException {
     boolean replayed = false;
     boolean successfulRecovery = false;
 
@@ -310,9 +452,9 @@ public class RecoveryStrategy implements Runnable, Closeable {
           core.getCoreDescriptor());
       return;
     }
-
-    // we temporary ignore peersync for realtimeReplicas mode
-    boolean firstTime = !onlyLeaderIndexes;
+    
+    // we temporary ignore peersync for tlog replicas
+    boolean firstTime = replicaType != Replica.Type.TLOG;
 
     List<Long> recentVersions;
     try (UpdateLog.RecentUpdates recentUpdates = ulog.getRecentUpdates()) {
@@ -364,7 +506,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
       }
     }
 
-    if (onlyLeaderIndexes) {
+    if (replicaType == Replica.Type.TLOG) {
       zkController.stopReplicationFromLeader(coreName);
     }
 
@@ -521,8 +663,8 @@ public class RecoveryStrategy implements Runnable, Closeable {
         if (successfulRecovery) {
           LOG.info("Registering as Active after recovery.");
           try {
-            if (onlyLeaderIndexes) {
-              zkController.startReplicationFromLeader(coreName);
+            if (replicaType == Replica.Type.TLOG) {
+              zkController.startReplicationFromLeader(coreName, true);
             }
             zkController.publish(core.getCoreDescriptor(), Replica.State.ACTIVE);
           } catch (Exception e) {
@@ -604,7 +746,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
     if (testing_beforeReplayBufferingUpdates != null) {
       testing_beforeReplayBufferingUpdates.run();
     }
-    if (onlyLeaderIndexes) {
+    if (replicaType == Replica.Type.TLOG) {
       // roll over all updates during buffering to new tlog, make RTG available
       SolrQueryRequest req = new LocalSolrQueryRequest(core,
           new ModifiableSolrParams());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java b/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java
index 92c9afe..e4240be 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java
@@ -154,9 +154,9 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
                 SHARD_ID_PROP, slice.getName(),
                 ZkStateReader.CORE_NAME_PROP, replica.getCoreName(),
                 ZkStateReader.REPLICA_PROP, replica.getName(),
+                ZkStateReader.REPLICA_TYPE, replica.getType().name(),
                 CoreAdminParams.NODE, source);
-            sourceReplicas.add(props
-            );
+            sourceReplicas.add(props);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java b/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java
index 817b371..0800e0f 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java
@@ -49,7 +49,12 @@ public class ReplicateFromLeader {
     this.coreName = coreName;
   }
 
-  public void startReplication() throws InterruptedException {
+  /**
+   * Start a replication handler thread that will periodically pull indices from the shard leader
+   * @param switchTransactionLog if true, ReplicationHandler will rotate the transaction log once
+   * the replication is done
+   */
+  public void startReplication(boolean switchTransactionLog) throws InterruptedException {
     try (SolrCore core = cc.getCore(coreName)) {
       if (core == null) {
         if (cc.isShutDown()) {
@@ -65,6 +70,7 @@ public class ReplicateFromLeader {
       } else if (uinfo.autoSoftCommmitMaxTime != -1) {
         pollIntervalStr = toPollIntervalStr(uinfo.autoSoftCommmitMaxTime/2);
       }
+      LOG.info("Will start replication from leader with poll interval: {}", pollIntervalStr );
 
       NamedList slaveConfig = new NamedList();
       slaveConfig.add("fetchFromLeader", true);
@@ -78,20 +84,22 @@ public class ReplicateFromLeader {
       }
 
       replicationProcess = new ReplicationHandler();
-      replicationProcess.setPollListener((solrCore, pollSuccess) -> {
-        if (pollSuccess) {
-          String commitVersion = getCommitVersion(core);
-          if (commitVersion == null) return;
-          if (Long.parseLong(commitVersion) == lastVersion) return;
-          UpdateLog updateLog = solrCore.getUpdateHandler().getUpdateLog();
-          SolrQueryRequest req = new LocalSolrQueryRequest(core,
-              new ModifiableSolrParams());
-          CommitUpdateCommand cuc = new CommitUpdateCommand(req, false);
-          cuc.setVersion(Long.parseLong(commitVersion));
-          updateLog.copyOverOldUpdates(cuc);
-          lastVersion = Long.parseLong(commitVersion);
-        }
-      });
+      if (switchTransactionLog) {
+        replicationProcess.setPollListener((solrCore, pollSuccess) -> {
+          if (pollSuccess) {
+            String commitVersion = getCommitVersion(core);
+            if (commitVersion == null) return;
+            if (Long.parseLong(commitVersion) == lastVersion) return;
+            UpdateLog updateLog = solrCore.getUpdateHandler().getUpdateLog();
+            SolrQueryRequest req = new LocalSolrQueryRequest(core,
+                new ModifiableSolrParams());
+            CommitUpdateCommand cuc = new CommitUpdateCommand(req, false);
+            cuc.setVersion(Long.parseLong(commitVersion));
+            updateLog.copyOverOldUpdates(cuc);
+            lastVersion = Long.parseLong(commitVersion);
+          }
+        });
+      }
       replicationProcess.init(replicationConfig);
       replicationProcess.inform(core);
     }
@@ -119,6 +127,8 @@ public class ReplicateFromLeader {
   }
 
   public void stopReplication() {
-    replicationProcess.close();
+    if (replicationProcess != null) {
+      replicationProcess.close();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java b/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java
index 23b2fb5..76c12b8 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RestoreCmd.java
@@ -39,6 +39,7 @@ import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.ImplicitDocRouter;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -63,11 +64,15 @@ import static org.apache.solr.cloud.OverseerCollectionMessageHandler.RANDOM;
 import static org.apache.solr.cloud.OverseerCollectionMessageHandler.SHARDS_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_TYPE;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
+import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
 
 public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
@@ -107,15 +112,23 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
         zkStateReader.getClusterState().getLiveNodes(), message, RANDOM);
 
     int numShards = backupCollectionState.getActiveSlices().size();
-    int repFactor = message.getInt(REPLICATION_FACTOR, backupCollectionState.getReplicationFactor());
+    
+    int numNrtReplicas = getInt(message, NRT_REPLICAS, backupCollectionState.getNumNrtReplicas(), 0);
+    if (numNrtReplicas == 0) {
+      numNrtReplicas = getInt(message, REPLICATION_FACTOR, backupCollectionState.getReplicationFactor(), 0);
+    }
+    int numTlogReplicas = getInt(message, TLOG_REPLICAS, backupCollectionState.getNumTlogReplicas(), 0);
+    int numPullReplicas = getInt(message, PULL_REPLICAS, backupCollectionState.getNumPullReplicas(), 0);
+    int totalReplicasPerShard = numNrtReplicas + numTlogReplicas + numPullReplicas;
+    
     int maxShardsPerNode = message.getInt(MAX_SHARDS_PER_NODE, backupCollectionState.getMaxShardsPerNode());
     int availableNodeCount = nodeList.size();
-    if ((numShards * repFactor) > (availableNodeCount * maxShardsPerNode)) {
+    if ((numShards * totalReplicasPerShard) > (availableNodeCount * maxShardsPerNode)) {
       throw new SolrException(ErrorCode.BAD_REQUEST,
           String.format(Locale.ROOT, "Solr cloud with available number of nodes:%d is insufficient for"
-              + " restoring a collection with %d shards, replication factor %d and maxShardsPerNode %d."
+              + " restoring a collection with %d shards, total replicas per shard %d and maxShardsPerNode %d."
               + " Consider increasing maxShardsPerNode value OR number of available nodes.",
-              availableNodeCount, numShards, repFactor, maxShardsPerNode));
+              availableNodeCount, numShards, totalReplicasPerShard, maxShardsPerNode));
     }
 
     //Upload the configs
@@ -201,7 +214,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
     restoreCollection.getSlices().forEach(x -> sliceNames.add(x.getName()));
 
     Map<ReplicaAssigner.Position, String> positionVsNodes = ocmh.identifyNodes(clusterState, nodeList,
-        message, sliceNames, repFactor);
+        message, sliceNames, numNrtReplicas, numTlogReplicas, numPullReplicas);
 
     //Create one replica per shard and copy backed up data to it
     for (Slice slice : restoreCollection.getSlices()) {
@@ -210,6 +223,15 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
       propMap.put(Overseer.QUEUE_OPERATION, CREATESHARD);
       propMap.put(COLLECTION_PROP, restoreCollectionName);
       propMap.put(SHARD_ID_PROP, slice.getName());
+      
+      if (numNrtReplicas >= 1) {
+        propMap.put(REPLICA_TYPE, Replica.Type.NRT.name());
+      } else if (numTlogReplicas >= 1) {
+        propMap.put(REPLICA_TYPE, Replica.Type.TLOG.name());
+      } else {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unexpected number of replicas, replicationFactor, " + 
+            Replica.Type.NRT + " or " + Replica.Type.TLOG + " must be greater than 0");
+      }
 
       // Get the first node matching the shard to restore in
       String node;
@@ -261,17 +283,39 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
     //refresh the location copy of collection state
     restoreCollection = zkStateReader.getClusterState().getCollection(restoreCollectionName);
 
-    //Add the remaining replicas for each shard
-    Integer numReplicas = restoreCollection.getReplicationFactor();
-    if (numReplicas != null && numReplicas > 1) {
+    if (totalReplicasPerShard > 1) {
       log.info("Adding replicas to restored collection={}", restoreCollection);
-
       for (Slice slice : restoreCollection.getSlices()) {
-        for (int i = 1; i < numReplicas; i++) {
-          log.debug("Adding replica for shard={} collection={} ", slice.getName(), restoreCollection);
+
+        //Add the remaining replicas for each shard, considering it's type
+        int createdNrtReplicas = 0, createdTlogReplicas = 0, createdPullReplicas = 0;
+
+        // We already created either a NRT or an TLOG replica as leader
+        if (numNrtReplicas > 0) {
+          createdNrtReplicas++;
+        } else if (createdTlogReplicas > 0) {
+          createdTlogReplicas++;
+        }
+        
+        for (int i = 1; i < totalReplicasPerShard; i++) {
+          Replica.Type typeToCreate;
+          if (createdNrtReplicas < numNrtReplicas) {
+            createdNrtReplicas++;
+            typeToCreate = Replica.Type.NRT;
+          } else if (createdTlogReplicas < numTlogReplicas) {
+            createdTlogReplicas++;
+            typeToCreate = Replica.Type.TLOG;
+          } else {
+            createdPullReplicas++;
+            typeToCreate = Replica.Type.PULL;
+            assert createdPullReplicas <= numPullReplicas: "Unexpected number of replicas";
+          }
+
+          log.debug("Adding replica for shard={} collection={} of type {} ", slice.getName(), restoreCollection, typeToCreate);
           HashMap<String, Object> propMap = new HashMap<>();
           propMap.put(COLLECTION_PROP, restoreCollectionName);
           propMap.put(SHARD_ID_PROP, slice.getName());
+          propMap.put(REPLICA_TYPE, typeToCreate.name());
 
           // Get the first node matching the shard to restore in
           String node;
@@ -298,4 +342,9 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
 
     log.info("Completed restoring collection={} backupName={}", restoreCollection, backupName);
   }
+
+  private int getInt(ZkNodeProps message, String propertyName, Integer default1, int default2) {
+    Integer value = message.getInt(REPLICATION_FACTOR, default1);
+    return value!=null?value:default2;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java
index 5a099e1..fe95458 100644
--- a/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/SplitShardCmd.java
@@ -205,7 +205,7 @@ public class SplitShardCmd implements Cmd {
       for (int i = 0; i < subRanges.size(); i++) {
         String subSlice = slice + "_" + i;
         subSlices.add(subSlice);
-        String subShardName = collectionName + "_" + subSlice + "_replica1";
+        String subShardName = Assign.buildCoreName(collectionName, subSlice, Replica.Type.NRT, 1);
         subShardNames.add(subShardName);
       }
 
@@ -385,7 +385,7 @@ public class SplitShardCmd implements Cmd {
       Map<ReplicaAssigner.Position, String> nodeMap = ocmh.identifyNodes(clusterState,
           new ArrayList<>(clusterState.getLiveNodes()),
           new ZkNodeProps(collection.getProperties()),
-          subSlices, repFactor - 1);
+          subSlices, repFactor - 1, 0, 0);
 
       List<Map<String, Object>> replicas = new ArrayList<>((repFactor - 1) * 2);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/ZkController.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index 5e00c32..cb8175e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -62,6 +62,7 @@ import org.apache.solr.common.cloud.DefaultZkCredentialsProvider;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.OnReconnect;
 import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Replica.Type;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkACLProvider;
@@ -883,12 +884,20 @@ public class ZkController {
       try {
         // If we're a preferred leader, insert ourselves at the head of the queue
         boolean joinAtHead = false;
-        Replica replica = zkStateReader.getClusterState().getReplica(desc.getCloudDescriptor().getCollectionName(),
-            coreZkNodeName);
+        Replica replica = zkStateReader.getClusterState().getReplica(collection, coreZkNodeName);
         if (replica != null) {
           joinAtHead = replica.getBool(SliceMutator.PREFERRED_LEADER_PROP, false);
         }
-        joinElection(desc, afterExpiration, joinAtHead);
+        //TODO WHy would replica be null?
+        if (replica == null || replica.getType() != Type.PULL) {
+          joinElection(desc, afterExpiration, joinAtHead);
+        } else if (replica.getType() == Type.PULL) {
+          if (joinAtHead) {
+            log.warn("Replica {} was designated as preferred leader but it's type is {}, It won't join election", coreZkNodeName, Type.PULL);
+          }
+          log.debug("Replica {} skipping election because it's type is {}", coreZkNodeName, Type.PULL);
+          startReplicationFromLeader(coreName, false);
+        }
       } catch (InterruptedException e) {
         // Restore the interrupted status
         Thread.currentThread().interrupt();
@@ -905,6 +914,8 @@ public class ZkController {
       String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
       log.debug("We are " + ourUrl + " and leader is " + leaderUrl);
       boolean isLeader = leaderUrl.equals(ourUrl);
+      Replica.Type replicaType =  zkStateReader.getClusterState().getCollection(collection).getReplica(coreZkNodeName).getType();
+      assert !(isLeader && replicaType == Type.PULL): "Pull replica became leader!";
       
       try (SolrCore core = cc.getCore(desc.getName())) {
         
@@ -915,16 +926,15 @@ public class ZkController {
         // leader election perhaps?
         
         UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
-        boolean onlyLeaderIndexes = zkStateReader.getClusterState().getCollection(collection).getRealtimeReplicas() == 1;
-        boolean isReplicaInOnlyLeaderIndexes = onlyLeaderIndexes && !isLeader;
-        if (isReplicaInOnlyLeaderIndexes) {
+        boolean isTlogReplicaAndNotLeader = replicaType == Replica.Type.TLOG && !isLeader;
+        if (isTlogReplicaAndNotLeader) {
           String commitVersion = ReplicateFromLeader.getCommitVersion(core);
           if (commitVersion != null) {
             ulog.copyOverOldUpdates(Long.parseLong(commitVersion));
           }
         }
         // we will call register again after zk expiration and on reload
-        if (!afterExpiration && !core.isReloaded() && ulog != null && !isReplicaInOnlyLeaderIndexes) {
+        if (!afterExpiration && !core.isReloaded() && ulog != null && !isTlogReplicaAndNotLeader) {
           // disable recovery in case shard is in construction state (for shard splits)
           Slice slice = getClusterState().getSlice(collection, shardId);
           if (slice.getState() != Slice.State.CONSTRUCTION || !isLeader) {
@@ -943,12 +953,13 @@ public class ZkController {
         boolean didRecovery
             = checkRecovery(recoverReloadedCores, isLeader, skipRecovery, collection, coreZkNodeName, core, cc, afterExpiration);
         if (!didRecovery) {
-          if (isReplicaInOnlyLeaderIndexes) {
-            startReplicationFromLeader(coreName);
+          if (isTlogReplicaAndNotLeader) {
+            startReplicationFromLeader(coreName, true);
           }
           publish(desc, Replica.State.ACTIVE);
         }
         
+        
         core.getCoreDescriptor().getCloudDescriptor().setHasRegistered(true);
       }
       
@@ -960,17 +971,25 @@ public class ZkController {
     }
   }
 
-  public void startReplicationFromLeader(String coreName) throws InterruptedException {
+  public void startReplicationFromLeader(String coreName, boolean switchTransactionLog) throws InterruptedException {
+    log.info("{} starting background replication from leader", coreName);
     ReplicateFromLeader replicateFromLeader = new ReplicateFromLeader(cc, coreName);
-    if (replicateFromLeaders.putIfAbsent(coreName, replicateFromLeader) == null) {
-      replicateFromLeader.startReplication();
+    synchronized (replicateFromLeader) { // synchronize to prevent any stop before we finish the start
+      if (replicateFromLeaders.putIfAbsent(coreName, replicateFromLeader) == null) {
+        replicateFromLeader.startReplication(switchTransactionLog);
+      } else {
+        log.warn("A replicate from leader instance already exists for core {}", coreName);
+      }
     }
   }
 
   public void stopReplicationFromLeader(String coreName) {
+    log.info("{} stopping background replication from leader", coreName);
     ReplicateFromLeader replicateFromLeader = replicateFromLeaders.remove(coreName);
     if (replicateFromLeader != null) {
-      replicateFromLeader.stopReplication();
+      synchronized (replicateFromLeader) {
+        replicateFromLeader.stopReplication();
+      }
     }
   }
 
@@ -1191,6 +1210,7 @@ public class ZkController {
       if (state != Replica.State.DOWN) {
         final Replica.State lirState = getLeaderInitiatedRecoveryState(collection, shardId, coreNodeName);
         if (lirState != null) {
+          assert cd.getCloudDescriptor().getReplicaType() != Replica.Type.PULL: "LIR should not happen for pull replicas!";
           if (state == Replica.State.ACTIVE) {
             // trying to become active, so leader-initiated state must be recovering
             if (lirState == Replica.State.RECOVERING) {
@@ -1217,6 +1237,7 @@ public class ZkController {
       props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
       props.put(ZkStateReader.SHARD_ID_PROP, cd.getCloudDescriptor().getShardId());
       props.put(ZkStateReader.COLLECTION_PROP, collection);
+      props.put(ZkStateReader.REPLICA_TYPE, cd.getCloudDescriptor().getReplicaType().toString());
       if (numShards != null) {
         props.put(ZkStateReader.NUM_SHARDS_PROP, numShards.toString());
       }
@@ -1272,13 +1293,15 @@ public class ZkController {
       assert false : "No collection was specified [" + collection + "]";
       return;
     }
+    Replica replica = zkStateReader.getClusterState().getReplica(collection, coreNodeName);
+    
+    if (replica == null || replica.getType() != Type.PULL) {
+      ElectionContext context = electionContexts.remove(new ContextKey(collection, coreNodeName));
 
-    ElectionContext context = electionContexts.remove(new ContextKey(collection, coreNodeName));
-
-    if (context != null) {
-      context.cancelElection();
+      if (context != null) {
+        context.cancelElection();
+      }
     }
-
     CloudDescriptor cloudDescriptor = cd.getCloudDescriptor();
     zkStateReader.unregisterCore(cloudDescriptor.getCollectionName());
 
@@ -2408,11 +2431,9 @@ public class ZkController {
 
       for (Slice slice : slices) {
         Collection<Replica> replicas = slice.getReplicas();
-        for (Replica replica : replicas) {
-          if (replica.getName().equals(
-              dcore.getCloudDescriptor().getCoreNodeName())) {
-            return true;
-          }
+        Replica r = slice.getReplica(dcore.getCloudDescriptor().getCoreNodeName());
+        if (r != null) {
+          return true;
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
index f03eeeb..9758c8f 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
@@ -271,11 +271,12 @@ public class ReplicaMutator {
 
     replicaProps.putAll(message.getProperties());
     if (slice != null) {
-      Replica oldReplica = slice.getReplicasMap().get(coreNodeName);
+      Replica oldReplica = slice.getReplica(coreNodeName);
       if (oldReplica != null) {
         if (oldReplica.containsKey(ZkStateReader.LEADER_PROP)) {
           replicaProps.put(ZkStateReader.LEADER_PROP, oldReplica.get(ZkStateReader.LEADER_PROP));
         }
+        replicaProps.put(ZkStateReader.REPLICA_TYPE, oldReplica.getType().toString());
         // Move custom props over.
         for (Map.Entry<String, Object> ent : oldReplica.getProperties().entrySet()) {
           if (ent.getKey().startsWith(COLL_PROP_PREFIX)) {
@@ -311,6 +312,8 @@ public class ReplicaMutator {
 
 
     Replica replica = new Replica(coreNodeName, replicaProps);
+    
+    log.debug("Will update state for replica: {}", replica);
 
     Map<String, Object> sliceProps = null;
     Map<String, Replica> replicas;
@@ -328,11 +331,11 @@ public class ReplicaMutator {
       sliceProps.put(ZkStateReader.STATE_PROP, shardState);
       sliceProps.put(Slice.PARENT, shardParent);
     }
-
     replicas.put(replica.getName(), replica);
     slice = new Slice(sliceName, replicas, sliceProps);
 
     DocCollection newCollection = CollectionMutator.updateSlice(collectionName, collection, slice);
+    log.debug("Collection is now: {}", newCollection);
     return new ZkWriteCommand(collectionName, newCollection);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
index 4d767ed..5724f17 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
@@ -16,14 +16,16 @@
  */
 package org.apache.solr.cloud.overseer;
 
-import java.lang.invoke.MethodHandles;
+import static org.apache.solr.cloud.OverseerCollectionMessageHandler.COLL_PROP_PREFIX;
+import static org.apache.solr.cloud.overseer.CollectionMutator.checkCollectionKeyExistence;
+import static org.apache.solr.common.util.Utils.makeMap;
 
+import java.lang.invoke.MethodHandles;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Set;
 
-import com.google.common.collect.ImmutableSet;
 import org.apache.solr.cloud.Assign;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.common.cloud.ClusterState;
@@ -37,9 +39,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.cloud.OverseerCollectionMessageHandler.COLL_PROP_PREFIX;
-import static org.apache.solr.cloud.overseer.CollectionMutator.checkCollectionKeyExistence;
-import static org.apache.solr.common.util.Utils.makeMap;
+import com.google.common.collect.ImmutableSet;
 
 public class SliceMutator {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -66,14 +66,14 @@ public class SliceMutator {
       log.error("Invalid Collection/Slice {}/{} ", coll, slice);
       return ZkStateWriter.NO_OP;
     }
-
     String coreNodeName = Assign.assignNode(collection);
     Replica replica = new Replica(coreNodeName,
         makeMap(
             ZkStateReader.CORE_NAME_PROP, message.getStr(ZkStateReader.CORE_NAME_PROP),
             ZkStateReader.BASE_URL_PROP, message.getStr(ZkStateReader.BASE_URL_PROP),
             ZkStateReader.STATE_PROP, message.getStr(ZkStateReader.STATE_PROP),
-            ZkStateReader.NODE_NAME_PROP, message.getStr(ZkStateReader.NODE_NAME_PROP)));
+            ZkStateReader.NODE_NAME_PROP, message.getStr(ZkStateReader.NODE_NAME_PROP), 
+            ZkStateReader.REPLICA_TYPE, message.get(ZkStateReader.REPLICA_TYPE)));
     return new ZkWriteCommand(coll, updateReplica(collection, sl, replica.getName(), replica));
   }
 
@@ -249,13 +249,15 @@ public class SliceMutator {
   }
 
   public static DocCollection updateReplica(DocCollection collection, final Slice slice, String coreNodeName, final Replica replica) {
-    Map<String, Replica> copy = slice.getReplicasCopy();
+    Map<String, Replica> replicasCopy = slice.getReplicasCopy();
     if (replica == null) {
-      copy.remove(coreNodeName);
+      replicasCopy.remove(coreNodeName);
     } else {
-      copy.put(replica.getName(), replica);
+      replicasCopy.put(replica.getName(), replica);
     }
-    Slice newSlice = new Slice(slice.getName(), copy, slice.getProperties());
+    Slice newSlice = new Slice(slice.getName(), replicasCopy, slice.getProperties());
+    log.debug("Old Slice: {}", slice);
+    log.debug("New Slice: {}", newSlice);
     return CollectionMutator.updateSlice(collection.getName(), collection, newSlice);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
index da0f57c..911a9e3 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
@@ -261,6 +261,7 @@ public class ZkStateWriter {
       }
     }
 
+    log.trace("New Cluster State is: {}", clusterState);
     return clusterState;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java b/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
index 3eab8b4..506e158 100644
--- a/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
+++ b/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
@@ -62,10 +62,12 @@ public class ReplicaAssigner {
   public static class Position implements Comparable<Position> {
     public final String shard;
     public final int index;
+    public final Replica.Type type;
 
-    public Position(String shard, int replicaIdx) {
+    public Position(String shard, int replicaIdx, Replica.Type type) {
       this.shard = shard;
       this.index = replicaIdx;
+      this.type = type;
     }
 
     @Override
@@ -188,7 +190,7 @@ public class ReplicaAssigner {
       List<Position> positions = new ArrayList<>();
       for (int pos : p) {
         for (int j = 0; j < shardVsReplicaCount.get(shardNames.get(pos)); j++) {
-          positions.add(new Position(shardNames.get(pos), j));
+          positions.add(new Position(shardNames.get(pos), j, Replica.Type.NRT));
         }
       }
       Collections.sort(positions);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 3ff5135..7471c08 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -16,6 +16,17 @@
  */
 package org.apache.solr.core;
 
+import static java.util.Objects.requireNonNull;
+import static org.apache.solr.common.params.CommonParams.AUTHC_PATH;
+import static org.apache.solr.common.params.CommonParams.AUTHZ_PATH;
+import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
+import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
+import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH;
+import static org.apache.solr.common.params.CommonParams.INFO_HANDLER_PATH;
+import static org.apache.solr.common.params.CommonParams.METRICS_PATH;
+import static org.apache.solr.common.params.CommonParams.ZK_PATH;
+import static org.apache.solr.security.AuthenticationPlugin.AUTHENTICATION_PLUGIN_PROP;
+
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
@@ -56,6 +67,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Replica.State;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.Utils;
@@ -95,17 +107,7 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static java.util.Objects.requireNonNull;
-import static org.apache.solr.common.params.CommonParams.AUTHC_PATH;
-import static org.apache.solr.common.params.CommonParams.AUTHZ_PATH;
-import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
-import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
-import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH;
-import static org.apache.solr.common.params.CommonParams.INFO_HANDLER_PATH;
-import static org.apache.solr.common.params.CommonParams.METRICS_PATH;
-import static org.apache.solr.common.params.CommonParams.ZK_PATH;
 import static org.apache.solr.core.CorePropertiesLocator.PROPERTIES_FILENAME;
-import static org.apache.solr.security.AuthenticationPlugin.AUTHENTICATION_PLUGIN_PROP;
 
 /**
  *
@@ -1185,10 +1187,15 @@ public class CoreContainer {
         SolrCore newCore = core.reload(coreConfig);
         registerCore(cd, newCore, false, false);
         if (getZkController() != null) {
-          boolean onlyLeaderIndexes = getZkController().getClusterState().getCollection(cd.getCollectionName()).getRealtimeReplicas() == 1;
-          if (onlyLeaderIndexes && !cd.getCloudDescriptor().isLeader()) {
+          DocCollection docCollection = getZkController().getClusterState().getCollection(cd.getCollectionName());
+          Replica replica = docCollection.getReplica(cd.getCloudDescriptor().getCoreNodeName());
+          assert replica != null;
+          if (replica.getType() == Replica.Type.TLOG) { //TODO: needed here?
             getZkController().stopReplicationFromLeader(core.getName());
-            getZkController().startReplicationFromLeader(newCore.getName());
+            if (!cd.getCloudDescriptor().isLeader()) {
+              getZkController().startReplicationFromLeader(newCore.getName(), true);
+            }
+            
           }
         }
       } catch (SolrCoreState.CoreIsClosedException e) {
@@ -1274,6 +1281,11 @@ public class CoreContainer {
     if (zkSys.getZkController() != null) {
       // cancel recovery in cloud mode
       core.getSolrCoreState().cancelRecovery();
+      if (core.getCoreDescriptor().getCloudDescriptor().getReplicaType() == Replica.Type.PULL
+          || core.getCoreDescriptor().getCloudDescriptor().getReplicaType() == Replica.Type.TLOG) { 
+        // Stop replication if this is part of a pull/tlog replica before closing the core
+        zkSys.getZkController().stopReplicationFromLeader(name);
+      }
     }
     
     core.unloadOnClose(cd, deleteIndexDir, deleteDataDir, deleteInstanceDir);


[4/6] lucene-solr:jira/solr-8668: SOLR-10233: Add support for replica types

Posted by cp...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index 96e505a..7d15701 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -178,11 +178,13 @@ public class IndexFetcher {
     public static final IndexFetchResult INDEX_FETCH_FAILURE = new IndexFetchResult("Fetching lastest index is failed", false, null);
     public static final IndexFetchResult INDEX_FETCH_SUCCESS = new IndexFetchResult("Fetching latest index is successful", true, null);
     public static final IndexFetchResult LOCK_OBTAIN_FAILED = new IndexFetchResult("Obtaining SnapPuller lock failed", false, null);
+    public static final IndexFetchResult CONTAINER_IS_SHUTTING_DOWN = new IndexFetchResult("I was asked to replicate but CoreContainer is shutting down", false, null);
     public static final IndexFetchResult MASTER_VERSION_ZERO = new IndexFetchResult("Index in peer is empty and never committed yet", true, null);
     public static final IndexFetchResult NO_INDEX_COMMIT_EXIST = new IndexFetchResult("No IndexCommit in local index", false, null);
     public static final IndexFetchResult PEER_INDEX_COMMIT_DELETED = new IndexFetchResult("No files to download because IndexCommit in peer was deleted", false, null);
     public static final IndexFetchResult LOCAL_ACTIVITY_DURING_REPLICATION = new IndexFetchResult("Local index modification during replication", false, null);
     public static final IndexFetchResult EXPECTING_NON_LEADER = new IndexFetchResult("Replicating from leader but I'm the shard leader", false, null);
+    public static final IndexFetchResult LEADER_IS_NOT_ACTIVE = new IndexFetchResult("Replicating from leader but leader is not active", false, null);
 
     IndexFetchResult(String message, boolean successful, Throwable exception) {
       this.message = message;
@@ -352,17 +354,32 @@ public class IndexFetcher {
       // when we are a bit more confident we may want to try a partial replication
       // if the error is connection related or something, but we have to be careful
       forceReplication = true;
+      LOG.info("Last replication failed, so I'll force replication");
     }
 
     try {
       if (fetchFromLeader) {
+        assert !solrCore.isClosed(): "Replication should be stopped before closing the core";
         Replica replica = getLeaderReplica();
         CloudDescriptor cd = solrCore.getCoreDescriptor().getCloudDescriptor();
         if (cd.getCoreNodeName().equals(replica.getName())) {
           return IndexFetchResult.EXPECTING_NON_LEADER;
         }
-        masterUrl = replica.getCoreUrl();
-        LOG.info("Updated masterUrl to " + masterUrl);
+        if (replica.getState() != Replica.State.ACTIVE) {
+          LOG.info("Replica {} is leader but it's state is {}, skipping replication", replica.getName(), replica.getState());
+          return IndexFetchResult.LEADER_IS_NOT_ACTIVE;
+        }
+        if (!solrCore.getCoreContainer().getZkController().getClusterState().liveNodesContain(replica.getNodeName())) {
+          LOG.info("Replica {} is leader but it's not hosted on a live node, skipping replication", replica.getName());
+          return IndexFetchResult.LEADER_IS_NOT_ACTIVE;
+        }
+        if (!replica.getCoreUrl().equals(masterUrl)) {
+          masterUrl = replica.getCoreUrl();
+          LOG.info("Updated masterUrl to {}", masterUrl);
+          // TODO: Do we need to set forceReplication = true?
+        } else {
+          LOG.debug("masterUrl didn't change");
+        }
       }
       //get the current 'replicateable' index version in the master
       NamedList response;
@@ -410,6 +427,7 @@ public class IndexFetcher {
         if (forceReplication && commit.getGeneration() != 0) {
           // since we won't get the files for an empty index,
           // we just clear ours and commit
+          LOG.info("New index in Master. Deleting mine...");
           RefCounted<IndexWriter> iw = solrCore.getUpdateHandler().getSolrCoreState().getIndexWriter(solrCore);
           try {
             iw.get().deleteAll();
@@ -422,6 +440,7 @@ public class IndexFetcher {
 
         //there is nothing to be replicated
         successfulInstall = true;
+        LOG.debug("Nothing to replicate, master's version is 0");
         return IndexFetchResult.MASTER_VERSION_ZERO;
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/handler/RealTimeGetHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/RealTimeGetHandler.java b/solr/core/src/java/org/apache/solr/handler/RealTimeGetHandler.java
index bce374f..9f2b693 100644
--- a/solr/core/src/java/org/apache/solr/handler/RealTimeGetHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/RealTimeGetHandler.java
@@ -16,14 +16,18 @@
  */
 package org.apache.solr.handler;
 
-import org.apache.solr.api.Api;
-import org.apache.solr.api.ApiBag;
-import org.apache.solr.handler.component.*;
-
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
+import org.apache.solr.handler.component.HttpShardHandler;
+import org.apache.solr.handler.component.RealTimeGetComponent;
+import org.apache.solr.handler.component.SearchHandler;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
 
 public class RealTimeGetHandler extends SearchHandler {
   @Override
@@ -33,6 +37,14 @@ public class RealTimeGetHandler extends SearchHandler {
     names.add(RealTimeGetComponent.COMPONENT_NAME);
     return names;
   }
+  
+  
+  @Override
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    // Tell HttpShardHandlerthat this request should only be distributed to NRT replicas
+    req.getContext().put(HttpShardHandler.ONLY_NRT_REPLICAS, Boolean.TRUE);
+    super.handleRequestBody(req, rsp);
+  }
 
   //////////////////////// SolrInfoMBeans methods //////////////////////
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index 94ff189..f3dcdeb 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -403,6 +403,10 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     String masterUrl = solrParams == null ? null : solrParams.get(MASTER_URL);
     if (!indexFetchLock.tryLock())
       return IndexFetchResult.LOCK_OBTAIN_FAILED;
+    if (core.getCoreContainer().isShutDown()) {
+      LOG.warn("I was asked to replicate but CoreContainer is shutting down");
+      return IndexFetchResult.CONTAINER_IS_SHUTTING_DOWN; 
+    }
     try {
       if (masterUrl != null) {
         if (currentIndexFetcher != null && currentIndexFetcher != pollingIndexFetcher) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 02d8e8f..b05dd84 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -114,15 +114,7 @@ import static org.apache.solr.common.cloud.DocCollection.DOC_ROUTER;
 import static org.apache.solr.common.cloud.DocCollection.RULE;
 import static org.apache.solr.common.cloud.DocCollection.SNITCH;
 import static org.apache.solr.common.cloud.DocCollection.STATE_FORMAT;
-import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS;
-import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
-import static org.apache.solr.common.cloud.ZkStateReader.REALTIME_REPLICAS;
-import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.*;
 import static org.apache.solr.common.params.CollectionAdminParams.COUNT_PROP;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.*;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
@@ -408,7 +400,9 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
           AUTO_ADD_REPLICAS,
           RULE,
           SNITCH,
-          REALTIME_REPLICAS);
+          PULL_REPLICAS,
+          TLOG_REPLICAS,
+          NRT_REPLICAS);
 
       if (props.get(STATE_FORMAT) == null) {
         props.put(STATE_FORMAT, "2");
@@ -635,7 +629,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
           _ROUTE_,
           CoreAdminParams.NAME,
           INSTANCE_DIR,
-          DATA_DIR);
+          DATA_DIR,
+          REPLICA_TYPE);
       return copyPropertiesWithPrefix(req.getParams(), props, COLL_PROP_PREFIX);
     }),
     OVERSEERSTATUS_OP(OVERSEERSTATUS, (req, rsp, h) -> (Map) new LinkedHashMap<>()),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
index 6746332..1710da9 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
@@ -227,6 +227,7 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
       .put(CoreAdminParams.ROLES, CoreDescriptor.CORE_ROLES)
       .put(CoreAdminParams.CORE_NODE_NAME, CoreDescriptor.CORE_NODE_NAME)
       .put(ZkStateReader.NUM_SHARDS_PROP, CloudDescriptor.NUM_SHARDS)
+      .put(CoreAdminParams.REPLICA_TYPE, CloudDescriptor.REPLICA_TYPE)
       .build();
 
   protected static Map<String, String> buildCoreParams(SolrParams params) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/handler/admin/PrepRecoveryOp.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/PrepRecoveryOp.java b/solr/core/src/java/org/apache/solr/handler/admin/PrepRecoveryOp.java
index 0c2c903..748982d 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/PrepRecoveryOp.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/PrepRecoveryOp.java
@@ -18,6 +18,7 @@
 package org.apache.solr.handler.admin;
 
 import java.lang.invoke.MethodHandles;
+import java.util.Objects;
 
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.solr.cloud.CloudDescriptor;
@@ -63,18 +64,20 @@ class PrepRecoveryOp implements CoreAdminHandler.CoreAdminOp {
     Boolean onlyIfLeader = params.getBool("onlyIfLeader");
     Boolean onlyIfLeaderActive = params.getBool("onlyIfLeaderActive");
 
-    log.info("Going to wait for coreNodeName: " + coreNodeName + ", state: " + waitForState
-        + ", checkLive: " + checkLive + ", onlyIfLeader: " + onlyIfLeader
-        + ", onlyIfLeaderActive: " + onlyIfLeaderActive);
 
-    int maxTries = 0;
+    CoreContainer coreContainer = it.handler.coreContainer;
+    // wait long enough for the leader conflict to work itself out plus a little extra
+    int conflictWaitMs = coreContainer.getZkController().getLeaderConflictResolveWait();
+    int maxTries = (int) Math.round(conflictWaitMs / 1000) + 3;
+    log.info("Going to wait for coreNodeName: {}, state: {}, checkLive: {}, onlyIfLeader: {}, onlyIfLeaderActive: {}, maxTime: {} s",
+        coreNodeName, waitForState, checkLive, onlyIfLeader, onlyIfLeaderActive, maxTries);
+    
     Replica.State state = null;
     boolean live = false;
     int retry = 0;
     while (true) {
-      CoreContainer coreContainer = it.handler.coreContainer;
       try (SolrCore core = coreContainer.getCore(cname)) {
-        if (core == null && retry == 30) {
+        if (core == null && retry == Math.min(30, maxTries)) {
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "core not found:"
               + cname);
         }
@@ -102,15 +105,6 @@ class PrepRecoveryOp implements CoreAdminHandler.CoreAdminOp {
             coreContainer.getZkController().getZkStateReader().forceUpdateCollection(collectionName);
           }
 
-          if (maxTries == 0) {
-            // wait long enough for the leader conflict to work itself out plus a little extra
-            int conflictWaitMs = coreContainer.getZkController().getLeaderConflictResolveWait();
-            maxTries = (int) Math.round(conflictWaitMs / 1000) + 3;
-            log.info("Will wait a max of " + maxTries + " seconds to see " + cname + " (" +
-                cloudDescriptor.getShardId() + " of " +
-                cloudDescriptor.getCollectionName() + ") have state: " + waitForState);
-          }
-
           ClusterState clusterState = coreContainer.getZkController().getClusterState();
           DocCollection collection = clusterState.getCollection(collectionName);
           Slice slice = collection.getSlice(cloudDescriptor.getShardId());
@@ -160,6 +154,7 @@ class PrepRecoveryOp implements CoreAdminHandler.CoreAdminOp {
           String collection = null;
           String leaderInfo = null;
           String shardId = null;
+          
           try {
             CloudDescriptor cloudDescriptor =
                 core.getCoreDescriptor().getCloudDescriptor();
@@ -175,8 +170,7 @@ class PrepRecoveryOp implements CoreAdminHandler.CoreAdminOp {
               "I was asked to wait on state " + waitForState + " for "
                   + shardId + " in " + collection + " on " + nodeName
                   + " but I still do not see the requested state. I see state: "
-                  + state.toString() + " live:" + live + " leader from ZK: " + leaderInfo
-          );
+                  + Objects.toString(state) + " live:" + live + " leader from ZK: " + leaderInfo);
         }
 
         if (coreContainer.isShutDown()) {
@@ -185,7 +179,7 @@ class PrepRecoveryOp implements CoreAdminHandler.CoreAdminOp {
         }
 
         // solrcloud_debug
-        if (log.isDebugEnabled()) {
+        if (log.isDebugEnabled() && core != null) {
           try {
             LocalSolrQueryRequest r = new LocalSolrQueryRequest(core,
                 new ModifiableSolrParams());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
index 8c0a9cb..4ec3b79 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
@@ -60,6 +60,14 @@ import org.slf4j.LoggerFactory;
 import org.slf4j.MDC;
 
 public class HttpShardHandler extends ShardHandler {
+  
+  /**
+   * If the request context map has an entry with this key and Boolean.TRUE as value,
+   * {@link #prepDistributed(ResponseBuilder)} will only include {@link org.apache.solr.common.cloud.Replica.Type#NRT} replicas as possible
+   * destination of the distributed request (or a leader replica of type {@link org.apache.solr.common.cloud.Replica.Type#TLOG}). This is used 
+   * by the RealtimeGet handler, since other types of replicas shouldn't respond to RTG requests
+   */
+  public static String ONLY_NRT_REPLICAS = "distribOnlyRealtime";
 
   private HttpShardHandlerFactory httpShardHandlerFactory;
   private CompletionService<ShardResponse> completionService;
@@ -349,9 +357,12 @@ public class HttpShardHandler extends ShardHandler {
       // and make it a non-distributed request.
       String ourSlice = cloudDescriptor.getShardId();
       String ourCollection = cloudDescriptor.getCollectionName();
+      // Some requests may only be fulfilled by replicas of type Replica.Type.NRT
+      boolean onlyNrtReplicas = Boolean.TRUE == req.getContext().get(ONLY_NRT_REPLICAS);
       if (rb.slices.length == 1 && rb.slices[0] != null
           && ( rb.slices[0].equals(ourSlice) || rb.slices[0].equals(ourCollection + "_" + ourSlice) )  // handle the <collection>_<slice> format
-          && cloudDescriptor.getLastPublished() == Replica.State.ACTIVE) {
+          && cloudDescriptor.getLastPublished() == Replica.State.ACTIVE
+          && (!onlyNrtReplicas || cloudDescriptor.getReplicaType() == Replica.Type.NRT)) {
         boolean shortCircuit = params.getBool("shortCircuit", true);       // currently just a debugging parameter to check distrib search on a single node
 
         String targetHandler = params.get(ShardParams.SHARDS_QT);
@@ -387,14 +398,36 @@ public class HttpShardHandler extends ShardHandler {
             continue;
             // throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "no such shard: " + sliceName);
           }
+          Replica shardLeader = null;
 
           final Collection<Replica> allSliceReplicas = slice.getReplicasMap().values();
           final List<Replica> eligibleSliceReplicas = new ArrayList<>(allSliceReplicas.size());
           for (Replica replica : allSliceReplicas) {
             if (!clusterState.liveNodesContain(replica.getNodeName())
-                || replica.getState() != Replica.State.ACTIVE) {
+                || replica.getState() != Replica.State.ACTIVE
+                || (onlyNrtReplicas && replica.getType() == Replica.Type.PULL)) {
               continue;
             }
+            
+            if (onlyNrtReplicas && replica.getType() == Replica.Type.TLOG) {
+              if (shardLeader == null) {
+                try {
+                  shardLeader = zkController.getZkStateReader().getLeaderRetry(cloudDescriptor.getCollectionName(), slice.getName());
+                } catch (InterruptedException e) {
+                  throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "Exception finding leader for shard " + slice.getName() + " in collection " 
+                      + cloudDescriptor.getCollectionName(), e);
+                } catch (SolrException e) {
+                  if (log.isDebugEnabled()) {
+                    log.debug("Exception finding leader for shard {} in collection {}. Collection State: {}", 
+                        slice.getName(), cloudDescriptor.getCollectionName(), zkController.getZkStateReader().getClusterState().getCollectionOrNull(cloudDescriptor.getCollectionName()));
+                  }
+                  throw e;
+                }
+              }
+              if (!replica.getName().equals(shardLeader.getName())) {
+                continue;
+              }
+            }
             eligibleSliceReplicas.add(replica);
           }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
index c0ceddb..6d70435 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
@@ -16,6 +16,10 @@
  */
 package org.apache.solr.handler.component;
 
+import static org.apache.solr.common.params.CommonParams.DISTRIB;
+import static org.apache.solr.common.params.CommonParams.ID;
+import static org.apache.solr.common.params.CommonParams.VERSION_FIELD;
+
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
@@ -24,6 +28,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
@@ -68,9 +73,9 @@ import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.DocList;
-import org.apache.solr.search.SolrDocumentFetcher;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.ReturnFields;
+import org.apache.solr.search.SolrDocumentFetcher;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.SolrReturnFields;
 import org.apache.solr.search.SyntaxError;
@@ -82,10 +87,6 @@ import org.apache.solr.util.RefCounted;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.common.params.CommonParams.DISTRIB;
-import static org.apache.solr.common.params.CommonParams.ID;
-import static org.apache.solr.common.params.CommonParams.VERSION_FIELD;
-
 public class RealTimeGetComponent extends SearchComponent
 {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -105,7 +106,21 @@ public class RealTimeGetComponent extends SearchComponent
     SolrQueryRequest req = rb.req;
     SolrQueryResponse rsp = rb.rsp;
     SolrParams params = req.getParams();
-
+    CloudDescriptor cloudDesc = req.getCore().getCoreDescriptor().getCloudDescriptor();
+
+    if (cloudDesc != null) {
+      Replica.Type replicaType = cloudDesc.getReplicaType();
+      if (replicaType != null) {
+        if (replicaType == Replica.Type.PULL) {
+          throw new SolrException(ErrorCode.BAD_REQUEST, 
+              String.format(Locale.ROOT, "%s can't handle realtime get requests. Replicas of type %s do not support these type of requests", 
+                  cloudDesc.getCoreNodeName(),
+                  Replica.Type.PULL));
+        } 
+        // non-leader TLOG replicas should not respond to distrib /get requests, but internal requests are OK
+      }
+    }
+    
     if (!params.getBool(COMPONENT_NAME, true)) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
index e7f6a7b..e481109 100644
--- a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
+++ b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
@@ -45,7 +45,7 @@ import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.core.SolrConfig.UpdateHandlerInfo;
 import org.apache.solr.core.SolrCore;
@@ -122,12 +122,9 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
     indexWriterCloseWaitsForMerges = updateHandlerInfo.indexWriterCloseWaitsForMerges;
 
     ZkController zkController = core.getCoreContainer().getZkController();
-    if (zkController != null) {
-      DocCollection dc = zkController.getClusterState().getCollection(core.getCoreDescriptor().getCollectionName());
-      if (dc.getRealtimeReplicas() == 1) {
-        commitWithinSoftCommit = false;
-        commitTracker.setOpenSearcher(true);
-      }
+    if (zkController != null && core.getCoreDescriptor().getCloudDescriptor().getReplicaType() == Replica.Type.TLOG) {
+      commitWithinSoftCommit = false;
+      commitTracker.setOpenSearcher(true);
     }
 
   }
@@ -249,7 +246,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
       cmd.overwrite = false;
     }
     try {
-      if ( (cmd.getFlags() & UpdateCommand.IGNORE_INDEXWRITER) != 0) {
+      if ((cmd.getFlags() & UpdateCommand.IGNORE_INDEXWRITER) != 0) {
         if (ulog != null) ulog.add(cmd);
         return 1;
       }
@@ -425,7 +422,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
     deleteByIdCommands.increment();
     deleteByIdCommandsCumulative.mark();
 
-    if ( (cmd.getFlags() & UpdateCommand.IGNORE_INDEXWRITER) != 0 ) {
+    if ((cmd.getFlags() & UpdateCommand.IGNORE_INDEXWRITER) != 0 ) {
       if (ulog != null) ulog.delete(cmd);
       return;
     }
@@ -489,7 +486,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
     deleteByQueryCommandsCumulative.mark();
     boolean madeIt=false;
     try {
-      if ( (cmd.getFlags() & UpdateCommand.IGNORE_INDEXWRITER) != 0) {
+      if ((cmd.getFlags() & UpdateCommand.IGNORE_INDEXWRITER) != 0) {
         if (ulog != null) ulog.deleteByQuery(cmd);
         madeIt = true;
         return;
@@ -548,7 +545,6 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
     }
   }
 
-
   @Override
   public int mergeIndexes(MergeIndexesCommand cmd) throws IOException {
     mergeIndexesCommands.mark();
@@ -921,7 +917,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
    * Calls either {@link IndexWriter#updateDocValues} or {@link IndexWriter#updateDocument} as 
    * needed based on {@link AddUpdateCommand#isInPlaceUpdate}.
    * <p>
-   * If the this is an UPDATE_INPLACE cmd, then all fields inclued in 
+   * If the this is an UPDATE_INPLACE cmd, then all fields included in 
    * {@link AddUpdateCommand#getLuceneDocument} must either be the uniqueKey field, or be DocValue 
    * only fields.
    * </p>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
index 49d2664..f0eb8bc 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
@@ -115,7 +115,7 @@ public abstract class UpdateHandler implements SolrInfoBean {
   public UpdateHandler(SolrCore core)  {
     this(core, null);
   }
-
+  
   public UpdateHandler(SolrCore core, UpdateLog updateLog)  {
     this.core=core;
     idField = core.getLatestSchema().getUniqueKeyField();
@@ -124,7 +124,9 @@ public abstract class UpdateHandler implements SolrInfoBean {
     PluginInfo ulogPluginInfo = core.getSolrConfig().getPluginInfo(UpdateLog.class.getName());
 
 
-    if (updateLog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled()) {
+    // If this is a replica of type PULL, don't create the update log
+    boolean skipUpdateLog = core.getCoreDescriptor().getCloudDescriptor() != null && !core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog();
+    if (updateLog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled() && !skipUpdateLog) {
       String dataDir = (String)ulogPluginInfo.initArgs.get("dir");
 
       String ulogDir = core.getCoreDescriptor().getUlogDir();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/update/UpdateLog.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index c50add4..87b93f4 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -1092,7 +1092,7 @@ public static final int VERSION_IDX = 1;
 
   /**
    * Replay current tlog, so all updates will be written to index.
-   * This is must do task for a append replica become a new leader.
+   * This is must do task for a tlog replica become a new leader.
    * @return future of this task
    */
   public Future<RecoveryInfo> recoverFromCurrentLog() {
@@ -1706,7 +1706,7 @@ public static final int VERSION_IDX = 1;
 
     public void doReplay(TransactionLog translog) {
       try {
-        loglog.warn("Starting log replay " + translog + " active=" + activeLog + " starting pos=" + recoveryInfo.positionOfStart);
+        loglog.warn("Starting log replay " + translog + " active=" + activeLog + " starting pos=" + recoveryInfo.positionOfStart + " inSortedOrder=" + inSortedOrder);
         long lastStatusTime = System.nanoTime();
         if (inSortedOrder) {
           tlogReader = translog.getSortedReader(recoveryInfo.positionOfStart);
@@ -1786,7 +1786,7 @@ public static final int VERSION_IDX = 1;
                 recoveryInfo.adds++;
                 AddUpdateCommand cmd = convertTlogEntryToAddUpdateCommand(req, entry, oper, version);
                 cmd.setFlags(UpdateCommand.REPLAY | UpdateCommand.IGNORE_AUTOCOMMIT);
-                log.debug("{} {}", oper == ADD ? "add" : "update", cmd);
+                if (debug) log.debug("{} {}", oper == ADD ? "add" : "update", cmd);
                 proc.processAdd(cmd);
                 break;
               }
@@ -1854,6 +1854,7 @@ public static final int VERSION_IDX = 1;
             // something wrong with the request?
           }
           assert TestInjection.injectUpdateLogReplayRandomPause();
+          
         }
 
         CommitUpdateCommand cmd = new CommitUpdateCommand(req, false);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index cb1b2fb..5269ecb 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -279,7 +280,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   // this is set to true in the constructor if the next processors in the chain
   // are custom and may modify the SolrInputDocument racing with its serialization for replication
   private final boolean cloneRequiredOnLeader;
-  private final boolean onlyLeaderIndexes;
+  private final Replica.Type replicaType;
 
   public DistributedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next) {
     this(req, rsp, new AtomicUpdateDocumentMerger(req), next);
@@ -324,12 +325,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     
     if (cloudDesc != null) {
       collection = cloudDesc.getCollectionName();
-      ClusterState cstate = zkController.getClusterState();
-      DocCollection coll = cstate.getCollection(collection);
-      onlyLeaderIndexes = coll.getRealtimeReplicas() == 1;
+      replicaType = cloudDesc.getReplicaType();
     } else {
       collection = null;
-      onlyLeaderIndexes = false;
+      replicaType = Replica.Type.NRT;
     }
 
     boolean shouldClone = false;
@@ -666,7 +665,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
 
 
   // used for deleteByQuery to get the list of nodes this leader should forward to
-  private List<Node> setupRequest() {
+  private List<Node> setupRequestForDBQ() {
     List<Node> nodes = null;
     String shardId = cloudDesc.getShardId();
 
@@ -680,7 +679,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
 
       forwardToLeader = false;
       List<ZkCoreNodeProps> replicaProps = zkController.getZkStateReader()
-          .getReplicaProps(collection, shardId, leaderReplica.getName(), null, Replica.State.DOWN);
+          .getReplicaProps(collection, shardId, leaderReplica.getName(), null, Replica.State.DOWN, EnumSet.of(Replica.Type.NRT, Replica.Type.TLOG));
       if (replicaProps != null) {
         nodes = new ArrayList<>(replicaProps.size());
         for (ZkCoreNodeProps props : replicaProps) {
@@ -1190,7 +1189,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
                 checkDeleteByQueries = true;
               }
             }
-            if (onlyLeaderIndexes && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
+            if (replicaType == Replica.Type.TLOG && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
               cmd.setFlags(cmd.getFlags() | UpdateCommand.IGNORE_INDEXWRITER);
             }
           }
@@ -1576,7 +1575,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     if (zkEnabled && DistribPhase.TOLEADER == phase) {
       // This core should be a leader
       isLeader = true;
-      replicas = setupRequest();
+      replicas = setupRequestForDBQ();
     } else if (DistribPhase.FROMLEADER == phase) {
       isLeader = false;
     }
@@ -1610,8 +1609,9 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
           String myShardId = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
           Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry(
               collection, myShardId);
+          // DBQ forwarded to NRT and TLOG replicas
           List<ZkCoreNodeProps> replicaProps = zkController.getZkStateReader()
-              .getReplicaProps(collection, myShardId, leaderReplica.getName(), null, Replica.State.DOWN);
+              .getReplicaProps(collection, myShardId, leaderReplica.getName(), null, Replica.State.DOWN, EnumSet.of(Replica.Type.NRT, Replica.Type.TLOG));
           if (replicaProps != null) {
             final List<Node> myReplicas = new ArrayList<>(replicaProps.size());
             for (ZkCoreNodeProps replicaProp : replicaProps) {
@@ -1699,10 +1699,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
             return;
           }
 
-          if (onlyLeaderIndexes && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
+          if (replicaType == Replica.Type.TLOG && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
+            // TLOG replica not leader, don't write the DBQ to IW
             cmd.setFlags(cmd.getFlags() | UpdateCommand.IGNORE_INDEXWRITER);
           }
-
           doLocalDelete(cmd);
         }
       }
@@ -1857,7 +1857,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
               }
             }
 
-            if (onlyLeaderIndexes && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
+            if (replicaType == Replica.Type.TLOG && (cmd.getFlags() & UpdateCommand.REPLAY) == 0) {
               cmd.setFlags(cmd.getFlags() | UpdateCommand.IGNORE_INDEXWRITER);
             }
           }
@@ -1884,14 +1884,19 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
       zkCheck();
       
       nodes = getCollectionUrls(req, req.getCore().getCoreDescriptor()
-          .getCloudDescriptor().getCollectionName());
-      if (isLeader && nodes.size() == 1) {
+          .getCloudDescriptor().getCollectionName(), EnumSet.of(Replica.Type.TLOG,Replica.Type.NRT));
+      if (nodes == null) {
+        // This could happen if there are only pull replicas
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
+            "Unable to distribute commit operation. No replicas available of types " + Replica.Type.TLOG + " or " + Replica.Type.NRT);
+      }
+      if (isLeader && nodes.size() == 1 && replicaType != Replica.Type.PULL) {
         singleLeader = true;
       }
     }
     
     if (!zkEnabled || req.getParams().getBool(COMMIT_END_POINT, false) || singleLeader) {
-      if (onlyLeaderIndexes) {
+      if (replicaType == Replica.Type.TLOG) {
         try {
           Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry(
               collection, cloudDesc.getShardId());
@@ -1904,12 +1909,19 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
             doLocalCommit(cmd);
           } else {
             assert TestInjection.waitForInSyncWithLeader(req.getCore(),
-                zkController, collection, cloudDesc.getShardId());
+                zkController, collection, cloudDesc.getShardId()): "Core " + req.getCore() + " not in sync with leader";
           }
         } catch (InterruptedException e) {
           throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Exception finding leader for shard " + cloudDesc.getShardId(), e);
         }
+      } else if (replicaType == Replica.Type.PULL) {
+        log.warn("Commit not supported on replicas of type " + Replica.Type.PULL);
       } else {
+        // NRT replicas will always commit
+        if (vinfo != null) {
+          long commitVersion = vinfo.getNewClock();
+          cmd.setVersion(commitVersion);
+        }
         doLocalCommit(cmd);
       }
     } else {
@@ -1958,7 +1970,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
  
 
   
-  private List<Node> getCollectionUrls(SolrQueryRequest req, String collection) {
+  private List<Node> getCollectionUrls(SolrQueryRequest req, String collection, EnumSet<Replica.Type> types) {
     ClusterState clusterState = req.getCore()
         .getCoreContainer().getZkController().getClusterState();
     Map<String,Slice> slices = clusterState.getSlicesMap(collection);
@@ -1973,6 +1985,9 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
       Map<String,Replica> shardMap = replicas.getReplicasMap();
       
       for (Entry<String,Replica> entry : shardMap.entrySet()) {
+        if (!types.contains(entry.getValue().getType())) {
+          continue;
+        }
         ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue());
         if (clusterState.liveNodesContain(nodeProps.getNodeName())) {
           urls.add(new StdNode(nodeProps, collection, replicas.getName()));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/java/org/apache/solr/util/TestInjection.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/TestInjection.java b/solr/core/src/java/org/apache/solr/util/TestInjection.java
index 6b7b1f8..5b0d047 100644
--- a/solr/core/src/java/org/apache/solr/util/TestInjection.java
+++ b/solr/core/src/java/org/apache/solr/util/TestInjection.java
@@ -75,7 +75,7 @@ public class TestInjection {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   
   private static final Pattern ENABLED_PERCENT = Pattern.compile("(true|false)(?:\\:(\\d+))?$", Pattern.CASE_INSENSITIVE);
-
+  
   private static final String LUCENE_TEST_CASE_FQN = "org.apache.lucene.util.LuceneTestCase";
 
   /** 
@@ -151,6 +151,7 @@ public class TestInjection {
     splitFailureBeforeReplicaCreation = null;
     prepRecoveryOpPauseForever = null;
     countPrepRecoveryOpPauseForever = new AtomicInteger(0);
+    waitForReplicasInSync = "true:60";
 
     for (Timer timer : timers) {
       timer.cancel();
@@ -387,9 +388,10 @@ public class TestInjection {
             String localVersion = searcher.get().getIndexReader().getIndexCommit().getUserData().get(SolrIndexWriter.COMMIT_TIME_MSEC_KEY);
             if (localVersion == null && leaderVersion == 0 && !core.getUpdateHandler().getUpdateLog().hasUncommittedChanges()) return true;
             if (localVersion != null && Long.parseLong(localVersion) == leaderVersion && (leaderVersion >= t || i >= 6)) {
-              log.info("Waiting time for replica in sync with leader: {}", System.currentTimeMillis()-currentTime);
+              log.info("Waiting time for tlog replica to be in sync with leader: {}", System.currentTimeMillis()-currentTime);
               return true;
             } else {
+              log.debug("Tlog replica not in sync with leader yet. Attempt: {}. Local Version={}, leader Version={}", i, localVersion, leaderVersion);
               Thread.sleep(500);
             }
           } finally {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test-files/solr/collection1/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig.xml
index 58f9551..a63f6cb 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig.xml
@@ -63,6 +63,10 @@
 
   <updateHandler class="solr.DirectUpdateHandler2">
 
+    <autoCommit>
+      <maxTime>${solr.autoCommit.maxTime:-1}</maxTime>
+    </autoCommit>
+
     <!-- autocommit pending docs if certain criteria are met
     <autoCommit>
       <maxDocs>10000</maxDocs>
@@ -478,7 +482,7 @@
       <str name="facet.query">foo_s:bar</str>
     </lst>
   </requestHandler>
-
+  
   <admin>
     <defaultQuery>solr</defaultQuery>
     <gettableFiles>solrconfig.xml schema.xml admin-extra.html</gettableFiles>
@@ -577,6 +581,8 @@
       <str name="df">text</str>
     </lst>
   </initParams>
+  
+  
 
 </config>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test-files/solr/configsets/cloud-minimal/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/configsets/cloud-minimal/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/cloud-minimal/conf/solrconfig.xml
index 059e58f..8da7d28 100644
--- a/solr/core/src/test-files/solr/configsets/cloud-minimal/conf/solrconfig.xml
+++ b/solr/core/src/test-files/solr/configsets/cloud-minimal/conf/solrconfig.xml
@@ -33,7 +33,7 @@
     <commitWithin>
       <softCommit>${solr.commitwithin.softcommit:true}</softCommit>
     </commitWithin>
-    <updateLog></updateLog>
+    <updateLog class="${solr.ulog:solr.UpdateLog}"></updateLog>
   </updateHandler>
 
   <requestHandler name="/select" class="solr.SearchHandler">

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/AbstractCloudBackupRestoreTestCase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/AbstractCloudBackupRestoreTestCase.java b/solr/core/src/test/org/apache/solr/cloud/AbstractCloudBackupRestoreTestCase.java
index 3414759..a6d130e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AbstractCloudBackupRestoreTestCase.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AbstractCloudBackupRestoreTestCase.java
@@ -32,6 +32,7 @@ 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.CollectionAdminRequest.ClusterProp;
 import org.apache.solr.client.solrj.response.RequestStatusState;
@@ -48,8 +49,6 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.common.params.ShardParams._ROUTE_;
-
 /**
  * This class implements the logic required to test Solr cloud backup/restore capability.
  */
@@ -84,11 +83,17 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
   @Test
   public void test() throws Exception {
     boolean isImplicit = random().nextBoolean();
+    boolean doSplitShardOperation = !isImplicit && random().nextBoolean();
     int replFactor = TestUtil.nextInt(random(), 1, 2);
+    int numTlogReplicas = TestUtil.nextInt(random(), 0, 1);
+    int numPullReplicas = TestUtil.nextInt(random(), 0, 1);
     CollectionAdminRequest.Create create =
-        CollectionAdminRequest.createCollection(getCollectionName(), "conf1", NUM_SHARDS, replFactor);
-    if (NUM_SHARDS * replFactor > cluster.getJettySolrRunners().size() || random().nextBoolean()) {
-      create.setMaxShardsPerNode(NUM_SHARDS);//just to assert it survives the restoration
+        CollectionAdminRequest.createCollection(getCollectionName(), "conf1", NUM_SHARDS, replFactor, numTlogReplicas, numPullReplicas);
+    if (NUM_SHARDS * (replFactor + numTlogReplicas + numPullReplicas) > cluster.getJettySolrRunners().size() || random().nextBoolean()) {
+      create.setMaxShardsPerNode((int)Math.ceil(NUM_SHARDS * (replFactor + numTlogReplicas + numPullReplicas) / cluster.getJettySolrRunners().size()));//just to assert it survives the restoration
+      if (doSplitShardOperation) {
+        create.setMaxShardsPerNode(create.getMaxShardsPerNode() * 2);
+      }
     }
     if (random().nextBoolean()) {
       create.setAutoAddReplicas(true);//just to assert it survives the restoration
@@ -112,7 +117,7 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
 
     indexDocs(getCollectionName());
 
-    if (!isImplicit && random().nextBoolean()) {
+    if (doSplitShardOperation) {
       // shard split the first shard
       int prevActiveSliceCount = getActiveSliceCount(getCollectionName());
       CollectionAdminRequest.SplitShard splitShard = CollectionAdminRequest.splitShard(getCollectionName());
@@ -235,9 +240,9 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
       CollectionAdminRequest.Restore restore = CollectionAdminRequest.restoreCollection(restoreCollectionName, backupName)
           .setLocation(backupLocation).setRepositoryName(getBackupRepoName());
 
-      if (origShardToDocCount.size() > cluster.getJettySolrRunners().size()) {
+      if (backupCollection.getReplicas().size() > cluster.getJettySolrRunners().size()) {
         // may need to increase maxShardsPerNode (e.g. if it was shard split, then now we need more)
-        restore.setMaxShardsPerNode(origShardToDocCount.size());
+        restore.setMaxShardsPerNode((int)Math.ceil(backupCollection.getReplicas().size()/cluster.getJettySolrRunners().size()));
       }
 
       if (rarely()) { // Try with createNodeSet configuration
@@ -304,9 +309,11 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
     Map<String,Integer> shardToDocCount = new TreeMap<>();
     for (Slice slice : docCollection.getActiveSlices()) {
       String shardName = slice.getName();
-      long docsInShard = client.query(docCollection.getName(), new SolrQuery("*:*").setParam(_ROUTE_, shardName))
-          .getResults().getNumFound();
-      shardToDocCount.put(shardName, (int) docsInShard);
+      try (HttpSolrClient leaderClient = new HttpSolrClient.Builder(slice.getLeader().getCoreUrl()).withHttpClient(client.getHttpClient()).build()) {
+        long docsInShard = leaderClient.query(new SolrQuery("*:*").setParam("distrib", "false"))
+            .getResults().getNumFound();
+        shardToDocCount.put(shardName, (int) docsInShard);
+      }
     }
     return shardToDocCount;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/AssignTest.java b/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
index 7593f3b..8e32510 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
@@ -87,4 +87,10 @@ public class AssignTest extends SolrTestCaseJ4 {
     assertEquals("core_node2", nodeName);
   }
   
+  @Test
+  public void testBuildCoreName() {
+    assertEquals("Core name pattern changed", "collection1_shard1_replica_n1", Assign.buildCoreName("collection1", "shard1", Replica.Type.NRT, 1));
+    assertEquals("Core name pattern changed", "collection1_shard2_replica_p2", Assign.buildCoreName("collection1", "shard2", Replica.Type.PULL,2));
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
index 5eb4b3b..c8e92fc 100644
--- a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
+++ b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
@@ -62,8 +62,8 @@ public class BasicDistributedZk2Test extends AbstractFullDistribZkTestBase {
   }
 
   @Override
-  protected int getRealtimeReplicas() {
-    return onlyLeaderIndexes? 1 : -1;
+  protected boolean useTlogReplicas() {
+    return onlyLeaderIndexes;
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
index 1c23c9c..18caa58 100644
--- a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
@@ -119,8 +119,8 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
   }
 
   @Override
-  protected int getRealtimeReplicas() {
-    return onlyLeaderIndexes? 1 : -1;
+  protected boolean useTlogReplicas() {
+    return onlyLeaderIndexes;
   }
 
   @Override
@@ -1075,12 +1075,6 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
     assertEquals(collection3Docs, collection2Docs - 1);
   }
   
-  protected SolrInputDocument getDoc(Object... fields) throws Exception {
-    SolrInputDocument doc = new SolrInputDocument();
-    addFields(doc, fields);
-    return doc;
-  }
-  
   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();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java
index ffc5262..2e31520 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java
@@ -16,33 +16,21 @@
  */
 package org.apache.solr.cloud;
 
-import java.lang.invoke.MethodHandles;
-import java.net.ConnectException;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.http.client.HttpClient;
-import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.SolrTestCaseJ4.SuppressObjectReleaseTracker;
 import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
-import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient;
-import org.apache.solr.client.solrj.impl.HttpClientUtil;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.IOUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 @Slow
 @SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-5776")
@@ -51,8 +39,6 @@ import org.slf4j.LoggerFactory;
 public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase {
   private static final int FAIL_TOLERANCE = 100;
 
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  
   private static final Integer RUN_LENGTH = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.runlength", "-1"));
 
   private final boolean onlyLeaderIndexes = random().nextBoolean();
@@ -112,8 +98,8 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
   }
 
   @Override
-  protected int getRealtimeReplicas() {
-    return onlyLeaderIndexes? 1 : -1;
+  protected boolean useTlogReplicas() {
+    return onlyLeaderIndexes;
   }
 
   @Test
@@ -158,8 +144,8 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
       // it's currently hard to know what requests failed when using ConcurrentSolrUpdateServer
       boolean runFullThrottle = random().nextBoolean();
       if (runFullThrottle) {
-        FullThrottleStoppableIndexingThread ftIndexThread = new FullThrottleStoppableIndexingThread(
-            clients, "ft1", true);
+        FullThrottleStoppableIndexingThread ftIndexThread = 
+            new FullThrottleStoppableIndexingThread(controlClient, cloudClient, clients, "ft1", true, this.clientSoTimeout);
         threads.add(ftIndexThread);
         ftIndexThread.start();
       }
@@ -289,111 +275,6 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
     return deleteFails;
   }
 
-  class FullThrottleStoppableIndexingThread extends StoppableIndexingThread {
-    private CloseableHttpClient httpClient = HttpClientUtil.createClient(null);
-    private volatile boolean stop = false;
-    int clientIndex = 0;
-    private ConcurrentUpdateSolrClient cusc;
-    private List<SolrClient> clients;
-    private AtomicInteger fails = new AtomicInteger();
-    
-    public FullThrottleStoppableIndexingThread(List<SolrClient> clients,
-                                               String id, boolean doDeletes) {
-      super(controlClient, cloudClient, id, doDeletes);
-      setName("FullThrottleStopableIndexingThread");
-      setDaemon(true);
-      this.clients = clients;
-
-      cusc = new ErrorLoggingConcurrentUpdateSolrClient(((HttpSolrClient) clients.get(0)).getBaseURL(), httpClient, 8, 2);
-      cusc.setConnectionTimeout(10000);
-      cusc.setSoTimeout(clientSoTimeout);
-    }
-    
-    @Override
-    public void run() {
-      int i = 0;
-      int numDeletes = 0;
-      int numAdds = 0;
-
-      while (true && !stop) {
-        String id = this.id + "-" + i;
-        ++i;
-        
-        if (doDeletes && random().nextBoolean() && deletes.size() > 0) {
-          String delete = deletes.remove(0);
-          try {
-            numDeletes++;
-            cusc.deleteById(delete);
-          } catch (Exception e) {
-            changeUrlOnError(e);
-            fails.incrementAndGet();
-          }
-        }
-        
-        try {
-          numAdds++;
-          if (numAdds > (TEST_NIGHTLY ? 4002 : 197))
-            continue;
-          SolrInputDocument doc = getDoc(
-              "id",
-              id,
-              i1,
-              50,
-              t1,
-              "Saxon heptarchies that used to rip around so in old times and raise Cain.  My, you ought to seen old Henry the Eight when he was in bloom.  He WAS a blossom.  He used to marry a new wife every day, and chop off her head next morning.  And he would do it just as indifferent as if ");
-          cusc.add(doc);
-        } catch (Exception e) {
-          changeUrlOnError(e);
-          fails.incrementAndGet();
-        }
-        
-        if (doDeletes && random().nextBoolean()) {
-          deletes.add(id);
-        }
-        
-      }
-
-      log.info("FT added docs:" + numAdds + " with " + fails + " fails" + " deletes:" + numDeletes);
-    }
-
-    private void changeUrlOnError(Exception e) {
-      if (e instanceof ConnectException) {
-        clientIndex++;
-        if (clientIndex > clients.size() - 1) {
-          clientIndex = 0;
-        }
-        cusc.shutdownNow();
-        cusc = new ErrorLoggingConcurrentUpdateSolrClient(((HttpSolrClient) clients.get(clientIndex)).getBaseURL(),
-            httpClient, 30, 3);
-      }
-    }
-    
-    @Override
-    public void safeStop() {
-      stop = true;
-      cusc.blockUntilFinished();
-      cusc.shutdownNow();
-      IOUtils.closeQuietly(httpClient);
-    }
-
-    @Override
-    public int getFailCount() {
-      return fails.get();
-    }
-    
-    @Override
-    public Set<String> getAddFails() {
-      throw new UnsupportedOperationException();
-    }
-    
-    @Override
-    public Set<String> getDeleteFails() {
-      throw new UnsupportedOperationException();
-    }
-    
-  };
-  
-  
   // skip the randoms - they can deadlock...
   @Override
   protected void indexr(Object... fields) throws Exception {
@@ -401,13 +282,4 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
     indexDoc(doc);
   }
 
-  static class ErrorLoggingConcurrentUpdateSolrClient extends ConcurrentUpdateSolrClient {
-    public ErrorLoggingConcurrentUpdateSolrClient(String serverUrl, HttpClient httpClient, int queueSize, int threadCount) {
-      super(serverUrl, httpClient, queueSize, threadCount, null, false);
-    }
-    @Override
-    public void handleError(Throwable ex) {
-      log.warn("cusc error", ex);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
new file mode 100644
index 0000000..11c25d3
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
@@ -0,0 +1,327 @@
+/*
+ * 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.EnumSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.solr.SolrTestCaseJ4.SuppressObjectReleaseTracker;
+import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
+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.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.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering;
+
+@Slow
+@SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-5776")
+@ThreadLeakLingering(linger = 60000)
+@SuppressObjectReleaseTracker(bugUrl="Testing purposes")
+public class ChaosMonkeyNothingIsSafeWithPullReplicasTest extends AbstractFullDistribZkTestBase {
+  private static final int FAIL_TOLERANCE = 100;
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  
+  private static final Integer RUN_LENGTH = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.runlength", "-1"));
+
+  private final boolean useTlogReplicas = random().nextBoolean();
+  
+  private final int numPullReplicas;
+  private final int numRealtimeOrTlogReplicas;
+  
+  protected int getPullReplicaCount() {
+    return numPullReplicas;
+  }
+
+  @BeforeClass
+  public static void beforeSuperClass() {
+    schemaString = "schema15.xml";      // we need a string id
+    if (usually()) {
+      System.setProperty("solr.autoCommit.maxTime", "15000");
+    }
+    TestInjection.waitForReplicasInSync = null;
+    setErrorHook();
+  }
+  
+  @AfterClass
+  public static void afterSuperClass() {
+    System.clearProperty("solr.autoCommit.maxTime");
+    clearErrorHook();
+    TestInjection.reset();
+  }
+  
+  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};
+
+  private int clientSoTimeout;
+  
+  public String[] getFieldNames() {
+    return fieldNames;
+  }
+
+  public RandVal[] getRandValues() {
+    return randVals;
+  }
+  
+  @Override
+  public void distribSetUp() throws Exception {
+    super.distribSetUp();
+    // can help to hide this when testing and looking at logs
+    //ignoreException("shard update error");
+    useFactory("solr.StandardDirectoryFactory");
+  }
+  
+  public ChaosMonkeyNothingIsSafeWithPullReplicasTest() {
+    super();
+    numPullReplicas = random().nextInt(TEST_NIGHTLY ? 2 : 1) + 1;
+    numRealtimeOrTlogReplicas = random().nextInt(TEST_NIGHTLY ? 4 : 3) + 1;
+    sliceCount = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.slicecount", "-1"));
+    if (sliceCount == -1) {
+      sliceCount = random().nextInt(TEST_NIGHTLY ? 3 : 2) + 1;
+    }
+
+    int numNodes = sliceCount * (numRealtimeOrTlogReplicas + numPullReplicas);
+    fixShardCount(numNodes);
+    log.info("Starting ChaosMonkey test with {} shards and {} nodes", sliceCount, numNodes);
+
+    // None of the operations used here are particularly costly, so this should work.
+    // Using this low timeout will also help us catch index stalling.
+    clientSoTimeout = 5000;
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return useTlogReplicas;
+  }
+
+  @Test
+  public void test() throws Exception {
+    cloudClient.setSoTimeout(clientSoTimeout);
+    DocCollection docCollection = cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION);
+    assertEquals(this.sliceCount, docCollection.getSlices().size());
+    Slice s = docCollection.getSlice("shard1");
+    assertNotNull(s);
+    assertEquals("Unexpected number of replicas. Collection: " + docCollection, numRealtimeOrTlogReplicas + numPullReplicas, s.getReplicas().size());
+    assertEquals("Unexpected number of pull replicas. Collection: " + docCollection, numPullReplicas, s.getReplicas(EnumSet.of(Replica.Type.PULL)).size());
+    assertEquals(useTlogReplicas()?0:numRealtimeOrTlogReplicas, s.getReplicas(EnumSet.of(Replica.Type.NRT)).size());
+    assertEquals(useTlogReplicas()?numRealtimeOrTlogReplicas:0, s.getReplicas(EnumSet.of(Replica.Type.TLOG)).size());
+    
+    boolean testSuccessful = false;
+    try {
+      handle.clear();
+      handle.put("timestamp", SKIPVAL);
+      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);
+      
+      // we cannot do delete by query
+      // as it's not supported for recovery
+      del("*:*");
+      
+      List<StoppableThread> threads = new ArrayList<>();
+      List<StoppableIndexingThread> indexTreads = new ArrayList<>();
+      int threadCount = TEST_NIGHTLY ? 3 : 1;
+      int i = 0;
+      for (i = 0; i < threadCount; i++) {
+        StoppableIndexingThread indexThread = new StoppableIndexingThread(controlClient, cloudClient, Integer.toString(i), true);
+        threads.add(indexThread);
+        indexTreads.add(indexThread);
+        indexThread.start();
+      }
+      
+      threadCount = 1;
+      i = 0;
+      for (i = 0; i < threadCount; i++) {
+        StoppableSearchThread searchThread = new StoppableSearchThread(cloudClient);
+        threads.add(searchThread);
+        searchThread.start();
+      }
+      
+      if (usually()) {
+        StoppableCommitThread commitThread = new StoppableCommitThread(cloudClient, 1000, false);
+        threads.add(commitThread);
+        commitThread.start();
+      }
+      
+      // TODO: we only do this sometimes so that we can sometimes compare against control,
+      // it's currently hard to know what requests failed when using ConcurrentSolrUpdateServer
+      boolean runFullThrottle = random().nextBoolean();
+      if (runFullThrottle) {
+        FullThrottleStoppableIndexingThread ftIndexThread = 
+            new FullThrottleStoppableIndexingThread(controlClient, cloudClient, clients, "ft1", true, this.clientSoTimeout);
+        threads.add(ftIndexThread);
+        ftIndexThread.start();
+      }
+      
+      chaosMonkey.startTheMonkey(true, 10000);
+      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)];
+        }
+        ChaosMonkey.wait(runLength, DEFAULT_COLLECTION, zkStateReader);
+      } finally {
+        chaosMonkey.stopTheMonkey();
+      }
+
+      // ideally this should go into chaosMonkey
+      restartZk(1000 * (5 + random().nextInt(4)));
+
+      for (StoppableThread indexThread : threads) {
+        indexThread.safeStop();
+      }
+      
+      // start any downed jetties to be sure we still will end up with a leader per shard...
+      
+      // wait for stop...
+      for (StoppableThread indexThread : threads) {
+        indexThread.join();
+      }
+      
+      // try and wait for any replications and what not to finish...
+      
+      ChaosMonkey.wait(2000, DEFAULT_COLLECTION, zkStateReader);
+      
+      // wait until there are no recoveries...
+      waitForThingsToLevelOut(Integer.MAX_VALUE);//Math.round((runLength / 1000.0f / 3.0f)));
+      
+      // make sure we again have leaders for each shard
+      for (int j = 1; j < sliceCount; j++) {
+        zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 30000);
+      }
+      
+      commit();
+      
+      // TODO: assert we didnt kill everyone
+      
+      zkStateReader.updateLiveNodes();
+      assertTrue(zkStateReader.getClusterState().getLiveNodes().size() > 0);
+      
+      
+      // we expect full throttle fails, but cloud client should not easily fail
+      for (StoppableThread indexThread : threads) {
+        if (indexThread instanceof StoppableIndexingThread && !(indexThread instanceof FullThrottleStoppableIndexingThread)) {
+          int failCount = ((StoppableIndexingThread) indexThread).getFailCount();
+          assertFalse("There were too many update fails (" + failCount + " > " + FAIL_TOLERANCE
+              + ") - we expect it can happen, but shouldn't easily", failCount > FAIL_TOLERANCE);
+        }
+      }
+      
+      waitForReplicationFromReplicas(DEFAULT_COLLECTION, zkStateReader, new TimeOut(30, TimeUnit.SECONDS));
+//      waitForAllWarmingSearchers();
+      
+      Set<String> addFails = getAddFails(indexTreads);
+      Set<String> deleteFails = getDeleteFails(indexTreads);
+      // full throttle thread can
+      // have request fails
+      checkShardConsistency(!runFullThrottle, true, addFails, deleteFails);      
+      
+      long ctrlDocs = controlClient.query(new SolrQuery("*:*")).getResults()
+      .getNumFound(); 
+      
+      // ensure we have added more than 0 docs
+      long cloudClientDocs = cloudClient.query(new SolrQuery("*:*"))
+          .getResults().getNumFound();
+      
+      assertTrue("Found " + ctrlDocs + " control docs", cloudClientDocs > 0);
+      
+      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()) {
+        restartZk(1000 * (5 + random().nextInt(4)));
+      }
+
+      try (CloudSolrClient client = createCloudClient("collection1")) {
+        // We don't really know how many live nodes we have at this point, so "maxShardsPerNode" needs to be > 1
+        createCollection(null, "testcollection",
+              1, 1, 10, client, null, "conf1"); 
+      }
+      List<Integer> numShardsNumReplicas = new ArrayList<>(2);
+      numShardsNumReplicas.add(1);
+      numShardsNumReplicas.add(1 + getPullReplicaCount());
+      checkForCollection("testcollection", numShardsNumReplicas, null);
+      
+      testSuccessful = true;
+    } finally {
+      if (!testSuccessful) {
+        logReplicaTypesReplicationInfo(DEFAULT_COLLECTION, cloudClient.getZkStateReader());
+        printLayout();
+      }
+    }
+  }
+
+  private Set<String> getAddFails(List<StoppableIndexingThread> threads) {
+    Set<String> addFails = new HashSet<String>();
+    for (StoppableIndexingThread thread : threads)   {
+      addFails.addAll(thread.getAddFails());
+//      addFails.addAll(thread.getAddFailsMinRf());
+    }
+    return addFails;
+  }
+  
+  private Set<String> getDeleteFails(List<StoppableIndexingThread> threads) {
+    Set<String> deleteFails = new HashSet<String>();
+    for (StoppableIndexingThread thread : threads)   {
+      deleteFails.addAll(thread.getDeleteFails());
+//      deleteFails.addAll(thread.getDeleteFailsMinRf());
+    }
+    return deleteFails;
+  }
+
+  // skip the randoms - they can deadlock...
+  @Override
+  protected void indexr(Object... fields) throws Exception {
+    SolrInputDocument doc = getDoc(fields);
+    indexDoc(doc);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
new file mode 100644
index 0000000..f2e8845
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPullReplicasTest.java
@@ -0,0 +1,254 @@
+/*
+ * 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.EnumSet;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.solr.SolrTestCaseJ4.SuppressObjectReleaseTracker;
+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.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Slow
+@SuppressObjectReleaseTracker(bugUrl="Testing purposes")
+public class ChaosMonkeySafeLeaderWithPullReplicasTest extends AbstractFullDistribZkTestBase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  
+  private static final Integer RUN_LENGTH = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.runlength", "-1"));
+  
+  private final boolean useTlogReplicas = random().nextBoolean();
+  
+  private final int numPullReplicas;
+  private final int numRealtimeOrTlogReplicas;
+  
+  @Override
+  protected int getPullReplicaCount() {
+    return numPullReplicas;
+  }
+  
+  @Override
+  protected boolean useTlogReplicas() {
+    return useTlogReplicas;
+  }
+
+  @BeforeClass
+  public static void beforeSuperClass() {
+    schemaString = "schema15.xml";      // we need a string id
+    if (usually()) {
+      System.setProperty("solr.autoCommit.maxTime", "15000");
+    }
+    TestInjection.waitForReplicasInSync = null;
+    setErrorHook();
+  }
+  
+  @AfterClass
+  public static void afterSuperClass() {
+    System.clearProperty("solr.autoCommit.maxTime");
+    clearErrorHook();
+    TestInjection.reset();
+  }
+
+  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;
+  }
+  
+  @Override
+  public void distribSetUp() throws Exception {
+    useFactory("solr.StandardDirectoryFactory");
+    super.distribSetUp();
+  }
+  
+  public ChaosMonkeySafeLeaderWithPullReplicasTest() {
+    super();
+    numPullReplicas = random().nextInt(TEST_NIGHTLY ? 3 : 2) + 1;;
+    numRealtimeOrTlogReplicas = random().nextInt(TEST_NIGHTLY ? 3 : 2) + 1;;
+    sliceCount = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.slicecount", "-1"));
+    if (sliceCount == -1) {
+      sliceCount = random().nextInt(TEST_NIGHTLY ? 3 : 2) + 1;
+    }
+
+    int numNodes = sliceCount * (numRealtimeOrTlogReplicas + numPullReplicas);
+    fixShardCount(numNodes);
+    log.info("Starting ChaosMonkey test with {} shards and {} nodes", sliceCount, numNodes);
+  }
+  
+  @Test
+  public void test() throws Exception {
+    DocCollection docCollection = cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION);
+    assertEquals(this.sliceCount, docCollection.getSlices().size());
+    Slice s = docCollection.getSlice("shard1");
+    assertNotNull(s);
+    assertEquals("Unexpected number of replicas. Collection: " + docCollection, numRealtimeOrTlogReplicas + numPullReplicas, s.getReplicas().size());
+    assertEquals("Unexpected number of pull replicas. Collection: " + docCollection, numPullReplicas, s.getReplicas(EnumSet.of(Replica.Type.PULL)).size());
+    assertEquals(useTlogReplicas()?0:numRealtimeOrTlogReplicas, s.getReplicas(EnumSet.of(Replica.Type.NRT)).size());
+    assertEquals(useTlogReplicas()?numRealtimeOrTlogReplicas:0, s.getReplicas(EnumSet.of(Replica.Type.TLOG)).size());
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+    
+    // randomly turn on 1 seconds 'soft' commit
+    randomlyEnableAutoSoftCommit();
+
+    tryDelete();
+    
+    List<StoppableThread> 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();
+    }
+    
+    StoppableCommitThread commitThread = new StoppableCommitThread(cloudClient, 1000, false);
+    threads.add(commitThread);
+    commitThread.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)];
+      }
+      
+      ChaosMonkey.wait(runLength, DEFAULT_COLLECTION, cloudClient.getZkStateReader());
+    } finally {
+      chaosMonkey.stopTheMonkey();
+    }
+    
+    for (StoppableThread thread : threads) {
+      thread.safeStop();
+    }
+    
+    // wait for stop...
+    for (StoppableThread thread : threads) {
+      thread.join();
+    }
+    
+    for (StoppableThread thread : threads) {
+      if (thread instanceof StoppableIndexingThread) {
+        assertEquals(0, ((StoppableIndexingThread)thread).getFailCount());
+      }
+    }
+    
+    // try and wait for any replications and what not to finish...
+
+    Thread.sleep(2000);
+
+    waitForThingsToLevelOut(180000);
+    
+    // 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(180000);
+    
+    log.info("control docs:" + controlClient.query(new SolrQuery("*:*")).getResults().getNumFound() + "\n\n");
+    
+    waitForReplicationFromReplicas(DEFAULT_COLLECTION, cloudClient.getZkStateReader(), new TimeOut(30, TimeUnit.SECONDS));
+//    waitForAllWarmingSearchers();
+
+    checkShardConsistency(batchSize == 1, true);
+    
+    // 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();
+    }
+
+    try (CloudSolrClient client = createCloudClient("collection1")) {
+        createCollection(null, "testcollection", 1, 1, 100, client, null, "conf1");
+
+    }
+    List<Integer> numShardsNumReplicas = new ArrayList<>(2);
+    numShardsNumReplicas.add(1);
+    numShardsNumReplicas.add(1 + getPullReplicaCount());
+    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();
+      }
+      Thread.sleep(100);
+    }
+  }
+  
+  // skip the randoms - they can deadlock...
+  @Override
+  protected void indexr(Object... fields) throws Exception {
+    SolrInputDocument doc = new SolrInputDocument();
+    addFields(doc, fields);
+    addFields(doc, "rnd_b", true);
+    indexDoc(doc);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
index ed9ed41..ea8598b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
@@ -286,7 +286,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
     // first we make a core with the core name the collections api
     // will try and use - this will cause our mock fail
     Create createCmd = new Create();
-    createCmd.setCoreName("halfcollection_shard1_replica1");
+    createCmd.setCoreName(Assign.buildCoreName("halfcollection", "shard1", Replica.Type.NRT, 1));
     createCmd.setCollection("halfcollectionblocker");
     String dataDir = createTempDir().toFile().getAbsolutePath();
     createCmd.setDataDir(dataDir);
@@ -298,7 +298,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
     }
 
     createCmd = new Create();
-    createCmd.setCoreName("halfcollection_shard1_replica1");
+    createCmd.setCoreName(Assign.buildCoreName("halfcollection", "shard1", Replica.Type.NRT, 1));
     createCmd.setCollection("halfcollectionblocker2");
     dataDir = createTempDir().toFile().getAbsolutePath();
     createCmd.setDataDir(dataDir);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index 0142c7a..643660b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -67,7 +67,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
     Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus();
     assertEquals(4, coresStatus.size());
     for (int i=0; i<4; i++) {
-      NamedList<Integer> status = coresStatus.get(collectionName + "_shard" + (i/2+1) + "_replica" + (i%2+1));
+      NamedList<Integer> status = coresStatus.get(Assign.buildCoreName(collectionName, "shard" + (i/2+1), Replica.Type.NRT, (i%2+1)));
       assertEquals(0, (int)status.get("status"));
       assertTrue(status.get("QTime") > 0);
     }
@@ -117,17 +117,17 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
 
   @Test
   public void testCreateAndDeleteShard() throws IOException, SolrServerException {
-
     // Create an implicit collection
     String collectionName = "solrj_implicit";
     CollectionAdminResponse response
-        = CollectionAdminRequest.createCollectionWithImplicitRouter(collectionName, "conf", "shardA,shardB", 1)
+        = CollectionAdminRequest.createCollectionWithImplicitRouter(collectionName, "conf", "shardA,shardB", 1, 1, 1)
+        .setMaxShardsPerNode(3)
         .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus();
-    assertEquals(2, coresStatus.size());
+    assertEquals(6, coresStatus.size());
 
     // Add a shard to the implicit collection
     response = CollectionAdminRequest.createShard(collectionName, "shardC").process(cluster.getSolrClient());
@@ -135,8 +135,10 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     coresStatus = response.getCollectionCoresStatus();
-    assertEquals(1, coresStatus.size());
-    assertEquals(0, (int) coresStatus.get(collectionName + "_shardC_replica1").get("status"));
+    assertEquals(3, coresStatus.size());
+    assertEquals(0, (int) coresStatus.get(Assign.buildCoreName(collectionName,  "shardC", Replica.Type.NRT, 1)).get("status"));
+    assertEquals(0, (int) coresStatus.get(Assign.buildCoreName(collectionName,  "shardC", Replica.Type.TLOG, 1)).get("status"));
+    assertEquals(0, (int) coresStatus.get(Assign.buildCoreName(collectionName,  "shardC", Replica.Type.PULL, 1)).get("status"));
 
     response = CollectionAdminRequest.deleteShard(collectionName, "shardC").process(cluster.getSolrClient());
 
@@ -174,8 +176,8 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus();
-    assertEquals(0, (int) coresStatus.get(collectionName + "_shard1_0_replica1").get("status"));
-    assertEquals(0, (int) coresStatus.get(collectionName + "_shard1_1_replica1").get("status"));
+    assertEquals(0, (int) coresStatus.get(Assign.buildCoreName(collectionName, "shard1_0" , Replica.Type.NRT, 1)).get("status"));
+    assertEquals(0, (int) coresStatus.get(Assign.buildCoreName(collectionName, "shard1_1" , Replica.Type.NRT, 1)).get("status"));
 
     waitForState("Expected all shards to be active and parent shard to be removed", collectionName, (n, c) -> {
       if (c.getSlice("shard1").getState() == Slice.State.ACTIVE)


[3/6] lucene-solr:jira/solr-8668: SOLR-10233: Add support for replica types

Posted by cp...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/DeleteNodeTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DeleteNodeTest.java b/solr/core/src/test/org/apache/solr/cloud/DeleteNodeTest.java
index 8d2f6f2..9f461f0 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DeleteNodeTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DeleteNodeTest.java
@@ -18,7 +18,6 @@
 package org.apache.solr.cloud;
 
 
-import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Set;
@@ -29,12 +28,9 @@ import org.apache.solr.client.solrj.response.RequestStatusState;
 import org.apache.solr.common.util.StrUtils;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class DeleteNodeTest extends SolrCloudTestCase {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
+  
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(6)
@@ -54,7 +50,10 @@ public class DeleteNodeTest extends SolrCloudTestCase {
     Set<String> liveNodes = cloudClient.getZkStateReader().getClusterState().getLiveNodes();
     ArrayList<String> l = new ArrayList<>(liveNodes);
     Collections.shuffle(l, random());
-    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(coll, "conf1", 5, 2);
+    CollectionAdminRequest.Create create = pickRandom(
+        CollectionAdminRequest.createCollection(coll, "conf1", 5, 2, 0, 0),
+        CollectionAdminRequest.createCollection(coll, "conf1", 5, 1, 1, 0),
+        CollectionAdminRequest.createCollection(coll, "conf1", 5, 0, 1, 1));
     create.setCreateNodeSet(StrUtils.join(l, ',')).setMaxShardsPerNode(3);
     cloudClient.request(create);
     String node2bdecommissioned = l.get(0);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java b/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java
index 5699a8f..4c6253e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java
@@ -19,9 +19,11 @@ package org.apache.solr.cloud;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.EnumSet;
 
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.CoreStatus;
+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;
@@ -101,11 +103,29 @@ public class DeleteReplicaTest extends SolrCloudTestCase {
   public void deleteReplicaByCount() throws Exception {
 
     final String collectionName = "deleteByCount";
-    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 3).process(cluster.getSolrClient());
+    pickRandom(
+        CollectionAdminRequest.createCollection(collectionName, "conf", 1, 3),
+        CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1, 1, 1),
+        CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1, 0, 2),
+        CollectionAdminRequest.createCollection(collectionName, "conf", 1, 0, 1, 2))
+    .process(cluster.getSolrClient());
     waitForState("Expected a single shard with three replicas", collectionName, clusterShape(1, 3));
 
     CollectionAdminRequest.deleteReplicasFromShard(collectionName, "shard1", 2).process(cluster.getSolrClient());
     waitForState("Expected a single shard with a single replica", collectionName, clusterShape(1, 1));
+    
+    try {
+      CollectionAdminRequest.deleteReplicasFromShard(collectionName, "shard1", 1).process(cluster.getSolrClient());
+      fail("Expected Exception, Can't delete the last replica by count");
+    } catch (SolrException e) {
+      // expected
+      assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, e.code());
+      assertTrue(e.getMessage().contains("There is only one replica available"));
+    }
+    DocCollection docCollection = getCollectionState(collectionName);
+    // We know that since leaders are preserved, PULL replicas should not be left alone in the shard
+    assertEquals(0, docCollection.getSlice("shard1").getReplicas(EnumSet.of(Replica.Type.PULL)).size());
+    
 
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
index 8904ea8..8f35c88 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
@@ -58,8 +58,8 @@ public class ForceLeaderTest extends HttpPartitionTest {
   private final boolean onlyLeaderIndexes = random().nextBoolean();
 
   @Override
-  protected int getRealtimeReplicas() {
-    return onlyLeaderIndexes? 1 : -1;
+  protected boolean useTlogReplicas() {
+    return onlyLeaderIndexes;
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/FullThrottleStoppableIndexingThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/FullThrottleStoppableIndexingThread.java b/solr/core/src/test/org/apache/solr/cloud/FullThrottleStoppableIndexingThread.java
new file mode 100644
index 0000000..b9e177a
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/FullThrottleStoppableIndexingThread.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.net.ConnectException;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.http.client.HttpClient;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.util.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class FullThrottleStoppableIndexingThread extends StoppableIndexingThread {
+  
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  /**
+   * 
+   */
+  private CloseableHttpClient httpClient = HttpClientUtil.createClient(null);
+  private volatile boolean stop = false;
+  int clientIndex = 0;
+  private ConcurrentUpdateSolrClient cusc;
+  private List<SolrClient> clients;
+  private AtomicInteger fails = new AtomicInteger();
+  
+  public FullThrottleStoppableIndexingThread(SolrClient controlClient, CloudSolrClient cloudClient, List<SolrClient> clients,
+                                             String id, boolean doDeletes, int clientSoTimeout) {
+    super(controlClient, cloudClient, id, doDeletes);
+    setName("FullThrottleStopableIndexingThread");
+    setDaemon(true);
+    this.clients = clients;
+
+    cusc = new ErrorLoggingConcurrentUpdateSolrClient(((HttpSolrClient) clients.get(0)).getBaseURL(), httpClient, 8, 2);
+    cusc.setConnectionTimeout(10000);
+    cusc.setSoTimeout(clientSoTimeout);
+  }
+  
+  @Override
+  public void run() {
+    int i = 0;
+    int numDeletes = 0;
+    int numAdds = 0;
+
+    while (true && !stop) {
+      String id = this.id + "-" + i;
+      ++i;
+      
+      if (doDeletes && LuceneTestCase.random().nextBoolean() && deletes.size() > 0) {
+        String delete = deletes.remove(0);
+        try {
+          numDeletes++;
+          cusc.deleteById(delete);
+        } catch (Exception e) {
+          changeUrlOnError(e);
+          fails.incrementAndGet();
+        }
+      }
+      
+      try {
+        numAdds++;
+        if (numAdds > (LuceneTestCase.TEST_NIGHTLY ? 4002 : 197))
+          continue;
+        SolrInputDocument doc = AbstractFullDistribZkTestBase.getDoc(
+            "id",
+            id,
+            i1,
+            50,
+            t1,
+            "Saxon heptarchies that used to rip around so in old times and raise Cain.  My, you ought to seen old Henry the Eight when he was in bloom.  He WAS a blossom.  He used to marry a new wife every day, and chop off her head next morning.  And he would do it just as indifferent as if ");
+        cusc.add(doc);
+      } catch (Exception e) {
+        changeUrlOnError(e);
+        fails.incrementAndGet();
+      }
+      
+      if (doDeletes && LuceneTestCase.random().nextBoolean()) {
+        deletes.add(id);
+      }
+      
+    }
+
+    log.info("FT added docs:" + numAdds + " with " + fails + " fails" + " deletes:" + numDeletes);
+  }
+
+  private void changeUrlOnError(Exception e) {
+    if (e instanceof ConnectException) {
+      clientIndex++;
+      if (clientIndex > clients.size() - 1) {
+        clientIndex = 0;
+      }
+      cusc.shutdownNow();
+      cusc = new ErrorLoggingConcurrentUpdateSolrClient(((HttpSolrClient) clients.get(clientIndex)).getBaseURL(),
+          httpClient, 30, 3);
+    }
+  }
+  
+  @Override
+  public void safeStop() {
+    stop = true;
+    cusc.blockUntilFinished();
+    cusc.shutdownNow();
+    IOUtils.closeQuietly(httpClient);
+  }
+
+  @Override
+  public int getFailCount() {
+    return fails.get();
+  }
+  
+  @Override
+  public Set<String> getAddFails() {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public Set<String> getDeleteFails() {
+    throw new UnsupportedOperationException();
+  }
+  
+  static class ErrorLoggingConcurrentUpdateSolrClient extends ConcurrentUpdateSolrClient {
+    @SuppressWarnings("deprecation")
+    public ErrorLoggingConcurrentUpdateSolrClient(String serverUrl, HttpClient httpClient, int queueSize, int threadCount) {
+      super(serverUrl, httpClient, queueSize, threadCount, null, false);
+    }
+    @Override
+    public void handleError(Throwable ex) {
+      log.warn("cusc error", ex);
+    }
+  }
+  
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
index 01002cf..2cc1c30 100644
--- a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
@@ -85,8 +85,8 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
   }
 
   @Override
-  protected int getRealtimeReplicas() {
-    return onlyLeaderIndexes? 1 : -1;
+  protected boolean useTlogReplicas() {
+    return onlyLeaderIndexes;
   }
 
   /**
@@ -110,10 +110,10 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
    */
   @Override
   public JettySolrRunner createJetty(File solrHome, String dataDir,
-      String shardList, String solrConfigOverride, String schemaOverride)
+      String shardList, String solrConfigOverride, String schemaOverride, Replica.Type replicaType)
       throws Exception
   {
-    return createProxiedJetty(solrHome, dataDir, shardList, solrConfigOverride, schemaOverride);
+    return createProxiedJetty(solrHome, dataDir, shardList, solrConfigOverride, schemaOverride, replicaType);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java
index 457b9d9..f3965ac 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java
@@ -46,8 +46,8 @@ public class LeaderInitiatedRecoveryOnCommitTest extends BasicDistributedZkTest
   }
 
   @Override
-  protected int getRealtimeReplicas() {
-    return onlyLeaderIndexes? 1 : -1;
+  protected boolean useTlogReplicas() {
+    return onlyLeaderIndexes;
   }
 
   @Override
@@ -161,9 +161,9 @@ public class LeaderInitiatedRecoveryOnCommitTest extends BasicDistributedZkTest
    */
   @Override
   public JettySolrRunner createJetty(File solrHome, String dataDir,
-                                     String shardList, String solrConfigOverride, String schemaOverride)
+                                     String shardList, String solrConfigOverride, String schemaOverride, Replica.Type replicaType)
       throws Exception {
-    return createProxiedJetty(solrHome, dataDir, shardList, solrConfigOverride, schemaOverride);
+    return createProxiedJetty(solrHome, dataDir, shardList, solrConfigOverride, schemaOverride, replicaType);
   }
 
   protected void sendCommitWithRetry(Replica replica) throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/OnlyLeaderIndexesTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OnlyLeaderIndexesTest.java b/solr/core/src/test/org/apache/solr/cloud/OnlyLeaderIndexesTest.java
deleted file mode 100644
index a4e8d6f..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/OnlyLeaderIndexesTest.java
+++ /dev/null
@@ -1,435 +0,0 @@
-/*
- * 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.Semaphore;
-
-import org.apache.lucene.index.IndexWriter;
-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.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.UpdateRequest;
-import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.common.SolrDocument;
-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.ZkNodeProps;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.core.SolrCore;
-import org.apache.solr.update.DirectUpdateHandler2;
-import org.apache.solr.update.SolrIndexWriter;
-import org.apache.solr.update.UpdateHandler;
-import org.apache.solr.update.UpdateLog;
-import org.apache.solr.util.RefCounted;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class OnlyLeaderIndexesTest extends SolrCloudTestCase {
-  private static final String COLLECTION = "collection1";
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
-    System.setProperty("solr.ulog.numRecordsToKeep", "1000");
-
-    configureCluster(3)
-        .addConfig("config", TEST_PATH().resolve("configsets")
-        .resolve("cloud-minimal-inplace-updates").resolve("conf"))
-        .configure();
-
-    CollectionAdminRequest
-        .createCollection(COLLECTION, "config", 1, 3)
-        .setRealtimeReplicas(1)
-        .setMaxShardsPerNode(1)
-        .process(cluster.getSolrClient());
-    AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(),
-        false, true, 30);
-  }
-
-  @Test
-  public void test() throws Exception {
-    basicTest();
-    recoveryTest();
-    dbiTest();
-    basicLeaderElectionTest();
-    outOfOrderDBQWithInPlaceUpdatesTest();
-  }
-
-  public void basicTest() throws Exception {
-    CloudSolrClient cloudClient = cluster.getSolrClient();
-    new UpdateRequest()
-        .add(sdoc("id", "1"))
-        .add(sdoc("id", "2"))
-        .add(sdoc("id", "3"))
-        .add(sdoc("id", "4"))
-        .process(cloudClient, COLLECTION);
-
-    {
-      UpdateHandler updateHandler = getSolrCore(true).get(0).getUpdateHandler();
-      RefCounted<IndexWriter> iwRef = updateHandler.getSolrCoreState().getIndexWriter(null);
-      assertTrue("IndexWriter at leader must see updates ", iwRef.get().hasUncommittedChanges());
-      iwRef.decref();
-    }
-
-    for (SolrCore solrCore : getSolrCore(false)) {
-      RefCounted<IndexWriter> iwRef = solrCore.getUpdateHandler().getSolrCoreState().getIndexWriter(null);
-      assertFalse("IndexWriter at replicas must not see updates ", iwRef.get().hasUncommittedChanges());
-      iwRef.decref();
-    }
-
-    checkRTG(1, 4, cluster.getJettySolrRunners());
-
-    new UpdateRequest()
-        .deleteById("1")
-        .deleteByQuery("id:2")
-        .process(cloudClient, COLLECTION);
-
-    // The DBQ is not processed at replicas, so we still can get doc2 and other docs by RTG
-    checkRTG(2,4, getSolrRunner(false));
-
-    new UpdateRequest()
-        .commit(cloudClient, COLLECTION);
-
-    checkShardConsistency(2, 1);
-
-    // Update log roll over
-    for (SolrCore solrCore : getSolrCore(false)) {
-      UpdateLog updateLog = solrCore.getUpdateHandler().getUpdateLog();
-      assertFalse(updateLog.hasUncommittedChanges());
-    }
-
-    // UpdateLog copy over old updates
-    for (int i = 15; i <= 150; i++) {
-      cloudClient.add(COLLECTION, sdoc("id",String.valueOf(i)));
-      if (random().nextInt(100) < 15 & i != 150) {
-        cloudClient.commit(COLLECTION);
-      }
-    }
-    checkRTG(120,150, cluster.getJettySolrRunners());
-    waitForReplicasCatchUp(20);
-  }
-
-  public void recoveryTest() throws Exception {
-    CloudSolrClient cloudClient = cluster.getSolrClient();
-    new UpdateRequest()
-        .deleteByQuery("*:*")
-        .commit(cluster.getSolrClient(), COLLECTION);
-    new UpdateRequest()
-        .add(sdoc("id", "3"))
-        .add(sdoc("id", "4"))
-        .commit(cloudClient, COLLECTION);
-    // Replica recovery
-    new UpdateRequest()
-        .add(sdoc("id", "5"))
-        .process(cloudClient, COLLECTION);
-    JettySolrRunner solrRunner = getSolrRunner(false).get(0);
-    ChaosMonkey.stop(solrRunner);
-    new UpdateRequest()
-        .add(sdoc("id", "6"))
-        .process(cloudClient, COLLECTION);
-    ChaosMonkey.start(solrRunner);
-    AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(),
-        false, true, 30);
-    // We skip peerSync, so replica will always trigger commit on leader
-    checkShardConsistency(4, 20);
-
-    // LTR can be kicked off, so waiting for replicas recovery
-    new UpdateRequest()
-        .add(sdoc("id", "7"))
-        .commit(cloudClient, COLLECTION);
-    AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(),
-        false, true, 30);
-    checkShardConsistency(5, 20);
-
-    // More Replica recovery testing
-    new UpdateRequest()
-        .add(sdoc("id", "8"))
-        .process(cloudClient, COLLECTION);
-    checkRTG(3,8, cluster.getJettySolrRunners());
-    DirectUpdateHandler2.commitOnClose = false;
-    ChaosMonkey.stop(solrRunner);
-    DirectUpdateHandler2.commitOnClose = true;
-    ChaosMonkey.start(solrRunner);
-    AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(),
-        false, true, 30);
-    checkRTG(3,8, cluster.getJettySolrRunners());
-    checkShardConsistency(6, 20);
-
-    // Test replica recovery apply buffer updates
-    Semaphore waitingForBufferUpdates = new Semaphore(0);
-    Semaphore waitingForReplay = new Semaphore(0);
-    RecoveryStrategy.testing_beforeReplayBufferingUpdates = () -> {
-      try {
-        waitingForReplay.release();
-        waitingForBufferUpdates.acquire();
-      } catch (InterruptedException e) {
-        e.printStackTrace();
-      }
-    };
-    ChaosMonkey.stop(solrRunner);
-    ChaosMonkey.start(solrRunner);
-    waitingForReplay.acquire();
-    new UpdateRequest()
-        .add(sdoc("id", "9"))
-        .add(sdoc("id", "10"))
-        .process(cloudClient, COLLECTION);
-    waitingForBufferUpdates.release();
-    RecoveryStrategy.testing_beforeReplayBufferingUpdates = null;
-    AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(),
-        false, true, 30);
-    checkRTG(3,10, cluster.getJettySolrRunners());
-    checkShardConsistency(6, 20);
-    for (SolrCore solrCore : getSolrCore(false)) {
-      RefCounted<IndexWriter> iwRef = solrCore.getUpdateHandler().getSolrCoreState().getIndexWriter(null);
-      assertFalse("IndexWriter at replicas must not see updates ", iwRef.get().hasUncommittedChanges());
-      iwRef.decref();
-    }
-  }
-
-  public void dbiTest() throws Exception{
-    CloudSolrClient cloudClient = cluster.getSolrClient();
-    new UpdateRequest()
-        .deleteByQuery("*:*")
-        .commit(cluster.getSolrClient(), COLLECTION);
-    new UpdateRequest()
-        .add(sdoc("id", "1"))
-        .commit(cloudClient, COLLECTION);
-    checkShardConsistency(1, 1);
-    new UpdateRequest()
-        .deleteById("1")
-        .process(cloudClient, COLLECTION);
-    try {
-      checkRTG(1, 1, cluster.getJettySolrRunners());
-    } catch (AssertionError e) {
-      return;
-    }
-    fail("Doc1 is deleted but it's still exist");
-  }
-
-  public void basicLeaderElectionTest() throws Exception {
-    CloudSolrClient cloudClient = cluster.getSolrClient();
-    new UpdateRequest()
-        .deleteByQuery("*:*")
-        .commit(cluster.getSolrClient(), COLLECTION);
-    new UpdateRequest()
-        .add(sdoc("id", "1"))
-        .add(sdoc("id", "2"))
-        .process(cloudClient, COLLECTION);
-    String oldLeader = getLeader();
-    JettySolrRunner oldLeaderJetty = getSolrRunner(true).get(0);
-    ChaosMonkey.kill(oldLeaderJetty);
-    for (int i = 0; i < 60; i++) { // wait till leader is changed
-      if (!oldLeader.equals(getLeader())) {
-        break;
-      }
-      Thread.sleep(100);
-    }
-    new UpdateRequest()
-        .add(sdoc("id", "3"))
-        .add(sdoc("id", "4"))
-        .process(cloudClient, COLLECTION);
-    ChaosMonkey.start(oldLeaderJetty);
-    AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(),
-        false, true, 60);
-    checkRTG(1,4, cluster.getJettySolrRunners());
-    new UpdateRequest()
-        .commit(cloudClient, COLLECTION);
-    checkShardConsistency(4,1);
-  }
-
-  private String getLeader() throws InterruptedException {
-    ZkNodeProps props = cluster.getSolrClient().getZkStateReader().getLeaderRetry("collection1", "shard1", 30000);
-    return props.getStr(ZkStateReader.NODE_NAME_PROP);
-  }
-
-  public void outOfOrderDBQWithInPlaceUpdatesTest() throws Exception {
-    new UpdateRequest()
-        .deleteByQuery("*:*")
-        .commit(cluster.getSolrClient(), COLLECTION);
-    List<UpdateRequest> updates = new ArrayList<>();
-    updates.add(simulatedUpdateRequest(null, "id", 1, "title_s", "title0_new", "inplace_updatable_int", 5, "_version_", Long.MAX_VALUE-100)); // full update
-    updates.add(simulatedDBQ("inplace_updatable_int:5", Long.MAX_VALUE-98));
-    updates.add(simulatedUpdateRequest(Long.MAX_VALUE-100, "id", 1, "inplace_updatable_int", 6, "_version_", Long.MAX_VALUE-99));
-    for (JettySolrRunner solrRunner: getSolrRunner(false)) {
-      try (SolrClient client = solrRunner.newClient()) {
-        for (UpdateRequest up : updates) {
-          up.process(client, COLLECTION);
-        }
-      }
-    }
-    JettySolrRunner oldLeaderJetty = getSolrRunner(true).get(0);
-    ChaosMonkey.kill(oldLeaderJetty);
-    AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(),
-        false, true, 30);
-    ChaosMonkey.start(oldLeaderJetty);
-    AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(),
-        false, true, 30);
-    new UpdateRequest()
-        .add(sdoc("id", "2"))
-        .commit(cluster.getSolrClient(), COLLECTION);
-    checkShardConsistency(2,20);
-    SolrDocument doc = cluster.getSolrClient().getById(COLLECTION,"1");
-    assertNotNull(doc.get("title_s"));
-  }
-
-  private UpdateRequest simulatedUpdateRequest(Long prevVersion, Object... fields) throws SolrServerException, IOException {
-    SolrInputDocument doc = sdoc(fields);
-
-    // get baseUrl of the leader
-    String baseUrl = getBaseUrl();
-
-    UpdateRequest ur = new UpdateRequest();
-    ur.add(doc);
-    ur.setParam("update.distrib", "FROMLEADER");
-    if (prevVersion != null) {
-      ur.setParam("distrib.inplace.prevversion", String.valueOf(prevVersion));
-      ur.setParam("distrib.inplace.update", "true");
-    }
-    ur.setParam("distrib.from", baseUrl);
-    return ur;
-  }
-
-  private UpdateRequest simulatedDBQ(String query, long version) throws SolrServerException, IOException {
-    String baseUrl = getBaseUrl();
-
-    UpdateRequest ur = new UpdateRequest();
-    ur.deleteByQuery(query);
-    ur.setParam("_version_", ""+version);
-    ur.setParam("update.distrib", "FROMLEADER");
-    ur.setParam("distrib.from", baseUrl);
-    return ur;
-  }
-
-  private String getBaseUrl() {
-    DocCollection collection = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION);
-    Slice slice = collection.getSlice("shard1");
-    return slice.getLeader().getCoreUrl();
-  }
-
-  private void checkRTG(int from, int to, List<JettySolrRunner> solrRunners) throws Exception{
-
-    for (JettySolrRunner solrRunner: solrRunners) {
-      try (SolrClient client = solrRunner.newClient()) {
-        for (int i = from; i <= to; i++) {
-          SolrQuery query = new SolrQuery("*:*");
-          query.set("distrib", false);
-          query.setRequestHandler("/get");
-          query.set("id",i);
-          QueryResponse res = client.query(COLLECTION, query);
-          assertNotNull("Can not find doc "+ i + " in " + solrRunner.getBaseUrl(),res.getResponse().get("doc"));
-        }
-      }
-    }
-
-  }
-
-  private void checkShardConsistency(int expected, int numTry) throws Exception{
-
-    for (int i = 0; i < numTry; i++) {
-      boolean inSync = true;
-      for (JettySolrRunner solrRunner: cluster.getJettySolrRunners()) {
-        try (SolrClient client = solrRunner.newClient()) {
-          SolrQuery query = new SolrQuery("*:*");
-          query.set("distrib", false);
-          long results = client.query(COLLECTION, query).getResults().getNumFound();
-          if (expected != results) {
-            inSync = false;
-            Thread.sleep(500);
-            break;
-          }
-        }
-      }
-      if (inSync) return;
-    }
-
-    fail("Some replicas are not in sync with leader");
-  }
-
-  private void waitForReplicasCatchUp(int numTry) throws IOException, InterruptedException {
-    String leaderTimeCommit = getSolrCore(true).get(0).getDeletionPolicy().getLatestCommit().getUserData().get(SolrIndexWriter.COMMIT_TIME_MSEC_KEY);
-    if (leaderTimeCommit == null) return;
-    for (int i = 0; i < numTry; i++) {
-      boolean inSync = true;
-      for (SolrCore solrCore : getSolrCore(false)) {
-        String replicateTimeCommit = solrCore.getDeletionPolicy().getLatestCommit().getUserData().get(SolrIndexWriter.COMMIT_TIME_MSEC_KEY);
-        if (!leaderTimeCommit.equals(replicateTimeCommit)) {
-          inSync = false;
-          Thread.sleep(500);
-          break;
-        }
-      }
-      if (inSync) return;
-    }
-
-    fail("Some replicas are not in sync with leader");
-
-  }
-
-  private List<SolrCore> getSolrCore(boolean isLeader) {
-    List<SolrCore> rs = new ArrayList<>();
-
-    CloudSolrClient cloudClient = cluster.getSolrClient();
-    DocCollection docCollection = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION);
-
-    for (JettySolrRunner solrRunner : cluster.getJettySolrRunners()) {
-      if (solrRunner.getCoreContainer() == null) continue;
-      for (SolrCore solrCore : solrRunner.getCoreContainer().getCores()) {
-        CloudDescriptor cloudDescriptor = solrCore.getCoreDescriptor().getCloudDescriptor();
-        Slice slice = docCollection.getSlice(cloudDescriptor.getShardId());
-        Replica replica = docCollection.getReplica(cloudDescriptor.getCoreNodeName());
-        if (slice.getLeader() == replica && isLeader) {
-          rs.add(solrCore);
-        } else if (slice.getLeader() != replica && !isLeader) {
-          rs.add(solrCore);
-        }
-      }
-    }
-    return rs;
-  }
-
-  private List<JettySolrRunner> getSolrRunner(boolean isLeader) {
-    List<JettySolrRunner> rs = new ArrayList<>();
-
-    CloudSolrClient cloudClient = cluster.getSolrClient();
-    DocCollection docCollection = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION);
-
-    for (JettySolrRunner solrRunner : cluster.getJettySolrRunners()) {
-      if (solrRunner.getCoreContainer() == null) continue;
-      for (SolrCore solrCore : solrRunner.getCoreContainer().getCores()) {
-        CloudDescriptor cloudDescriptor = solrCore.getCoreDescriptor().getCloudDescriptor();
-        Slice slice = docCollection.getSlice(cloudDescriptor.getShardId());
-        Replica replica = docCollection.getReplica(cloudDescriptor.getCoreNodeName());
-        if (slice.getLeader() == replica && isLeader) {
-          rs.add(solrRunner);
-        } else if (slice.getLeader() != replica && !isLeader) {
-          rs.add(solrRunner);
-        }
-      }
-    }
-    return rs;
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
index 48ac91f..91da2c1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
@@ -27,6 +27,7 @@ import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.cloud.Overseer.LeaderStatus;
 import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent;
 import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -346,7 +347,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
     assertEquals(numberOfSlices * numberOfReplica, coreNames.size());
     for (int i = 1; i <= numberOfSlices; i++) {
       for (int j = 1; j <= numberOfReplica; j++) {
-        String coreName = COLLECTION_NAME + "_shard" + i + "_replica" + j;
+        String coreName = Assign.buildCoreName(COLLECTION_NAME, "shard" + i, Replica.Type.NRT, j);
         assertTrue("Shard " + coreName + " was not created",
             coreNames.contains(coreName));
         

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java b/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java
index a8e14bf..8290e12 100644
--- a/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/RecoveryAfterSoftCommitTest.java
@@ -40,8 +40,8 @@ public class RecoveryAfterSoftCommitTest extends AbstractFullDistribZkTestBase {
   }
 
   @Override
-  protected int getRealtimeReplicas() {
-    return onlyLeaderIndexes? 1: -1;
+  protected boolean useTlogReplicas() {
+    return onlyLeaderIndexes;
   }
 
   @BeforeClass
@@ -64,10 +64,10 @@ public class RecoveryAfterSoftCommitTest extends AbstractFullDistribZkTestBase {
    */
   @Override
   public JettySolrRunner createJetty(File solrHome, String dataDir,
-                                     String shardList, String solrConfigOverride, String schemaOverride)
+                                     String shardList, String solrConfigOverride, String schemaOverride, Replica.Type replicaType)
       throws Exception
   {
-    return createProxiedJetty(solrHome, dataDir, shardList, solrConfigOverride, schemaOverride);
+    return createProxiedJetty(solrHome, dataDir, shardList, solrConfigOverride, schemaOverride, replicaType);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/ReplaceNodeTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ReplaceNodeTest.java b/solr/core/src/test/org/apache/solr/cloud/ReplaceNodeTest.java
index 1c7575d..e1af607 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ReplaceNodeTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ReplaceNodeTest.java
@@ -21,6 +21,7 @@ package org.apache.solr.cloud;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.Set;
 
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
@@ -29,6 +30,9 @@ 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.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.util.StrUtils;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -60,7 +64,12 @@ public class ReplaceNodeTest extends SolrCloudTestCase {
     Collections.shuffle(l, random());
     String emptyNode = l.remove(0);
     String node2bdecommissioned = l.get(0);
-    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(coll, "conf1", 5, 2);
+    CollectionAdminRequest.Create create;
+    create = pickRandom(CollectionAdminRequest.createCollection(coll, "conf1", 5, 2),
+                        CollectionAdminRequest.createCollection(coll, "conf1", 5, 1,1,0),
+                        CollectionAdminRequest.createCollection(coll, "conf1", 5, 0,1,1),
+                        CollectionAdminRequest.createCollection(coll, "conf1", 5, 1,0,1),
+                        CollectionAdminRequest.createCollection(coll, "conf1", 5, 0,2,0));
     create.setCreateNodeSet(StrUtils.join(l, ',')).setMaxShardsPerNode(3);
     cloudClient.request(create);
     log.info("excluded_node : {}  ", emptyNode);
@@ -98,7 +107,15 @@ public class ReplaceNodeTest extends SolrCloudTestCase {
     assertTrue(success);
     try (HttpSolrClient coreclient = getHttpSolrClient(cloudClient.getZkStateReader().getBaseUrlForNodeName(emptyNode))) {
       CoreAdminResponse status = CoreAdminRequest.getStatus(null, coreclient);
-      assertTrue(status.getCoreStatus().size() == 0);
+      assertEquals("Expecting no cores but found some: " + status.getCoreStatus(), 0, status.getCoreStatus().size());
+    }
+    
+    DocCollection collection = cloudClient.getZkStateReader().getClusterState().getCollection(coll);
+    assertEquals(create.getNumShards().intValue(), collection.getSlices().size());
+    for (Slice s:collection.getSlices()) {
+      assertEquals(create.getNumNrtReplicas().intValue(), s.getReplicas(EnumSet.of(Replica.Type.NRT)).size());
+      assertEquals(create.getNumTlogReplicas().intValue(), s.getReplicas(EnumSet.of(Replica.Type.TLOG)).size());
+      assertEquals(create.getNumPullReplicas().intValue(), s.getReplicas(EnumSet.of(Replica.Type.PULL)).size());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java b/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java
index 9100eee..abd394a 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java
@@ -64,10 +64,10 @@ public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
    */
   @Override
   public JettySolrRunner createJetty(File solrHome, String dataDir,
-      String shardList, String solrConfigOverride, String schemaOverride)
+      String shardList, String solrConfigOverride, String schemaOverride, Replica.Type replicaType)
       throws Exception {
 
-    return createProxiedJetty(solrHome, dataDir, shardList, solrConfigOverride, schemaOverride);
+    return createProxiedJetty(solrHome, dataDir, shardList, solrConfigOverride, schemaOverride, replicaType);
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java b/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
index bf9b5e0..72f0694 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
@@ -86,12 +86,6 @@ public class ShardSplitTest extends BasicDistributedZkTest {
     useFactory(null);
   }
 
-  //TODO for now, onlyLeaderIndexes do not work with ShardSplitTest
-  @Override
-  protected int getRealtimeReplicas() {
-    return -1;
-  }
-
   @Test
   public void test() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
index b592861..c7fc0e8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
@@ -66,8 +66,7 @@ public class TestCloudRecovery extends SolrCloudTestCase {
 
     onlyLeaderIndexes = random().nextBoolean();
     CollectionAdminRequest
-        .createCollection(COLLECTION, "config", 2, 2)
-        .setRealtimeReplicas(onlyLeaderIndexes? 1: -1)
+        .createCollection(COLLECTION, "config", 2, onlyLeaderIndexes?0:2,onlyLeaderIndexes?2:0,0)
         .setMaxShardsPerNode(2)
         .process(cluster.getSolrClient());
     AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(),
@@ -111,7 +110,7 @@ public class TestCloudRecovery extends SolrCloudTestCase {
     assertEquals(4, resp.getResults().getNumFound());
     // Make sure all nodes is recover from tlog
     if (onlyLeaderIndexes) {
-      // Leader election can be kicked off, so 2 append replicas will replay its tlog before becoming new leader
+      // Leader election can be kicked off, so 2 tlog replicas will replay its tlog before becoming new leader
       assertTrue( countReplayLog.get() >=2);
     } else {
       assertEquals(4, countReplayLog.get());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java b/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
index 8fbfee3..74ad7bd 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
@@ -60,8 +60,12 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
   @ShardsFixed(num = 2)
   public void test() throws Exception {
     try (CloudSolrClient client = createCloudClient(null)) {
-      CollectionAdminRequest.Create req = CollectionAdminRequest.createCollection(COLLECTION_NAME, "conf1",2,2);
-      req.setRealtimeReplicas(1);
+      CollectionAdminRequest.Create req;
+      if (useTlogReplicas()) {
+        req = CollectionAdminRequest.createCollection(COLLECTION_NAME, "conf1",2, 0, 1, 1);
+      } else {
+        req = CollectionAdminRequest.createCollection(COLLECTION_NAME, "conf1",2, 1, 0, 1);
+      }
       req.setMaxShardsPerNode(2);
       client.request(req);
       createCollection(null, COLLECTION_NAME1, 1, 1, 1, client, null, "conf1");
@@ -173,7 +177,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
       Map<String, Object> collection = (Map<String, Object>) collections.get(COLLECTION_NAME);
       assertNotNull(collection);
       assertEquals("conf1", collection.get("configName"));
-      assertEquals("1", collection.get("realtimeReplicas"));
+//      assertEquals("1", collection.get("nrtReplicas"));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
new file mode 100644
index 0000000..cb732ff
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
@@ -0,0 +1,576 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import org.apache.http.client.HttpClient;
+import org.apache.lucene.util.LuceneTestCase.Slow;
+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.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+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.CollectionStatePredicate;
+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.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.apache.zookeeper.KeeperException;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.carrotsearch.randomizedtesting.annotations.Repeat;
+
+@Slow
+public class TestPullReplica extends SolrCloudTestCase {
+  
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  
+  private String collectionName = null;
+  private final static int REPLICATION_TIMEOUT_SECS = 10;
+  
+  private String suggestedCollectionName() {
+    return (getTestClass().getSimpleName().replace("Test", "") + "_" + getTestName().split(" ")[0]).replaceAll("(.)(\\p{Upper})", "$1_$2").toLowerCase(Locale.ROOT);
+  }
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    TestInjection.waitForReplicasInSync = null; // We'll be explicit about this in this test
+    configureCluster(2) // 2 + random().nextInt(3) 
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+    Boolean useLegacyCloud = rarely();
+    LOG.info("Using legacyCloud?: {}", useLegacyCloud);
+    CollectionAdminRequest.ClusterProp clusterPropRequest = CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, String.valueOf(useLegacyCloud));
+    CollectionAdminResponse response = clusterPropRequest.process(cluster.getSolrClient());
+    assertEquals(0, response.getStatus());
+  }
+  
+  @AfterClass
+  public static void tearDownCluster() {
+    TestInjection.reset();
+  }
+  
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    collectionName = suggestedCollectionName();
+    expectThrows(SolrException.class, () -> getCollectionState(collectionName));
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    for (JettySolrRunner jetty:cluster.getJettySolrRunners()) {
+      if (!jetty.isRunning()) {
+        LOG.warn("Jetty {} not running, probably some bad test. Starting it", jetty.getLocalPort());
+        ChaosMonkey.start(jetty);
+      }
+    }
+    if (cluster.getSolrClient().getZkStateReader().getClusterState().getCollectionOrNull(collectionName) != null) {
+      LOG.info("tearDown deleting collection");
+      CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
+      LOG.info("Collection deleted");
+      waitForDeletion(collectionName);
+    }
+    super.tearDown();
+  }
+  
+  @Repeat(iterations=2) // 2 times to make sure cleanup is complete and we can create the same collection
+  public void testCreateDelete() throws Exception {
+    try {
+      CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1, 0, 3)
+      .setMaxShardsPerNode(100)
+      .process(cluster.getSolrClient());
+      boolean reloaded = false;
+      while (true) {
+        DocCollection docCollection = getCollectionState(collectionName);
+        assertNotNull(docCollection);
+        assertEquals("Expecting 4 relpicas per shard",
+            8, docCollection.getReplicas().size());
+        assertEquals("Expecting 6 pull replicas, 3 per shard",
+            6, docCollection.getReplicas(EnumSet.of(Replica.Type.PULL)).size());
+        assertEquals("Expecting 2 writer replicas, one per shard",
+            2, docCollection.getReplicas(EnumSet.of(Replica.Type.NRT)).size());
+        for (Slice s:docCollection.getSlices()) {
+          // read-only replicas can never become leaders
+          assertFalse(s.getLeader().getType() == Replica.Type.PULL);
+          List<String> shardElectionNodes = cluster.getZkClient().getChildren(ZkStateReader.getShardLeadersElectPath(collectionName, s.getName()), null, true);
+          assertEquals("Unexpected election nodes for Shard: " + s.getName() + ": " + Arrays.toString(shardElectionNodes.toArray()), 
+              1, shardElectionNodes.size());
+        }
+        assertUlogPresence(docCollection);
+        if (reloaded) {
+          break;
+        } else {
+          // reload
+          CollectionAdminResponse response = CollectionAdminRequest.reloadCollection(collectionName)
+          .process(cluster.getSolrClient());
+          assertEquals(0, response.getStatus());
+          reloaded = true;
+        }
+      }
+    } finally {
+      zkClient().printLayoutToStdOut();
+    }
+  }
+  
+  /**
+   * Asserts that Update logs don't exist for replicas of type {@link org.apache.solr.common.cloud.Replica.Type#PULL}
+   */
+  private void assertUlogPresence(DocCollection collection) {
+    for (Slice s:collection.getSlices()) {
+      for (Replica r:s.getReplicas()) {
+        if (r.getType() == Replica.Type.NRT) {
+          continue;
+        }
+        SolrCore core = null;
+        try {
+          core = cluster.getReplicaJetty(r).getCoreContainer().getCore(r.getCoreName());
+          assertNotNull(core);
+          assertFalse("Update log should not exist for replicas of type Passive but file is present: " + core.getUlogDir(),
+              new java.io.File(core.getUlogDir()).exists());
+        } finally {
+          core.close();
+        }
+      }
+    }
+  }
+  
+  @SuppressWarnings("unchecked")
+  public void testAddDocs() throws Exception {
+    int numReadOnlyReplicas = 1 + random().nextInt(3);
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1, 0, numReadOnlyReplicas)
+    .setMaxShardsPerNode(100)
+    .process(cluster.getSolrClient());
+    waitForState("Expected collection to be created with 1 shard and " + (numReadOnlyReplicas + 1) + " replicas", collectionName, clusterShape(1, numReadOnlyReplicas + 1));
+    DocCollection docCollection = assertNumberOfReplicas(1, 0, numReadOnlyReplicas, false, true);
+    assertEquals(1, docCollection.getSlices().size());
+    
+    cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "1", "foo", "bar"));
+    cluster.getSolrClient().commit(collectionName);
+    
+    Slice s = docCollection.getSlices().iterator().next();
+    try (HttpSolrClient leaderClient = getHttpSolrClient(s.getLeader().getCoreUrl())) {
+      assertEquals(1, leaderClient.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+    
+    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+    for (Replica r:s.getReplicas(EnumSet.of(Replica.Type.PULL))) {
+      //TODO: assert replication < REPLICATION_TIMEOUT_SECS
+      try (HttpSolrClient readOnlyReplicaClient = getHttpSolrClient(r.getCoreUrl())) {
+        while (true) {
+          try {
+            assertEquals("Replica " + r.getName() + " not up to date after 10 seconds",
+                1, readOnlyReplicaClient.query(new SolrQuery("*:*")).getResults().getNumFound());
+            break;
+          } catch (AssertionError e) {
+            if (t.hasTimedOut()) {
+              throw e;
+            } else {
+              Thread.sleep(100);
+            }
+          }
+        }
+        SolrQuery req = new SolrQuery(
+            "qt", "/admin/plugins",
+            "stats", "true");
+        QueryResponse statsResponse = readOnlyReplicaClient.query(req);
+        assertEquals("Replicas shouldn't process the add document request: " + statsResponse, 
+            0L, ((Map<String, Object>)((NamedList<Object>)statsResponse.getResponse()).findRecursive("plugins", "UPDATE", "updateHandler", "stats")).get("UPDATE.updateHandler.adds"));
+      }
+    }
+    assertUlogPresence(docCollection);
+  }
+  
+  public void testAddRemovePullReplica() throws Exception {
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1, 0, 0)
+      .setMaxShardsPerNode(100)
+      .process(cluster.getSolrClient());
+    cluster.getSolrClient().getZkStateReader().registerCore(collectionName); //TODO: Why is this needed? see SOLR-9440
+    waitForState("Expected collection to be created with 2 shards and 1 replica each", collectionName, clusterShape(2, 1));
+    DocCollection docCollection = assertNumberOfReplicas(2, 0, 0, false, true);
+    assertEquals(2, docCollection.getSlices().size());
+    
+    CollectionAdminRequest.addReplicaToShard(collectionName, "shard1", Replica.Type.PULL).process(cluster.getSolrClient());
+    docCollection = assertNumberOfReplicas(2, 0, 1, true, false);
+    CollectionAdminRequest.addReplicaToShard(collectionName, "shard2", Replica.Type.PULL).process(cluster.getSolrClient());    
+    docCollection = assertNumberOfReplicas(2, 0, 2, true, false);
+    
+    waitForState("Expecting collection to have 2 shards and 2 replica each", collectionName, clusterShape(2, 2));
+    
+    //Delete pull replica from shard1
+    CollectionAdminRequest.deleteReplica(
+        collectionName, 
+        "shard1", 
+        docCollection.getSlice("shard1").getReplicas(EnumSet.of(Replica.Type.PULL)).get(0).getName())
+    .process(cluster.getSolrClient());
+    assertNumberOfReplicas(2, 0, 1, true, true);
+  }
+  
+  public void testRemoveAllWriterReplicas() throws Exception {
+    doTestNoLeader(true);
+  }
+  
+  public void testKillLeader() throws Exception {
+    doTestNoLeader(false);
+  }
+  
+  @Ignore("Ignore until I figure out a way to reliably record state transitions")
+  public void testPullReplicaStates() throws Exception {
+    // Validate that pull replicas go through the correct states when starting, stopping, reconnecting
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1, 0, 0)
+      .setMaxShardsPerNode(100)
+      .process(cluster.getSolrClient());
+//    cluster.getSolrClient().getZkStateReader().registerCore(collectionName); //TODO: Is this needed? 
+    waitForState("Replica not added", collectionName, activeReplicaCount(1, 0, 0));
+    addDocs(500);
+    List<Replica.State> statesSeen = new ArrayList<>(3);
+    cluster.getSolrClient().registerCollectionStateWatcher(collectionName, (liveNodes, collectionState) -> {
+      Replica r = collectionState.getSlice("shard1").getReplica("core_node2");
+      LOG.info("CollectionStateWatcher state change: {}", r);
+      if (r == null) {
+        return false;
+      }
+      statesSeen.add(r.getState());
+      LOG.info("CollectionStateWatcher saw state: {}", r.getState());
+      return r.getState() == Replica.State.ACTIVE;
+    });
+    CollectionAdminRequest.addReplicaToShard(collectionName, "shard1", Replica.Type.PULL).process(cluster.getSolrClient());
+    waitForState("Replica not added", collectionName, activeReplicaCount(1, 0, 1));
+    zkClient().printLayoutToStdOut();
+    LOG.info("Saw states: " + Arrays.toString(statesSeen.toArray()));
+    assertEquals("Expecting DOWN->RECOVERING->ACTIVE but saw: " + Arrays.toString(statesSeen.toArray()), 3, statesSeen.size());
+    assertEquals("Expecting DOWN->RECOVERING->ACTIVE but saw: " + Arrays.toString(statesSeen.toArray()), Replica.State.DOWN, statesSeen.get(0));
+    assertEquals("Expecting DOWN->RECOVERING->ACTIVE but saw: " + Arrays.toString(statesSeen.toArray()), Replica.State.RECOVERING, statesSeen.get(0));
+    assertEquals("Expecting DOWN->RECOVERING->ACTIVE but saw: " + Arrays.toString(statesSeen.toArray()), Replica.State.ACTIVE, statesSeen.get(0));
+  }
+  
+  public void testRealTimeGet() throws SolrServerException, IOException, KeeperException, InterruptedException {
+    // should be redirected to Replica.Type.NRT
+    int numReplicas = random().nextBoolean()?1:2;
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, numReplicas, 0, numReplicas)
+      .setMaxShardsPerNode(100)
+      .process(cluster.getSolrClient());
+    waitForState("Unexpected replica count", collectionName, activeReplicaCount(numReplicas, 0, numReplicas));
+    DocCollection docCollection = assertNumberOfReplicas(numReplicas, 0, numReplicas, false, true);
+    HttpClient httpClient = cluster.getSolrClient().getHttpClient();
+    int id = 0;
+    Slice slice = docCollection.getSlice("shard1");
+    List<String> ids = new ArrayList<>(slice.getReplicas().size());
+    for (Replica rAdd:slice.getReplicas()) {
+      try (HttpSolrClient client = getHttpSolrClient(rAdd.getCoreUrl(), httpClient)) {
+        client.add(new SolrInputDocument("id", String.valueOf(id), "foo_s", "bar"));
+      }
+      SolrDocument docCloudClient = cluster.getSolrClient().getById(collectionName, String.valueOf(id));
+      assertEquals("bar", docCloudClient.getFieldValue("foo_s"));
+      for (Replica rGet:slice.getReplicas()) {
+        try (HttpSolrClient client = getHttpSolrClient(rGet.getCoreUrl(), httpClient)) {
+          SolrDocument doc = client.getById(String.valueOf(id));
+          assertEquals("bar", doc.getFieldValue("foo_s"));
+        }
+      }
+      ids.add(String.valueOf(id));
+      id++;
+    }
+    SolrDocumentList previousAllIdsResult = null;
+    for (Replica rAdd:slice.getReplicas()) {
+      try (HttpSolrClient client = getHttpSolrClient(rAdd.getCoreUrl(), httpClient)) {
+        SolrDocumentList allIdsResult = client.getById(ids);
+        if (previousAllIdsResult != null) {
+          assertTrue(compareSolrDocumentList(previousAllIdsResult, allIdsResult));
+        } else {
+          // set the first response here
+          previousAllIdsResult = allIdsResult;
+          assertEquals("Unexpected number of documents", ids.size(), allIdsResult.getNumFound());
+        }
+      }
+      id++;
+    }
+  }
+  
+  /*
+   * validate that replication still happens on a new leader
+   */
+  private void doTestNoLeader(boolean removeReplica) throws Exception {
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1, 0, 1)
+      .setMaxShardsPerNode(100)
+      .process(cluster.getSolrClient());
+    cluster.getSolrClient().getZkStateReader().registerCore(collectionName); //TODO: Why is this needed? see SOLR-9440 
+    waitForState("Expected collection to be created with 1 shard and 2 replicas", collectionName, clusterShape(1, 2));
+    DocCollection docCollection = assertNumberOfReplicas(1, 0, 1, false, true);
+    
+    // Add a document and commit
+    cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "1", "foo", "bar"));
+    cluster.getSolrClient().commit(collectionName);
+    Slice s = docCollection.getSlices().iterator().next();
+    try (HttpSolrClient leaderClient = getHttpSolrClient(s.getLeader().getCoreUrl())) {
+      assertEquals(1, leaderClient.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+    
+    waitForNumDocsInAllReplicas(1, docCollection.getReplicas(EnumSet.of(Replica.Type.PULL)));
+    
+    // Delete leader replica from shard1
+    ignoreException("No registered leader was found"); //These are expected
+    JettySolrRunner leaderJetty = null;
+    if (removeReplica) {
+      CollectionAdminRequest.deleteReplica(
+          collectionName, 
+          "shard1", 
+          s.getLeader().getName())
+      .process(cluster.getSolrClient());
+    } else {
+      leaderJetty = cluster.getReplicaJetty(s.getLeader());
+      ChaosMonkey.kill(leaderJetty);
+      waitForState("Leader replica not removed", collectionName, clusterShape(1, 1));
+      // Wait for cluster state to be updated
+      waitForState("Replica state not updated in cluster state", 
+          collectionName, clusterStateReflectsActiveAndDownReplicas());
+    }
+    docCollection = assertNumberOfReplicas(0, 0, 1, true, true);
+    
+    // Check that there is no leader for the shard
+    Replica leader = docCollection.getSlice("shard1").getLeader();
+    assertTrue(leader == null || !leader.isActive(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes()));
+    
+    // Pull replica on the other hand should be active
+    Replica pullReplica = docCollection.getSlice("shard1").getReplicas(EnumSet.of(Replica.Type.PULL)).get(0);
+    assertTrue(pullReplica.isActive(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes()));
+
+    // add document, this should fail since there is no leader. Pull replica should not accept the update
+    expectThrows(SolrException.class, () -> 
+      cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "2", "foo", "zoo"))
+    );
+    
+    // Also fails if I send the update to the pull replica explicitly
+    try (HttpSolrClient pullReplicaClient = getHttpSolrClient(docCollection.getReplicas(EnumSet.of(Replica.Type.PULL)).get(0).getCoreUrl())) {
+      expectThrows(SolrException.class, () -> 
+        cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "2", "foo", "zoo"))
+      );
+    }
+    
+    // Queries should still work
+    waitForNumDocsInAllReplicas(1, docCollection.getReplicas(EnumSet.of(Replica.Type.PULL)));
+    // Add nrt replica back. Since there is no nrt now, new nrt will have no docs. There will be data loss, since the it will become the leader
+    // and pull replicas will replicate from it. Maybe we want to change this. Replicate from pull replicas is not a good idea, since they
+    // are by definition out of date.
+    if (removeReplica) {
+      CollectionAdminRequest.addReplicaToShard(collectionName, "shard1", Replica.Type.NRT).process(cluster.getSolrClient());
+    } else {
+      ChaosMonkey.start(leaderJetty);
+    }
+    waitForState("Expected collection to be 1x2", collectionName, clusterShape(1, 2));
+    unIgnoreException("No registered leader was found"); // Should have a leader from now on
+
+    // Validate that the new nrt replica is the leader now
+    cluster.getSolrClient().getZkStateReader().forceUpdateCollection(collectionName);
+    docCollection = getCollectionState(collectionName);
+    leader = docCollection.getSlice("shard1").getLeader();
+    assertTrue(leader != null && leader.isActive(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes()));
+
+    // If jetty is restarted, the replication is not forced, and replica doesn't replicate from leader until new docs are added. Is this the correct behavior? Why should these two cases be different?
+    if (removeReplica) {
+      // Pull replicas will replicate the empty index if a new replica was added and becomes leader
+      waitForNumDocsInAllReplicas(0, docCollection.getReplicas(EnumSet.of(Replica.Type.PULL)));
+    }
+    
+    // add docs agin
+    cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "2", "foo", "zoo"));
+    s = docCollection.getSlices().iterator().next();
+    try (HttpSolrClient leaderClient = getHttpSolrClient(s.getLeader().getCoreUrl())) {
+      leaderClient.commit();
+      assertEquals(1, leaderClient.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+    waitForNumDocsInAllReplicas(1, docCollection.getReplicas(EnumSet.of(Replica.Type.PULL)), "id:2");
+    waitForNumDocsInAllReplicas(1, docCollection.getReplicas(EnumSet.of(Replica.Type.PULL)));
+  }
+  
+  public void testKillPullReplica() throws Exception {
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1, 0, 1)
+      .setMaxShardsPerNode(100)
+      .process(cluster.getSolrClient());
+//    cluster.getSolrClient().getZkStateReader().registerCore(collectionName); //TODO: Is this needed? 
+    waitForState("Expected collection to be created with 1 shard and 2 replicas", collectionName, clusterShape(1, 2));
+    DocCollection docCollection = assertNumberOfReplicas(1, 0, 1, false, true);
+    assertEquals(1, docCollection.getSlices().size());
+    
+    waitForNumDocsInAllActiveReplicas(0);
+    cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "1", "foo", "bar"));
+    cluster.getSolrClient().commit(collectionName);
+    waitForNumDocsInAllActiveReplicas(1);
+    
+    JettySolrRunner pullReplicaJetty = cluster.getReplicaJetty(docCollection.getSlice("shard1").getReplicas(EnumSet.of(Replica.Type.PULL)).get(0));
+    ChaosMonkey.kill(pullReplicaJetty);
+    waitForState("Replica not removed", collectionName, activeReplicaCount(1, 0, 0));
+    // Also wait for the replica to be placed in state="down"
+    waitForState("Didn't update state", collectionName, clusterStateReflectsActiveAndDownReplicas());
+    
+    cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "2", "foo", "bar"));
+    cluster.getSolrClient().commit(collectionName);
+    waitForNumDocsInAllActiveReplicas(2);
+    
+    ChaosMonkey.start(pullReplicaJetty);
+    waitForState("Replica not added", collectionName, activeReplicaCount(1, 0, 1));
+    waitForNumDocsInAllActiveReplicas(2);
+  }
+  
+  public void testSearchWhileReplicationHappens() {
+      
+  }
+  
+  private void waitForNumDocsInAllActiveReplicas(int numDocs) throws IOException, SolrServerException, InterruptedException {
+    DocCollection docCollection = getCollectionState(collectionName);
+    waitForNumDocsInAllReplicas(numDocs, docCollection.getReplicas().stream().filter(r -> r.getState() == Replica.State.ACTIVE).collect(Collectors.toList()));
+  }
+    
+  private void waitForNumDocsInAllReplicas(int numDocs, Collection<Replica> replicas) throws IOException, SolrServerException, InterruptedException {
+    waitForNumDocsInAllReplicas(numDocs, replicas, "*:*");
+  }
+  
+  private void waitForNumDocsInAllReplicas(int numDocs, Collection<Replica> replicas, String query) throws IOException, SolrServerException, InterruptedException {
+    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+    for (Replica r:replicas) {
+      try (HttpSolrClient replicaClient = getHttpSolrClient(r.getCoreUrl())) {
+        while (true) {
+          try {
+            assertEquals("Replica " + r.getName() + " not up to date after " + REPLICATION_TIMEOUT_SECS + " seconds",
+                numDocs, replicaClient.query(new SolrQuery(query)).getResults().getNumFound());
+            break;
+          } catch (AssertionError e) {
+            if (t.hasTimedOut()) {
+              throw e;
+            } else {
+              Thread.sleep(100);
+            }
+          }
+        }
+      }
+    }
+  }
+  
+  private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
+      LOG.info("Collection not yet deleted");
+      try {
+        Thread.sleep(100);
+        if (t.hasTimedOut()) {
+          fail("Timed out waiting for collection " + collection + " to be deleted.");
+        }
+        cluster.getSolrClient().getZkStateReader().forceUpdateCollection(collection);
+      } catch(SolrException e) {
+        return;
+      }
+      
+    }
+  }
+  
+  private DocCollection assertNumberOfReplicas(int numNrtReplicas, int numTlogReplicas, int numPullReplicas, boolean updateCollection, boolean activeOnly) throws KeeperException, InterruptedException {
+    if (updateCollection) {
+      cluster.getSolrClient().getZkStateReader().forceUpdateCollection(collectionName);
+    }
+    DocCollection docCollection = getCollectionState(collectionName);
+    assertNotNull(docCollection);
+    assertEquals("Unexpected number of writer replicas: " + docCollection, numNrtReplicas, 
+        docCollection.getReplicas(EnumSet.of(Replica.Type.NRT)).stream().filter(r->!activeOnly || r.getState() == Replica.State.ACTIVE).count());
+    assertEquals("Unexpected number of pull replicas: " + docCollection, numPullReplicas, 
+        docCollection.getReplicas(EnumSet.of(Replica.Type.PULL)).stream().filter(r->!activeOnly || r.getState() == Replica.State.ACTIVE).count());
+    assertEquals("Unexpected number of active replicas: " + docCollection, numTlogReplicas, 
+        docCollection.getReplicas(EnumSet.of(Replica.Type.TLOG)).stream().filter(r->!activeOnly || r.getState() == Replica.State.ACTIVE).count());
+    return docCollection;
+  }
+  
+  /*
+   * passes only if all replicas are active or down, and the "liveNodes" reflect the same status
+   */
+  private CollectionStatePredicate clusterStateReflectsActiveAndDownReplicas() {
+    return (liveNodes, collectionState) -> {
+      for (Replica r:collectionState.getReplicas()) {
+        if (r.getState() != Replica.State.DOWN && r.getState() != Replica.State.ACTIVE) {
+          return false;
+        }
+        if (r.getState() == Replica.State.DOWN && liveNodes.contains(r.getNodeName())) {
+          return false;
+        }
+        if (r.getState() == Replica.State.ACTIVE && !liveNodes.contains(r.getNodeName())) {
+          return false;
+        }
+      }
+      return true;
+    };
+  }
+  
+  
+  private CollectionStatePredicate activeReplicaCount(int numNrtReplicas, int numTlogReplicas, int numPullReplicas) {
+    return (liveNodes, collectionState) -> {
+      int nrtFound = 0, tlogFound = 0, pullFound = 0;
+      if (collectionState == null)
+        return false;
+      for (Slice slice : collectionState) {
+        for (Replica replica : slice) {
+          if (replica.isActive(liveNodes))
+            switch (replica.getType()) {
+              case TLOG:
+                tlogFound++;
+                break;
+              case PULL:
+                pullFound++;
+                break;
+              case NRT:
+                nrtFound++;
+                break;
+              default:
+                throw new AssertionError("Unexpected replica type");
+            }
+        }
+      }
+      return numNrtReplicas == nrtFound && numTlogReplicas == tlogFound && numPullReplicas == pullFound;
+    };
+  }
+  
+  private void addDocs(int numDocs) throws SolrServerException, IOException {
+    List<SolrInputDocument> docs = new ArrayList<>(numDocs);
+    for (int i = 0; i < numDocs; i++) {
+      docs.add(new SolrInputDocument("id", String.valueOf(i), "fieldName_s", String.valueOf(i)));
+    }
+    cluster.getSolrClient().add(collectionName, docs);
+    cluster.getSolrClient().commit(collectionName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
new file mode 100644
index 0000000..6a22d99
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
@@ -0,0 +1,344 @@
+/*
+ * 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.URI;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
+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.CollectionAdminRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.CollectionStatePredicate;
+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.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.apache.zookeeper.KeeperException;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-5776")
+public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
+  
+  private final static int REPLICATION_TIMEOUT_SECS = 10;
+  
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static Map<URI, SocketProxy> proxies;
+  private static Map<URI, JettySolrRunner> jettys;
+
+  private String collectionName = null;
+  
+  private String suggestedCollectionName() {
+    return (getTestClass().getSimpleName().replace("Test", "") + "_" + getTestName().split(" ")[0]).replaceAll("(.)(\\p{Upper})", "$1_$2").toLowerCase(Locale.ROOT);
+  }
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    TestInjection.waitForReplicasInSync = null; // We'll be explicit about this in this test
+    configureCluster(4) 
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+    // Add proxies
+    proxies = new HashMap<>(cluster.getJettySolrRunners().size());
+    jettys = new HashMap<>(cluster.getJettySolrRunners().size());
+    for (JettySolrRunner jetty:cluster.getJettySolrRunners()) {
+      SocketProxy proxy = new SocketProxy();
+      jetty.setProxyPort(proxy.getListenPort());
+      cluster.stopJettySolrRunner(jetty);//TODO: Can we avoid this restart
+      cluster.startJettySolrRunner(jetty);
+      proxy.open(jetty.getBaseUrl().toURI());
+      LOG.info("Adding proxy for URL: " + jetty.getBaseUrl() + ". Proxy: " + proxy.getUrl());
+      proxies.put(proxy.getUrl(), proxy);
+      jettys.put(proxy.getUrl(), jetty);
+    }
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    while (true) {
+      try {
+        CollectionAdminRequest.ClusterProp clusterPropRequest = CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, "false");
+        CollectionAdminResponse response = clusterPropRequest.process(cluster.getSolrClient());
+        assertEquals(0, response.getStatus());
+        break;
+      } catch (SolrServerException e) {
+        Thread.sleep(50);
+        if (t.hasTimedOut()) {
+          throw e;
+        }
+      }
+    }
+  }
+  
+  @AfterClass
+  public static void tearDownCluster() throws Exception {
+    for (SocketProxy proxy:proxies.values()) {
+      proxy.close();
+    }
+    proxies = null;
+    jettys = null;
+    TestInjection.reset();
+  }
+  
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    collectionName = suggestedCollectionName();
+    expectThrows(SolrException.class, () -> getCollectionState(collectionName));
+    cluster.getSolrClient().setDefaultCollection(collectionName);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    if (cluster.getSolrClient().getZkStateReader().getClusterState().getCollectionOrNull(collectionName) != null) {
+      LOG.info("tearDown deleting collection");
+      CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
+      LOG.info("Collection deleted");
+      waitForDeletion(collectionName);
+    }
+    collectionName = null;
+    super.tearDown();
+  }
+  
+//  @Repeat(iterations=10)
+  public void testCantConnectToPullReplica() throws Exception {
+    int numShards = 2;
+    CollectionAdminRequest.createCollection(collectionName, "conf", numShards, 1, 0, 1)
+      .setMaxShardsPerNode(1)
+      .process(cluster.getSolrClient());
+    addDocs(10);
+    DocCollection docCollection = assertNumberOfReplicas(numShards, 0, numShards, false, true);
+    Slice s = docCollection.getSlices().iterator().next();
+    SocketProxy proxy = getProxyForReplica(s.getReplicas(EnumSet.of(Replica.Type.PULL)).get(0));
+    try {
+      proxy.close();
+      for (int i = 1; i <= 10; i ++) {
+        addDocs(10 + i);
+        try (HttpSolrClient leaderClient = getHttpSolrClient(s.getLeader().getCoreUrl())) {
+          assertNumDocs(10 + i, leaderClient);
+        }
+      }
+      try (HttpSolrClient pullReplicaClient = getHttpSolrClient(s.getReplicas(EnumSet.of(Replica.Type.PULL)).get(0).getCoreUrl())) {
+        pullReplicaClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+        fail("Shouldn't be able to query the pull replica");
+      } catch (SolrServerException e) {
+        //expected
+      }
+      assertNumberOfReplicas(numShards, 0, numShards, true, true);// Replica should still be active, since it doesn't disconnect from ZooKeeper
+      {
+        long numFound = 0;
+        TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+        while (numFound < 20 && !t.hasTimedOut()) {
+          Thread.sleep(200);
+          numFound = cluster.getSolrClient().query(collectionName, new SolrQuery("*:*")).getResults().getNumFound();
+        }
+      }
+    } finally {
+      proxy.reopen();
+    }
+    
+    try (HttpSolrClient pullReplicaClient = getHttpSolrClient(s.getReplicas(EnumSet.of(Replica.Type.PULL)).get(0).getCoreUrl())) {
+      assertNumDocs(20, pullReplicaClient);
+    }
+  }
+  
+  public void testCantConnectToLeader() throws Exception {
+    int numShards = 1;
+    CollectionAdminRequest.createCollection(collectionName, "conf", numShards, 1, 0, 1)
+      .setMaxShardsPerNode(1)
+      .process(cluster.getSolrClient());
+    addDocs(10);
+    DocCollection docCollection = assertNumberOfReplicas(numShards, 0, numShards, false, true);
+    Slice s = docCollection.getSlices().iterator().next();
+    SocketProxy proxy = getProxyForReplica(s.getLeader());
+    try {
+      // wait for replication
+      try (HttpSolrClient pullReplicaClient = getHttpSolrClient(s.getReplicas(EnumSet.of(Replica.Type.PULL)).get(0).getCoreUrl())) {
+        assertNumDocs(10, pullReplicaClient);
+      }
+      proxy.close();
+      expectThrows(SolrException.class, ()->addDocs(1));
+      try (HttpSolrClient pullReplicaClient = getHttpSolrClient(s.getReplicas(EnumSet.of(Replica.Type.PULL)).get(0).getCoreUrl())) {
+        assertNumDocs(10, pullReplicaClient);
+      }
+      assertNumDocs(10, cluster.getSolrClient());
+    } finally {
+      LOG.info("Opening leader node");
+      proxy.reopen();
+    }
+//     Back to normal
+//    Even if the leader is back to normal, the replica can get broken pipe for some time when trying to connect to it. The commit
+//    can fail if it's sent to the replica and it forwards it to the leader, and since it uses CUSC the error is hidden! That breaks
+//    the last part of this test.
+//    addDocs(20);
+//    assertNumDocs(20, cluster.getSolrClient(), 300);
+//    try (HttpSolrClient pullReplicaClient = getHttpSolrClient(s.getReplicas(EnumSet.of(Replica.Type.PULL)).get(0).getCoreUrl())) {
+//      assertNumDocs(20, pullReplicaClient);
+//    }
+  }
+  
+  public void testPullReplicaDisconnectsFromZooKeeper() throws Exception {
+    int numShards = 1;
+    CollectionAdminRequest.createCollection(collectionName, "conf", numShards, 1, 0, 1)
+      .setMaxShardsPerNode(1)
+      .process(cluster.getSolrClient());
+    addDocs(10);
+    DocCollection docCollection = assertNumberOfReplicas(numShards, 0, numShards, false, true);
+    Slice s = docCollection.getSlices().iterator().next();
+    try (HttpSolrClient pullReplicaClient = getHttpSolrClient(s.getReplicas(EnumSet.of(Replica.Type.PULL)).get(0).getCoreUrl())) {
+      assertNumDocs(10, pullReplicaClient);
+    }
+    addDocs(20);
+    JettySolrRunner jetty = getJettyForReplica(s.getReplicas(EnumSet.of(Replica.Type.PULL)).get(0));
+    cluster.expireZkSession(jetty);
+    addDocs(30);
+    waitForState("Expecting node to be disconnected", collectionName, activeReplicaCount(1, 0, 0));
+    addDocs(40);
+    waitForState("Expecting node to be disconnected", collectionName, activeReplicaCount(1, 0, 1));
+    try (HttpSolrClient pullReplicaClient = getHttpSolrClient(s.getReplicas(EnumSet.of(Replica.Type.PULL)).get(0).getCoreUrl())) {
+      assertNumDocs(40, pullReplicaClient);
+    }
+  }
+  
+  private void assertNumDocs(int numDocs, SolrClient client, int timeoutSecs) throws InterruptedException, SolrServerException, IOException {
+    TimeOut t = new TimeOut(timeoutSecs, TimeUnit.SECONDS);
+    long numFound = -1;
+    while (!t.hasTimedOut()) {
+      Thread.sleep(200);
+      numFound = client.query(new SolrQuery("*:*")).getResults().getNumFound();
+      if (numFound == numDocs) {
+        return;
+      }
+    }
+    fail("Didn't get expected doc count. Expected: " + numDocs + ", Found: " + numFound);
+  }
+  
+  
+  private void assertNumDocs(int numDocs, SolrClient client) throws InterruptedException, SolrServerException, IOException {
+    assertNumDocs(numDocs, client, REPLICATION_TIMEOUT_SECS);
+  }
+
+  private void addDocs(int numDocs) throws SolrServerException, IOException {
+    List<SolrInputDocument> docs = new ArrayList<>(numDocs);
+    for (int i = 0; i < numDocs; i++) {
+      docs.add(new SolrInputDocument("id", String.valueOf(i), "fieldName_s", String.valueOf(i)));
+    }
+    cluster.getSolrClient().add(collectionName, docs);
+    cluster.getSolrClient().commit(collectionName);
+  }
+  
+  private DocCollection assertNumberOfReplicas(int numWriter, int numActive, int numPassive, boolean updateCollection, boolean activeOnly) throws KeeperException, InterruptedException {
+    if (updateCollection) {
+      cluster.getSolrClient().getZkStateReader().forceUpdateCollection(collectionName);
+    }
+    DocCollection docCollection = getCollectionState(collectionName);
+    assertNotNull(docCollection);
+    assertEquals("Unexpected number of writer replicas: " + docCollection, numWriter, 
+        docCollection.getReplicas(EnumSet.of(Replica.Type.NRT)).stream().filter(r->!activeOnly || r.getState() == Replica.State.ACTIVE).count());
+    assertEquals("Unexpected number of pull replicas: " + docCollection, numPassive, 
+        docCollection.getReplicas(EnumSet.of(Replica.Type.PULL)).stream().filter(r->!activeOnly || r.getState() == Replica.State.ACTIVE).count());
+    assertEquals("Unexpected number of active replicas: " + docCollection, numActive, 
+        docCollection.getReplicas(EnumSet.of(Replica.Type.TLOG)).stream().filter(r->!activeOnly || r.getState() == Replica.State.ACTIVE).count());
+    return docCollection;
+  }
+  
+  protected JettySolrRunner getJettyForReplica(Replica replica) throws Exception {
+    String replicaBaseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
+    assertNotNull(replicaBaseUrl);
+    URL baseUrl = new URL(replicaBaseUrl);
+
+    JettySolrRunner proxy = jettys.get(baseUrl.toURI());
+    assertNotNull("No proxy found for " + baseUrl + "!", proxy);
+    return proxy;
+  }  
+  
+  protected SocketProxy getProxyForReplica(Replica replica) throws Exception {
+    String replicaBaseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
+    assertNotNull(replicaBaseUrl);
+    URL baseUrl = new URL(replicaBaseUrl);
+
+    SocketProxy proxy = proxies.get(baseUrl.toURI());
+    if (proxy == null && !baseUrl.toExternalForm().endsWith("/")) {
+      baseUrl = new URL(baseUrl.toExternalForm() + "/");
+      proxy = proxies.get(baseUrl.toURI());
+    }
+    assertNotNull("No proxy found for " + baseUrl + "!", proxy);
+    return proxy;
+  }
+  
+  private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
+      LOG.info("Collection not yet deleted");
+      try {
+        Thread.sleep(100);
+        if (t.hasTimedOut()) {
+          fail("Timed out waiting for collection " + collection + " to be deleted.");
+        }
+        cluster.getSolrClient().getZkStateReader().forceUpdateCollection(collection);
+      } catch(SolrException e) {
+        return;
+      }
+      
+    }
+  }
+  
+  private CollectionStatePredicate activeReplicaCount(int numWriter, int numActive, int numPassive) {
+    return (liveNodes, collectionState) -> {
+      int writersFound = 0, activesFound = 0, passivesFound = 0;
+      if (collectionState == null)
+        return false;
+      for (Slice slice : collectionState) {
+        for (Replica replica : slice) {
+          if (replica.isActive(liveNodes))
+            switch (replica.getType()) {
+              case TLOG:
+                activesFound++;
+                break;
+              case PULL:
+                passivesFound++;
+                break;
+              case NRT:
+                writersFound++;
+                break;
+              default:
+                throw new AssertionError("Unexpected replica type");
+            }
+        }
+      }
+      return numWriter == writersFound && numActive == activesFound && numPassive == passivesFound;
+    };
+  }
+
+}


[6/6] lucene-solr:jira/solr-8668: Merge commit '2fc41d565a4a0408a09856a37d3be7d87414ba3f' into jira/solr-8668

Posted by cp...@apache.org.
Merge commit '2fc41d565a4a0408a09856a37d3be7d87414ba3f' into jira/solr-8668

Resolved Conflicts:
	solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/ab9aad20
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/ab9aad20
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/ab9aad20

Branch: refs/heads/jira/solr-8668
Commit: ab9aad20ab499ddc3977c0c012046a369b16f8b9
Parents: e4cbe0e 2fc41d5
Author: Christine Poerschke <cp...@apache.org>
Authored: Thu May 25 18:39:11 2017 +0100
Committer: Christine Poerschke <cp...@apache.org>
Committed: Thu May 25 18:39:11 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  14 +
 .../org/apache/solr/cloud/AddReplicaCmd.java    |   9 +-
 .../src/java/org/apache/solr/cloud/Assign.java  |  12 +-
 .../org/apache/solr/cloud/CloudDescriptor.java  |  25 +-
 .../apache/solr/cloud/CreateCollectionCmd.java  |  40 +-
 .../org/apache/solr/cloud/CreateShardCmd.java   |  48 +-
 .../org/apache/solr/cloud/ElectionContext.java  |  13 +-
 .../java/org/apache/solr/cloud/MigrateCmd.java  |   8 +-
 .../org/apache/solr/cloud/MoveReplicaCmd.java   |   4 +-
 .../cloud/OverseerCollectionMessageHandler.java |  91 +-
 .../org/apache/solr/cloud/RecoveryStrategy.java | 166 +++-
 .../org/apache/solr/cloud/ReplaceNodeCmd.java   |   4 +-
 .../apache/solr/cloud/ReplicateFromLeader.java  |  42 +-
 .../java/org/apache/solr/cloud/RestoreCmd.java  |  71 +-
 .../org/apache/solr/cloud/SplitShardCmd.java    |   4 +-
 .../org/apache/solr/cloud/ZkController.java     |  67 +-
 .../solr/cloud/overseer/ReplicaMutator.java     |   7 +-
 .../solr/cloud/overseer/SliceMutator.java       |  24 +-
 .../solr/cloud/overseer/ZkStateWriter.java      |   1 +
 .../apache/solr/cloud/rule/ReplicaAssigner.java |   6 +-
 .../org/apache/solr/core/CoreContainer.java     |  38 +-
 .../org/apache/solr/handler/IndexFetcher.java   |  23 +-
 .../apache/solr/handler/RealTimeGetHandler.java |  20 +-
 .../apache/solr/handler/ReplicationHandler.java |   4 +
 .../solr/handler/admin/CollectionsHandler.java  |  17 +-
 .../solr/handler/admin/CoreAdminHandler.java    |   1 +
 .../solr/handler/admin/PrepRecoveryOp.java      |  30 +-
 .../handler/component/HttpShardHandler.java     |  37 +-
 .../handler/component/RealTimeGetComponent.java |  27 +-
 .../solr/update/DirectUpdateHandler2.java       |  20 +-
 .../org/apache/solr/update/UpdateHandler.java   |   6 +-
 .../java/org/apache/solr/update/UpdateLog.java  |   7 +-
 .../processor/DistributedUpdateProcessor.java   |  51 +-
 .../org/apache/solr/util/TestInjection.java     |   6 +-
 .../solr/collection1/conf/solrconfig.xml        |   8 +-
 .../cloud-minimal/conf/solrconfig.xml           |   2 +-
 .../AbstractCloudBackupRestoreTestCase.java     |  29 +-
 .../test/org/apache/solr/cloud/AssignTest.java  |   6 +
 .../solr/cloud/BasicDistributedZk2Test.java     |   4 +-
 .../solr/cloud/BasicDistributedZkTest.java      |  10 +-
 .../cloud/ChaosMonkeyNothingIsSafeTest.java     | 136 +--
 ...MonkeyNothingIsSafeWithPullReplicasTest.java | 327 +++++++
 ...aosMonkeySafeLeaderWithPullReplicasTest.java | 254 ++++++
 .../cloud/CollectionsAPIDistributedZkTest.java  |   4 +-
 .../solr/cloud/CollectionsAPISolrJTest.java     |  18 +-
 .../org/apache/solr/cloud/DeleteNodeTest.java   |  11 +-
 .../apache/solr/cloud/DeleteReplicaTest.java    |  22 +-
 .../org/apache/solr/cloud/ForceLeaderTest.java  |   4 +-
 .../FullThrottleStoppableIndexingThread.java    | 156 ++++
 .../apache/solr/cloud/HttpPartitionTest.java    |   8 +-
 .../LeaderInitiatedRecoveryOnCommitTest.java    |   8 +-
 .../solr/cloud/OnlyLeaderIndexesTest.java       | 435 ----------
 ...verseerCollectionConfigSetProcessorTest.java |   3 +-
 .../solr/cloud/RecoveryAfterSoftCommitTest.java |   8 +-
 .../org/apache/solr/cloud/ReplaceNodeTest.java  |  21 +-
 .../solr/cloud/ReplicationFactorTest.java       |   4 +-
 .../org/apache/solr/cloud/ShardSplitTest.java   |   6 -
 .../apache/solr/cloud/TestCloudRecovery.java    |   5 +-
 .../apache/solr/cloud/TestCollectionAPI.java    |  10 +-
 .../org/apache/solr/cloud/TestPullReplica.java  | 576 +++++++++++++
 .../cloud/TestPullReplicaErrorHandling.java     | 344 ++++++++
 .../org/apache/solr/cloud/TestTlogReplica.java  | 845 +++++++++++++++++++
 .../cloud/hdfs/HdfsBasicDistributedZkTest.java  |   4 +-
 .../reporters/solr/SolrCloudReportersTest.java  |   2 +-
 .../solr/update/TestInPlaceUpdatesDistrib.java  |  10 +-
 .../solrj/impl/ConcurrentUpdateSolrClient.java  |  23 +-
 .../solrj/request/CollectionAdminRequest.java   | 121 ++-
 .../apache/solr/common/cloud/DocCollection.java |  56 +-
 .../org/apache/solr/common/cloud/Replica.java   |  32 +
 .../org/apache/solr/common/cloud/Slice.java     |  21 +-
 .../apache/solr/common/cloud/ZkStateReader.java |  16 +-
 .../solr/common/params/CoreAdminParams.java     |   5 +
 ...ollectionAdminRequestRequiredParamsTest.java |   9 +
 .../solr/BaseDistributedSearchTestCase.java     |   3 +-
 .../java/org/apache/solr/SolrTestCaseJ4.java    |  31 +-
 .../cloud/AbstractFullDistribZkTestBase.java    | 264 +++++-
 .../java/org/apache/solr/cloud/ChaosMonkey.java | 159 +++-
 .../solr/cloud/StoppableCommitThread.java       |  69 ++
 .../solr/cloud/StoppableIndexingThread.java     |   2 +-
 .../solr/cloud/StoppableSearchThread.java       |   2 +-
 80 files changed, 4026 insertions(+), 1010 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ab9aad20/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ab9aad20/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
diff --cc solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index 629666e,a55e2ea..f41f9cf
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@@ -2435,6 -2450,37 +2442,10 @@@ public abstract class SolrTestCaseJ4 ex
      newestSearcher.decref();
    }
  
+   protected void waitForWarming() throws InterruptedException {
+     waitForWarming(h.getCore());
+   }
+ 
 -  @BeforeClass
 -  public static void chooseMPForMP() throws Exception {
 -    if (random().nextBoolean()) {
 -      System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "true");
 -      System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "false");
 -    } else {
 -      System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "false");
 -      System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "true");
 -    }
 -  }
 -
 -  @AfterClass
 -  public static void unchooseMPForMP() {
 -    System.clearProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY);
 -    System.clearProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY);
 -  }
 -
 -  @Deprecated // remove with SOLR-8668
 -  protected static void systemSetPropertySolrTestsMergePolicy(String value) {
 -    System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICY, value);
 -  }
 -
 -  @Deprecated // remove with SOLR-8668
 -  protected static void systemClearPropertySolrTestsMergePolicy() {
 -    System.clearProperty(SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICY);
 -  }
 -
    protected static void systemSetPropertySolrTestsMergePolicyFactory(String value) {
      System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY, value);
    }


[2/6] lucene-solr:jira/solr-8668: SOLR-10233: Add support for replica types

Posted by cp...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
new file mode 100644
index 0000000..034a8bf
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
@@ -0,0 +1,845 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import org.apache.http.client.HttpClient;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.util.LuceneTestCase.Slow;
+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.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+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.CollectionStatePredicate;
+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.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.update.DirectUpdateHandler2;
+import org.apache.solr.update.SolrIndexWriter;
+import org.apache.solr.update.UpdateHandler;
+import org.apache.solr.update.UpdateLog;
+import org.apache.solr.util.RefCounted;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.apache.zookeeper.KeeperException;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.carrotsearch.randomizedtesting.annotations.Repeat;
+
+@Slow
+public class TestTlogReplica extends SolrCloudTestCase {
+  
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  
+  private String collectionName = null;
+  private final static int REPLICATION_TIMEOUT_SECS = 10;
+  
+  private String suggestedCollectionName() {
+    return (getTestClass().getSimpleName().replace("Test", "") + "_" + getTestName().split(" ")[0]).replaceAll("(.)(\\p{Upper})", "$1_$2").toLowerCase(Locale.ROOT);
+  }
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    TestInjection.waitForReplicasInSync = null; // We'll be explicit about this in this test
+    configureCluster(2) // 2 + random().nextInt(3) 
+        .addConfig("conf", configset("cloud-minimal-inplace-updates"))
+        .configure();
+    Boolean useLegacyCloud = rarely();
+    LOG.info("Using legacyCloud?: {}", useLegacyCloud);
+    CollectionAdminRequest.ClusterProp clusterPropRequest = CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, String.valueOf(useLegacyCloud));
+    CollectionAdminResponse response = clusterPropRequest.process(cluster.getSolrClient());
+    assertEquals(0, response.getStatus());
+  }
+  
+  @AfterClass
+  public static void tearDownCluster() {
+    TestInjection.reset();
+  }
+  
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    collectionName = suggestedCollectionName();
+    expectThrows(SolrException.class, () -> getCollectionState(collectionName));
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    for (JettySolrRunner jetty:cluster.getJettySolrRunners()) {
+      if (!jetty.isRunning()) {
+        LOG.warn("Jetty {} not running, probably some bad test. Starting it", jetty.getLocalPort());
+        ChaosMonkey.start(jetty);
+      }
+    }
+    if (cluster.getSolrClient().getZkStateReader().getClusterState().getCollectionOrNull(collectionName) != null) {
+      LOG.info("tearDown deleting collection");
+      CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
+      waitForDeletion(collectionName);
+    }
+    super.tearDown();
+  }
+  
+  /**
+   * Asserts that Update logs exist for replicas of type {@link org.apache.solr.common.cloud.Replica.Type#NRT}, but not
+   * for replicas of type {@link org.apache.solr.common.cloud.Replica.Type#PULL}
+   */
+  private void assertUlogPresence(DocCollection collection) {
+    for (Slice s:collection.getSlices()) {
+      for (Replica r:s.getReplicas()) {
+        SolrCore core = null;
+        try {
+          core = cluster.getReplicaJetty(r).getCoreContainer().getCore(r.getCoreName());
+          assertNotNull(core);
+          assertTrue("Update log should exist for replicas of type Append", 
+              new java.io.File(core.getUlogDir()).exists());
+        } finally {
+          core.close();
+        }
+      }
+    }
+  }
+  
+  @Repeat(iterations=2) // 2 times to make sure cleanup is complete and we can create the same collection
+  public void testCreateDelete() throws Exception {
+    try {
+      CollectionAdminRequest.createCollection(collectionName, "conf", 2, 0, 4, 0)
+      .setMaxShardsPerNode(100)
+      .process(cluster.getSolrClient());
+      boolean reloaded = false;
+      while (true) {
+        DocCollection docCollection = getCollectionState(collectionName);
+        assertNotNull(docCollection);
+        assertEquals("Expecting 2 shards",
+            2, docCollection.getSlices().size());
+        assertEquals("Expecting 4 relpicas per shard",
+            8, docCollection.getReplicas().size());
+        assertEquals("Expecting 8 tlog replicas, 4 per shard",
+            8, docCollection.getReplicas(EnumSet.of(Replica.Type.TLOG)).size());
+        assertEquals("Expecting no nrt replicas",
+            0, docCollection.getReplicas(EnumSet.of(Replica.Type.NRT)).size());
+        assertEquals("Expecting no pull replicas",
+            0, docCollection.getReplicas(EnumSet.of(Replica.Type.PULL)).size());
+        for (Slice s:docCollection.getSlices()) {
+          assertTrue(s.getLeader().getType() == Replica.Type.TLOG);
+          List<String> shardElectionNodes = cluster.getZkClient().getChildren(ZkStateReader.getShardLeadersElectPath(collectionName, s.getName()), null, true);
+          assertEquals("Unexpected election nodes for Shard: " + s.getName() + ": " + Arrays.toString(shardElectionNodes.toArray()), 
+              4, shardElectionNodes.size());
+        }
+        assertUlogPresence(docCollection);
+        if (reloaded) {
+          break;
+        } else {
+          // reload
+          CollectionAdminResponse response = CollectionAdminRequest.reloadCollection(collectionName)
+          .process(cluster.getSolrClient());
+          assertEquals(0, response.getStatus());
+          reloaded = true;
+        }
+      }
+    } finally {
+      zkClient().printLayoutToStdOut();
+    }
+  }
+  
+  @SuppressWarnings("unchecked")
+  public void testAddDocs() throws Exception {
+    int numTlogReplicas = 1 + random().nextInt(3);
+    DocCollection docCollection = createAndWaitForCollection(1, 0, numTlogReplicas, 0);
+    assertEquals(1, docCollection.getSlices().size());
+    
+    cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "1", "foo", "bar"));
+    cluster.getSolrClient().commit(collectionName);
+    
+    Slice s = docCollection.getSlices().iterator().next();
+    try (HttpSolrClient leaderClient = getHttpSolrClient(s.getLeader().getCoreUrl())) {
+      assertEquals(1, leaderClient.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+    
+    TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
+    for (Replica r:s.getReplicas(EnumSet.of(Replica.Type.TLOG))) {
+      //TODO: assert replication < REPLICATION_TIMEOUT_SECS
+      try (HttpSolrClient tlogReplicaClient = getHttpSolrClient(r.getCoreUrl())) {
+        while (true) {
+          try {
+            assertEquals("Replica " + r.getName() + " not up to date after 10 seconds",
+                1, tlogReplicaClient.query(new SolrQuery("*:*")).getResults().getNumFound());
+            // Append replicas process all updates
+            SolrQuery req = new SolrQuery(
+                "qt", "/admin/plugins",
+                "stats", "true");
+            QueryResponse statsResponse = tlogReplicaClient.query(req);
+            assertEquals("Append replicas should recive all updates. Replica: " + r + ", response: " + statsResponse, 
+                1L, ((Map<String, Object>)((NamedList<Object>)statsResponse.getResponse()).findRecursive("plugins", "UPDATE", "updateHandler", "stats")).get("UPDATE.updateHandler.cumulativeAdds.count"));
+            break;
+          } catch (AssertionError e) {
+            if (t.hasTimedOut()) {
+              throw e;
+            } else {
+              Thread.sleep(100);
+            }
+          }
+        }
+      }
+    }
+    assertUlogPresence(docCollection);
+  }
+  
+  public void testAddRemoveTlogReplica() throws Exception {
+    DocCollection docCollection = createAndWaitForCollection(2, 0, 1, 0);
+    assertEquals(2, docCollection.getSlices().size());
+    
+    CollectionAdminRequest.addReplicaToShard(collectionName, "shard1", Replica.Type.TLOG).process(cluster.getSolrClient());
+    docCollection = assertNumberOfReplicas(0, 3, 0, true, false);
+    CollectionAdminRequest.addReplicaToShard(collectionName, "shard2", Replica.Type.TLOG).process(cluster.getSolrClient());    
+    docCollection = assertNumberOfReplicas(0, 4, 0, true, false);
+    
+    waitForState("Expecting collection to have 2 shards and 2 replica each", collectionName, clusterShape(2, 2));
+    
+    //Delete tlog replica from shard1
+    CollectionAdminRequest.deleteReplica(
+        collectionName, 
+        "shard1", 
+        docCollection.getSlice("shard1").getReplicas(EnumSet.of(Replica.Type.TLOG)).get(0).getName())
+    .process(cluster.getSolrClient());
+    assertNumberOfReplicas(0, 3, 0, true, true);
+  }
+  
+  public void testRemoveLeader() throws Exception {
+    doReplaceLeader(true);
+  }
+  
+  public void testKillLeader() throws Exception {
+    doReplaceLeader(false);
+  }
+  
+  public void testRealTimeGet() throws SolrServerException, IOException, KeeperException, InterruptedException {
+    // should be redirected to Replica.Type.REALTIME
+    int numReplicas = random().nextBoolean()?1:2;
+    int numNrtReplicas = random().nextBoolean()?0:2;
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, numNrtReplicas, numReplicas, 0)
+      .setMaxShardsPerNode(100)
+      .process(cluster.getSolrClient());
+    waitForState("Unexpected replica count", collectionName, activeReplicaCount(numNrtReplicas, numReplicas, 0));
+    DocCollection docCollection = assertNumberOfReplicas(numNrtReplicas, numReplicas, 0, false, true);
+    HttpClient httpClient = cluster.getSolrClient().getHttpClient();
+    int id = 0;
+    Slice slice = docCollection.getSlice("shard1");
+    List<String> ids = new ArrayList<>(slice.getReplicas().size());
+    for (Replica rAdd:slice.getReplicas()) {
+      try (HttpSolrClient client = getHttpSolrClient(rAdd.getCoreUrl(), httpClient)) {
+        client.add(new SolrInputDocument("id", String.valueOf(id), "foo_s", "bar"));
+      }
+      SolrDocument docCloudClient = cluster.getSolrClient().getById(collectionName, String.valueOf(id));
+      assertEquals("bar", docCloudClient.getFieldValue("foo_s"));
+      for (Replica rGet:slice.getReplicas()) {
+        try (HttpSolrClient client = getHttpSolrClient(rGet.getCoreUrl(), httpClient)) {
+          SolrDocument doc = client.getById(String.valueOf(id));
+          assertEquals("bar", doc.getFieldValue("foo_s"));
+        }
+      }
+      ids.add(String.valueOf(id));
+      id++;
+    }
+    SolrDocumentList previousAllIdsResult = null;
+    for (Replica rAdd:slice.getReplicas()) {
+      try (HttpSolrClient client = getHttpSolrClient(rAdd.getCoreUrl(), httpClient)) {
+        SolrDocumentList allIdsResult = client.getById(ids);
+        if (previousAllIdsResult != null) {
+          assertTrue(compareSolrDocumentList(previousAllIdsResult, allIdsResult));
+        } else {
+          // set the first response here
+          previousAllIdsResult = allIdsResult;
+          assertEquals("Unexpected number of documents", ids.size(), allIdsResult.getNumFound());
+        }
+      }
+      id++;
+    }
+  }
+  
+  /*
+   * validate leader election and that replication still happens on a new leader
+   */
+  private void doReplaceLeader(boolean removeReplica) throws Exception {
+    DocCollection docCollection = createAndWaitForCollection(1, 0, 2, 0);
+    
+    // Add a document and commit
+    cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "1", "foo", "bar"));
+    cluster.getSolrClient().commit(collectionName);
+    Slice s = docCollection.getSlices().iterator().next();
+    try (HttpSolrClient leaderClient = getHttpSolrClient(s.getLeader().getCoreUrl())) {
+      assertEquals(1, leaderClient.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+    
+    waitForNumDocsInAllReplicas(1, docCollection.getReplicas(EnumSet.of(Replica.Type.TLOG)), REPLICATION_TIMEOUT_SECS);
+    
+    // Delete leader replica from shard1
+    JettySolrRunner leaderJetty = null;
+    if (removeReplica) {
+      CollectionAdminRequest.deleteReplica(
+          collectionName, 
+          "shard1", 
+          s.getLeader().getName())
+      .process(cluster.getSolrClient());
+    } else {
+      leaderJetty = cluster.getReplicaJetty(s.getLeader());
+      ChaosMonkey.kill(leaderJetty);
+      waitForState("Leader replica not removed", collectionName, clusterShape(1, 1));
+      // Wait for cluster state to be updated
+      waitForState("Replica state not updated in cluster state", 
+          collectionName, clusterStateReflectsActiveAndDownReplicas());
+    }
+    docCollection = assertNumberOfReplicas(0, 1, 0, true, true);
+    
+    // Wait until a new leader is elected
+    TimeOut t = new TimeOut(30, TimeUnit.SECONDS);
+    while (!t.hasTimedOut()) {
+      docCollection = getCollectionState(collectionName);
+      Replica leader = docCollection.getSlice("shard1").getLeader();
+      if (leader != null && leader.isActive(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes())) {
+        break;
+      }
+      Thread.sleep(500);
+    }
+    assertFalse("Timeout waiting for a new leader to be elected", t.hasTimedOut());
+    
+    // There is a new leader, I should be able to add and commit
+    cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "2", "foo", "zoo"));
+    cluster.getSolrClient().commit(collectionName);
+    
+    // Queries should still work
+    waitForNumDocsInAllReplicas(2, docCollection.getReplicas(EnumSet.of(Replica.Type.TLOG)), REPLICATION_TIMEOUT_SECS);
+    // Start back the node
+    if (removeReplica) {
+      CollectionAdminRequest.addReplicaToShard(collectionName, "shard1", Replica.Type.TLOG).process(cluster.getSolrClient());
+    } else {
+      ChaosMonkey.start(leaderJetty);
+    }
+    waitForState("Expected collection to be 1x2", collectionName, clusterShape(1, 2));
+    // added replica should replicate from the leader
+    waitForNumDocsInAllReplicas(2, docCollection.getReplicas(EnumSet.of(Replica.Type.TLOG)), REPLICATION_TIMEOUT_SECS);
+  }
+  
+  public void testKillTlogReplica() throws Exception {
+    DocCollection docCollection = createAndWaitForCollection(1, 0, 2, 0);
+    
+    waitForNumDocsInAllActiveReplicas(0);
+    cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "1", "foo", "bar"));
+    cluster.getSolrClient().commit(collectionName);
+    waitForNumDocsInAllActiveReplicas(1);
+    
+    JettySolrRunner pullReplicaJetty = cluster.getReplicaJetty(docCollection.getSlice("shard1").getReplicas(EnumSet.of(Replica.Type.TLOG)).get(0));
+    ChaosMonkey.kill(pullReplicaJetty);
+    waitForState("Replica not removed", collectionName, activeReplicaCount(0, 1, 0));
+//    // Also wait for the replica to be placed in state="down"
+//    waitForState("Didn't update state", collectionName, clusterStateReflectsActiveAndDownReplicas());
+    
+    cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "2", "foo", "bar"));
+    cluster.getSolrClient().commit(collectionName);
+    waitForNumDocsInAllActiveReplicas(2);
+    
+    ChaosMonkey.start(pullReplicaJetty);
+    waitForState("Replica not added", collectionName, activeReplicaCount(0, 2, 0));
+    waitForNumDocsInAllActiveReplicas(2);
+  }
+  
+  public void testOnlyLeaderIndexes() throws Exception {
+    createAndWaitForCollection(1, 0, 2, 0);
+    
+    CloudSolrClient cloudClient = cluster.getSolrClient();
+    new UpdateRequest()
+        .add(sdoc("id", "1"))
+        .add(sdoc("id", "2"))
+        .add(sdoc("id", "3"))
+        .add(sdoc("id", "4"))
+        .process(cloudClient, collectionName);
+
+    {
+      UpdateHandler updateHandler = getSolrCore(true).get(0).getUpdateHandler();
+      RefCounted<IndexWriter> iwRef = updateHandler.getSolrCoreState().getIndexWriter(null);
+      assertTrue("IndexWriter at leader must see updates ", iwRef.get().hasUncommittedChanges());
+      iwRef.decref();
+    }
+
+    for (SolrCore solrCore : getSolrCore(false)) {
+      RefCounted<IndexWriter> iwRef = solrCore.getUpdateHandler().getSolrCoreState().getIndexWriter(null);
+      assertFalse("IndexWriter at replicas must not see updates ", iwRef.get().hasUncommittedChanges());
+      iwRef.decref();
+    }
+
+    checkRTG(1, 4, cluster.getJettySolrRunners());
+
+    new UpdateRequest()
+        .deleteById("1")
+        .deleteByQuery("id:2")
+        .process(cloudClient, collectionName);
+
+    // The DBQ is not processed at replicas, so we still can get doc2 and other docs by RTG
+    checkRTG(2,4, getSolrRunner(false));
+
+    new UpdateRequest()
+        .commit(cloudClient, collectionName);
+
+    waitForNumDocsInAllActiveReplicas(2);
+
+    // Update log roll over
+    for (SolrCore solrCore : getSolrCore(false)) {
+      UpdateLog updateLog = solrCore.getUpdateHandler().getUpdateLog();
+      assertFalse(updateLog.hasUncommittedChanges());
+    }
+
+    // UpdateLog copy over old updates
+    for (int i = 15; i <= 150; i++) {
+      cloudClient.add(collectionName, sdoc("id",String.valueOf(i)));
+      if (random().nextInt(100) < 15 & i != 150) {
+        cloudClient.commit(collectionName);
+      }
+    }
+    checkRTG(120,150, cluster.getJettySolrRunners());
+    waitForReplicasCatchUp(20);
+  }
+  
+  @SuppressWarnings("unchecked")
+  public void testRecovery() throws Exception {
+    boolean useKill = random().nextBoolean();
+    createAndWaitForCollection(1, 0, 2, 0);
+    
+    CloudSolrClient cloudClient = cluster.getSolrClient();
+    new UpdateRequest()
+        .add(sdoc("id", "3"))
+        .add(sdoc("id", "4"))
+        .commit(cloudClient, collectionName);
+    new UpdateRequest()
+        .add(sdoc("id", "5"))
+        .process(cloudClient, collectionName);
+    JettySolrRunner solrRunner = getSolrRunner(false).get(0);
+    if (useKill) { 
+      ChaosMonkey.kill(solrRunner);
+    } else {
+      ChaosMonkey.stop(solrRunner);
+    }
+    waitForState("Replica still up", collectionName, activeReplicaCount(0,1,0));
+    new UpdateRequest()
+        .add(sdoc("id", "6"))
+        .process(cloudClient, collectionName);
+    ChaosMonkey.start(solrRunner);
+    waitForState("Replica didn't recover", collectionName, activeReplicaCount(0,2,0));
+    // We skip peerSync, so replica will always trigger commit on leader
+    // We query only the non-leader replicas, since we haven't opened a new searcher on the leader yet
+    waitForNumDocsInAllReplicas(4, getNonLeaderReplias(collectionName), 0);// Should be immediate
+    
+    // If I add the doc immediately, the leader fails to communicate with the follower with broken pipe.
+    // Options are, wait or retry...
+    for (int i = 0; i < 3; i++) {
+      UpdateRequest ureq = new UpdateRequest().add(sdoc("id", "7"));
+      ureq.setParam("collection", collectionName);
+      ureq.setParam(UpdateRequest.MIN_REPFACT, "2");
+      NamedList<Object> response = cloudClient.request(ureq);
+      if ((Integer)((NamedList<Object>)response.get("responseHeader")).get(UpdateRequest.REPFACT) >= 2) {
+        break;
+      }
+      LOG.info("Min RF not achieved yet. retrying");
+    }
+    checkRTG(3,7, cluster.getJettySolrRunners());
+    DirectUpdateHandler2.commitOnClose = false;
+    ChaosMonkey.stop(solrRunner);
+    waitForState("Replica still up", collectionName, activeReplicaCount(0,1,0));
+    DirectUpdateHandler2.commitOnClose = true;
+    ChaosMonkey.start(solrRunner);
+    waitForState("Replica didn't recover", collectionName, activeReplicaCount(0,2,0));
+    checkRTG(3,7, cluster.getJettySolrRunners());
+    waitForNumDocsInAllReplicas(5, getNonLeaderReplias(collectionName), 0);// Should be immediate
+    cluster.getSolrClient().commit(collectionName);
+
+    // Test replica recovery apply buffer updates
+    Semaphore waitingForBufferUpdates = new Semaphore(0);
+    Semaphore waitingForReplay = new Semaphore(0);
+    RecoveryStrategy.testing_beforeReplayBufferingUpdates = () -> {
+      try {
+        waitingForReplay.release();
+        waitingForBufferUpdates.acquire();
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+        fail("Test interrupted: " + e.getMessage());
+      }
+    };
+    if (useKill) { 
+      ChaosMonkey.kill(solrRunner);
+    } else {
+      ChaosMonkey.stop(solrRunner);
+    }
+    ChaosMonkey.start(solrRunner);
+    waitingForReplay.acquire();
+    new UpdateRequest()
+        .add(sdoc("id", "8"))
+        .add(sdoc("id", "9"))
+        .process(cloudClient, collectionName);
+    waitingForBufferUpdates.release();
+    RecoveryStrategy.testing_beforeReplayBufferingUpdates = null;
+    waitForState("Replica didn't recover", collectionName, activeReplicaCount(0,2,0));
+    checkRTG(3,9, cluster.getJettySolrRunners());
+    for (SolrCore solrCore : getSolrCore(false)) {
+      RefCounted<IndexWriter> iwRef = solrCore.getUpdateHandler().getSolrCoreState().getIndexWriter(null);
+      assertFalse("IndexWriter at replicas must not see updates ", iwRef.get().hasUncommittedChanges());
+      iwRef.decref();
+    }
+  }
+  
+  private List<Replica> getNonLeaderReplias(String collectionName) {
+    return getCollectionState(collectionName).getReplicas().stream().filter(
+        (r)-> !r.getBool("leader", false)).collect(Collectors.toList());
+  }
+  
+  public void testDeleteById() throws Exception{
+    createAndWaitForCollection(1,0,2,0);
+    CloudSolrClient cloudClient = cluster.getSolrClient();
+    new UpdateRequest()
+        .deleteByQuery("*:*")
+        .commit(cluster.getSolrClient(), collectionName);
+    new UpdateRequest()
+        .add(sdoc("id", "1"))
+        .commit(cloudClient, collectionName);
+    waitForNumDocsInAllActiveReplicas(1);
+    new UpdateRequest()
+        .deleteById("1")
+        .process(cloudClient, collectionName);
+    boolean successs = false;
+    try {
+      checkRTG(1, 1, cluster.getJettySolrRunners());
+      successs = true;
+    } catch (AssertionError e) {
+      //expected
+    }
+    assertFalse("Doc1 is deleted but it's still exist", successs);
+  }
+  
+  public void testBasicLeaderElection() throws Exception {
+    createAndWaitForCollection(1,0,2,0);
+    CloudSolrClient cloudClient = cluster.getSolrClient();
+    new UpdateRequest()
+        .deleteByQuery("*:*")
+        .commit(cluster.getSolrClient(), collectionName);
+    new UpdateRequest()
+        .add(sdoc("id", "1"))
+        .add(sdoc("id", "2"))
+        .process(cloudClient, collectionName);
+    JettySolrRunner oldLeaderJetty = getSolrRunner(true).get(0);
+    ChaosMonkey.kill(oldLeaderJetty);
+    waitForState("Replica not removed", collectionName, activeReplicaCount(0, 1, 0));
+    new UpdateRequest()
+        .add(sdoc("id", "3"))
+        .add(sdoc("id", "4"))
+        .process(cloudClient, collectionName);
+    ChaosMonkey.start(oldLeaderJetty);
+    waitForState("Replica not added", collectionName, activeReplicaCount(0, 2, 0));
+    checkRTG(1,4, cluster.getJettySolrRunners());
+    new UpdateRequest()
+        .commit(cloudClient, collectionName);
+    waitForNumDocsInAllActiveReplicas(4, 0);
+  }
+  
+  public void testOutOfOrderDBQWithInPlaceUpdates() throws Exception {
+    createAndWaitForCollection(1,0,2,0);
+    assertFalse(getSolrCore(true).get(0).getLatestSchema().getField("inplace_updatable_int").indexed());
+    assertFalse(getSolrCore(true).get(0).getLatestSchema().getField("inplace_updatable_int").stored());
+    assertTrue(getSolrCore(true).get(0).getLatestSchema().getField("inplace_updatable_int").hasDocValues());
+    List<UpdateRequest> updates = new ArrayList<>();
+    updates.add(simulatedUpdateRequest(null, "id", 1, "title_s", "title0_new", "inplace_updatable_int", 5, "_version_", 1L)); // full update
+    updates.add(simulatedDBQ("inplace_updatable_int:5", 3L));
+    updates.add(simulatedUpdateRequest(1L, "id", 1, "inplace_updatable_int", 6, "_version_", 2L));
+    for (JettySolrRunner solrRunner: getSolrRunner(false)) {
+      try (SolrClient client = solrRunner.newClient()) {
+        for (UpdateRequest up : updates) {
+          up.process(client, collectionName);
+        }
+      }
+    }
+    JettySolrRunner oldLeaderJetty = getSolrRunner(true).get(0);
+    ChaosMonkey.kill(oldLeaderJetty);
+    waitForState("Replica not removed", collectionName, activeReplicaCount(0, 1, 0));
+    ChaosMonkey.start(oldLeaderJetty);
+    waitForState("Replica not added", collectionName, activeReplicaCount(0, 2, 0));
+    checkRTG(1,1, cluster.getJettySolrRunners());
+    SolrDocument doc = cluster.getSolrClient().getById(collectionName,"1");
+    assertNotNull(doc.get("title_s"));
+  }
+  
+  private UpdateRequest simulatedUpdateRequest(Long prevVersion, Object... fields) throws SolrServerException, IOException {
+    SolrInputDocument doc = sdoc(fields);
+
+    // get baseUrl of the leader
+    String baseUrl = getBaseUrl();
+
+    UpdateRequest ur = new UpdateRequest();
+    ur.add(doc);
+    ur.setParam("update.distrib", "FROMLEADER");
+    if (prevVersion != null) {
+      ur.setParam("distrib.inplace.prevversion", String.valueOf(prevVersion));
+      ur.setParam("distrib.inplace.update", "true");
+    }
+    ur.setParam("distrib.from", baseUrl);
+    return ur;
+  }
+
+  private UpdateRequest simulatedDBQ(String query, long version) throws SolrServerException, IOException {
+    String baseUrl = getBaseUrl();
+
+    UpdateRequest ur = new UpdateRequest();
+    ur.deleteByQuery(query);
+    ur.setParam("_version_", ""+version);
+    ur.setParam("update.distrib", "FROMLEADER");
+    ur.setParam("distrib.from", baseUrl);
+    return ur;
+  }
+
+  private String getBaseUrl() {
+    DocCollection collection = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(collectionName);
+    Slice slice = collection.getSlice("shard1");
+    return slice.getLeader().getCoreUrl();
+  }
+
+  private DocCollection createAndWaitForCollection(int numShards, int numNrtReplicas, int numTlogReplicas, int numPullReplicas) throws SolrServerException, IOException, KeeperException, InterruptedException {
+    CollectionAdminRequest.createCollection(collectionName, "conf", numShards, numNrtReplicas, numTlogReplicas, numPullReplicas)
+    .setMaxShardsPerNode(100)
+    .process(cluster.getSolrClient());
+    int numReplicasPerShard = numNrtReplicas + numTlogReplicas + numPullReplicas;
+    cluster.getSolrClient().getZkStateReader().registerCore(collectionName); //TODO: Why is this needed? see SOLR-9440 
+    waitForState("Expected collection to be created with " + numShards + " shards and  " + numReplicasPerShard + " replicas", 
+        collectionName, clusterShape(numShards, numReplicasPerShard));
+    return assertNumberOfReplicas(numNrtReplicas*numShards, numTlogReplicas*numShards, numPullReplicas*numShards, false, true);
+  }
+  
+  private void waitForNumDocsInAllActiveReplicas(int numDocs) throws IOException, SolrServerException, InterruptedException {
+    waitForNumDocsInAllActiveReplicas(numDocs, REPLICATION_TIMEOUT_SECS);
+  }
+  
+  private void waitForNumDocsInAllActiveReplicas(int numDocs, int timeout) throws IOException, SolrServerException, InterruptedException {
+    DocCollection docCollection = getCollectionState(collectionName);
+    waitForNumDocsInAllReplicas(numDocs, docCollection.getReplicas().stream().filter(r -> r.getState() == Replica.State.ACTIVE).collect(Collectors.toList()), timeout);
+  }
+    
+  private void waitForNumDocsInAllReplicas(int numDocs, Collection<Replica> replicas, int timeout) throws IOException, SolrServerException, InterruptedException {
+    waitForNumDocsInAllReplicas(numDocs, replicas, "*:*", timeout);
+  }
+  
+  private void waitForNumDocsInAllReplicas(int numDocs, Collection<Replica> replicas, String query, int timeout) throws IOException, SolrServerException, InterruptedException {
+    TimeOut t = new TimeOut(timeout, TimeUnit.SECONDS);
+    for (Replica r:replicas) {
+      if (!r.isActive(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes())) {
+        continue;
+      }
+      try (HttpSolrClient replicaClient = getHttpSolrClient(r.getCoreUrl())) {
+        while (true) {
+          try {
+            assertEquals("Replica " + r.getName() + " not up to date after " + timeout + " seconds",
+                numDocs, replicaClient.query(new SolrQuery(query)).getResults().getNumFound());
+            break;
+          } catch (AssertionError e) {
+            if (t.hasTimedOut()) {
+              throw e;
+            } else {
+              Thread.sleep(100);
+            }
+          }
+        }
+      }
+    }
+  }
+  
+  private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
+    TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
+    while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
+      try {
+        Thread.sleep(100);
+        if (t.hasTimedOut()) {
+          fail("Timed out waiting for collection " + collection + " to be deleted.");
+        }
+        cluster.getSolrClient().getZkStateReader().forceUpdateCollection(collection);
+      } catch(SolrException e) {
+        return;
+      }
+      
+    }
+  }
+  
+  private DocCollection assertNumberOfReplicas(int numNrtReplicas, int numTlogReplicas, int numPullReplicas, boolean updateCollection, boolean activeOnly) throws KeeperException, InterruptedException {
+    if (updateCollection) {
+      cluster.getSolrClient().getZkStateReader().forceUpdateCollection(collectionName);
+    }
+    DocCollection docCollection = getCollectionState(collectionName);
+    assertNotNull(docCollection);
+    assertEquals("Unexpected number of nrt replicas: " + docCollection, numNrtReplicas, 
+        docCollection.getReplicas(EnumSet.of(Replica.Type.NRT)).stream().filter(r->!activeOnly || r.getState() == Replica.State.ACTIVE).count());
+    assertEquals("Unexpected number of pull replicas: " + docCollection, numPullReplicas, 
+        docCollection.getReplicas(EnumSet.of(Replica.Type.PULL)).stream().filter(r->!activeOnly || r.getState() == Replica.State.ACTIVE).count());
+    assertEquals("Unexpected number of tlog replicas: " + docCollection, numTlogReplicas, 
+        docCollection.getReplicas(EnumSet.of(Replica.Type.TLOG)).stream().filter(r->!activeOnly || r.getState() == Replica.State.ACTIVE).count());
+    return docCollection;
+  }
+  
+  /*
+   * passes only if all replicas are active or down, and the "liveNodes" reflect the same status
+   */
+  private CollectionStatePredicate clusterStateReflectsActiveAndDownReplicas() {
+    return (liveNodes, collectionState) -> {
+      for (Replica r:collectionState.getReplicas()) {
+        if (r.getState() != Replica.State.DOWN && r.getState() != Replica.State.ACTIVE) {
+          return false;
+        }
+        if (r.getState() == Replica.State.DOWN && liveNodes.contains(r.getNodeName())) {
+          return false;
+        }
+        if (r.getState() == Replica.State.ACTIVE && !liveNodes.contains(r.getNodeName())) {
+          return false;
+        }
+      }
+      return true;
+    };
+  }
+  
+  
+  private CollectionStatePredicate activeReplicaCount(int numNrtReplicas, int numTlogReplicas, int numPullReplicas) {
+    return (liveNodes, collectionState) -> {
+      int nrtFound = 0, tlogFound = 0, pullFound = 0;
+      if (collectionState == null)
+        return false;
+      for (Slice slice : collectionState) {
+        for (Replica replica : slice) {
+          if (replica.isActive(liveNodes))
+            switch (replica.getType()) {
+              case TLOG:
+                tlogFound++;
+                break;
+              case PULL:
+                pullFound++;
+                break;
+              case NRT:
+                nrtFound++;
+                break;
+              default:
+                throw new AssertionError("Unexpected replica type");
+            }
+        }
+      }
+      return numNrtReplicas == nrtFound && numTlogReplicas == tlogFound && numPullReplicas == pullFound;
+    };
+  }
+  
+  private List<SolrCore> getSolrCore(boolean isLeader) {
+    List<SolrCore> rs = new ArrayList<>();
+
+    CloudSolrClient cloudClient = cluster.getSolrClient();
+    DocCollection docCollection = cloudClient.getZkStateReader().getClusterState().getCollection(collectionName);
+
+    for (JettySolrRunner solrRunner : cluster.getJettySolrRunners()) {
+      if (solrRunner.getCoreContainer() == null) continue;
+      for (SolrCore solrCore : solrRunner.getCoreContainer().getCores()) {
+        CloudDescriptor cloudDescriptor = solrCore.getCoreDescriptor().getCloudDescriptor();
+        Slice slice = docCollection.getSlice(cloudDescriptor.getShardId());
+        Replica replica = docCollection.getReplica(cloudDescriptor.getCoreNodeName());
+        if (slice.getLeader().equals(replica) && isLeader) {
+          rs.add(solrCore);
+        } else if (!slice.getLeader().equals(replica) && !isLeader) {
+          rs.add(solrCore);
+        }
+      }
+    }
+    return rs;
+  }
+  
+  private void checkRTG(int from, int to, List<JettySolrRunner> solrRunners) throws Exception{
+    for (JettySolrRunner solrRunner: solrRunners) {
+      try (SolrClient client = solrRunner.newClient()) {
+        for (int i = from; i <= to; i++) {
+          SolrQuery query = new SolrQuery();
+          query.set("distrib", false);
+          query.setRequestHandler("/get");
+          query.set("id",i);
+          QueryResponse res = client.query(collectionName, query);
+          assertNotNull("Can not find doc "+ i + " in " + solrRunner.getBaseUrl(),res.getResponse().get("doc"));
+        }
+      }
+    }
+  }
+  
+  private List<JettySolrRunner> getSolrRunner(boolean isLeader) {
+    List<JettySolrRunner> rs = new ArrayList<>();
+    CloudSolrClient cloudClient = cluster.getSolrClient();
+    DocCollection docCollection = cloudClient.getZkStateReader().getClusterState().getCollection(collectionName);
+    for (JettySolrRunner solrRunner : cluster.getJettySolrRunners()) {
+      if (solrRunner.getCoreContainer() == null) continue;
+      for (SolrCore solrCore : solrRunner.getCoreContainer().getCores()) {
+        CloudDescriptor cloudDescriptor = solrCore.getCoreDescriptor().getCloudDescriptor();
+        Slice slice = docCollection.getSlice(cloudDescriptor.getShardId());
+        Replica replica = docCollection.getReplica(cloudDescriptor.getCoreNodeName());
+        if (slice.getLeader() == replica && isLeader) {
+          rs.add(solrRunner);
+        } else if (slice.getLeader() != replica && !isLeader) {
+          rs.add(solrRunner);
+        }
+      }
+    }
+    return rs;
+  }
+  
+  private void waitForReplicasCatchUp(int numTry) throws IOException, InterruptedException {
+    String leaderTimeCommit = getSolrCore(true).get(0).getDeletionPolicy().getLatestCommit().getUserData().get(SolrIndexWriter.COMMIT_TIME_MSEC_KEY);
+    if (leaderTimeCommit == null) return;
+    for (int i = 0; i < numTry; i++) {
+      boolean inSync = true;
+      for (SolrCore solrCore : getSolrCore(false)) {
+        String replicateTimeCommit = solrCore.getDeletionPolicy().getLatestCommit().getUserData().get(SolrIndexWriter.COMMIT_TIME_MSEC_KEY);
+        if (!leaderTimeCommit.equals(replicateTimeCommit)) {
+          inSync = false;
+          Thread.sleep(500);
+          break;
+        }
+      }
+      if (inSync) return;
+    }
+
+    fail("Some replicas are not in sync with leader");
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsBasicDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsBasicDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsBasicDistributedZkTest.java
index 1bba523..1534162 100644
--- a/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsBasicDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsBasicDistributedZkTest.java
@@ -44,8 +44,8 @@ public class HdfsBasicDistributedZkTest extends BasicDistributedZkTest {
   }
 
   @Override
-  protected int getRealtimeReplicas() {
-    return -1;
+  protected boolean useTlogReplicas() {
+    return false;
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java
index df7e642..4184817 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrCloudReportersTest.java
@@ -78,7 +78,7 @@ public class SolrCloudReportersTest extends SolrCloudTestCase {
           String coreName = core.getName();
           String collectionName = core.getCoreDescriptor().getCollectionName();
           String coreNodeName = core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName();
-          String replicaName = coreName.split("_")[3];
+          String replicaName = coreName.substring(coreName.indexOf("_replica_") + 1);
           String shardId = core.getCoreDescriptor().getCloudDescriptor().getShardId();
 
           assertEquals("solr.core." + collectionName + "." + shardId + "." + replicaName, registryName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
index f9f377c..44b4a4e 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
@@ -111,8 +111,8 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
   }
 
   @Override
-  protected int getRealtimeReplicas() {
-    return onlyLeaderIndexes? 1 : -1;
+  protected boolean useTlogReplicas() {
+    return onlyLeaderIndexes;
   }
 
   @After
@@ -270,7 +270,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
 
   private void reorderedDBQIndividualReplicaTest() throws Exception {
     if (onlyLeaderIndexes) {
-      log.info("RTG with DBQs are not working in append replicas");
+      log.info("RTG with DBQs are not working in tlog replicas");
       return;
     }
     clearIndex();
@@ -743,7 +743,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
    */
   private void reorderedDBQsResurrectionTest() throws Exception {
     if (onlyLeaderIndexes) {
-      log.info("RTG with DBQs are not working in append replicas");
+      log.info("RTG with DBQs are not working in tlog replicas");
       return;
     }
     clearIndex();
@@ -1145,7 +1145,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
    */
   private void reorderedDBQsUsingUpdatedValueFromADroppedUpdate() throws Exception {
     if (onlyLeaderIndexes) {
-      log.info("RTG with DBQs are not working in append replicas");
+      log.info("RTG with DBQs are not working in tlog replicas");
       return;
     }
     clearIndex();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
index fa93503..bc37c13 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
@@ -449,9 +449,12 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
     try {
       Runner r = new Runner();
       runners.add(r);
-      
-      scheduler.execute(r);  // this can throw an exception if the scheduler has been shutdown, but that should be fine.
-
+      try {
+        scheduler.execute(r);  // this can throw an exception if the scheduler has been shutdown, but that should be fine.
+      } catch (RuntimeException e) {
+        runners.remove(r);
+        throw e;
+      }
     } finally {
       MDC.remove("ConcurrentUpdateSolrClient.url");
     }
@@ -640,9 +643,15 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
   }
 
   private void waitForEmptyQueue() {
+    boolean threadInterrupted = Thread.currentThread().isInterrupted();
 
     while (!queue.isEmpty()) {
       if (log.isDebugEnabled()) emptyQueueLoops.incrementAndGet();
+      if (scheduler.isTerminated()) {
+        log.warn("The task queue still has elements but the update scheduler {} is terminated. Can't process any more tasks. "
+            + "Queue size: {}, Runners: {}. Current thread Interrupted? {}", scheduler, queue.size(), runners.size(), threadInterrupted);
+        break;
+      }
 
       synchronized (runners) {
         int queueSize = queue.size();
@@ -656,10 +665,16 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
         try {
           queue.wait(250);
         } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
+          // If we set the thread as interrupted again, the next time the wait it's called i t's going to return immediately
+          threadInterrupted = true;
+          log.warn("Thread interrupted while waiting for update queue to be empty. There are still {} elements in the queue.", 
+              queue.size());
         }
       }
     }
+    if (threadInterrupted) {
+      Thread.currentThread().interrupt();
+    }
   }
 
   public void handleError(Throwable ex) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
index 0c1116a..6c00b5c 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
@@ -16,6 +16,8 @@
  */
 package org.apache.solr.client.solrj.request;
 
+import static org.apache.solr.common.params.CollectionAdminParams.COUNT_PROP;
+
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Map;
@@ -34,6 +36,7 @@ import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.ImplicitDocRouter;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.CollectionParams.CollectionAction;
@@ -45,7 +48,6 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.NamedList;
 
-import static org.apache.solr.common.params.CollectionAdminParams.COUNT_PROP;
 import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SET_PARAM;
 import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM;
 
@@ -323,10 +325,23 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
    * @param collection the collection name
    * @param config     the collection config
    * @param numShards  the number of shards in the collection
-   * @param numReplicas the replication factor of the collection
+   * @param numNrtReplicas the number of {@link org.apache.solr.common.cloud.Replica.Type#NRT} replicas
+   * @param numTlogReplicas the number of {@link org.apache.solr.common.cloud.Replica.Type#TLOG} replicas
+   * @param numPullReplicas the number of {@link org.apache.solr.common.cloud.Replica.Type#PULL} replicas
+   */
+  public static Create createCollection(String collection, String config, int numShards, int numNrtReplicas, int numTlogReplicas, int numPullReplicas) {
+    return new Create(collection, config, numShards, numNrtReplicas, numTlogReplicas, numPullReplicas);
+  }
+  
+  /**
+   * Returns a SolrRequest for creating a collection
+   * @param collection the collection name
+   * @param config     the collection config
+   * @param numShards  the number of shards in the collection
+   * @param numReplicas the replication factor of the collection (same as numNrtReplicas)
    */
   public static Create createCollection(String collection, String config, int numShards, int numReplicas) {
-    return new Create(collection, config, numShards, numReplicas);
+    return new Create(collection, config, numShards, numReplicas, 0, 0);
   }
 
   /**
@@ -340,7 +355,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
    * @param numReplicas the replication factor of the collection
    */
   public static Create createCollection(String collection, int numShards, int numReplicas) {
-    return new Create(collection, numShards, numReplicas);
+    return new Create(collection, null, numShards, numReplicas, 0, 0);
   }
 
   /**
@@ -353,6 +368,22 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
   public static Create createCollectionWithImplicitRouter(String collection, String config, String shards, int numReplicas) {
     return new Create(collection, config, shards, numReplicas);
   }
+  
+  /**
+   * Returns a SolrRequest for creating a collection with the implicit router and specific types of replicas
+   * @param collection  the collection name
+   * @param config      the collection config
+   * @param shards      a shard definition string
+   * @param numNrtReplicas the number of replicas of type {@link org.apache.solr.common.cloud.Replica.Type#NRT}
+   * @param numTlogReplicas the number of replicas of type {@link org.apache.solr.common.cloud.Replica.Type#TLOG}
+   * @param numPullReplicas the number of replicas of type {@link org.apache.solr.common.cloud.Replica.Type#PULL}
+   */
+  public static Create createCollectionWithImplicitRouter(String collection, String config, String shards, int numNrtReplicas, int numTlogReplicas, int numPullReplicas) {
+    Create createRequest = new Create(collection, config, shards, numNrtReplicas);
+    createRequest.tlogReplicas = numTlogReplicas;
+    createRequest.pullReplicas = numPullReplicas;
+    return createRequest;
+  }
 
   // CREATE request
   public static class Create extends AsyncCollectionSpecificAdminRequest {
@@ -364,39 +395,36 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     protected String routerField;
     protected Integer numShards;
     protected Integer maxShardsPerNode;
-    protected Integer replicationFactor;
+    protected Integer nrtReplicas;
+    protected Integer pullReplicas;
+    protected Integer tlogReplicas;
 
     private Properties properties;
     protected Boolean autoAddReplicas;
-    protected Integer realtimeReplicas;
     protected Integer stateFormat;
     private String[] rule , snitch;
 
     /**
-     * @deprecated Use {@link #createCollection(String, String, int, int)}
+     * @deprecated Use {@link #createCollection(String, String, int, int, int, int)}
      */
     @Deprecated
     public Create() {
       super(CollectionAction.CREATE, null);
     }
-
-    private Create(String collection, String config, int numShards, int numReplicas) {
+    
+    private Create(String collection, String config, int numShards, int numNrtReplicas, int numTlogReplicas, int numPullReplicas) { // TODO: maybe add other constructors
       super(CollectionAction.CREATE, SolrIdentifierValidator.validateCollectionName(collection));
       this.configName = config;
       this.numShards = numShards;
-      this.replicationFactor = numReplicas;
-    }
-
-    private Create(String collection, int numShards, int numReplicas) {
-      super(CollectionAction.CREATE, SolrIdentifierValidator.validateCollectionName(collection));
-      this.numShards = numShards;
-      this.replicationFactor = numReplicas;
+      this.nrtReplicas = numNrtReplicas;
+      this.pullReplicas = numPullReplicas;
+      this.tlogReplicas = numTlogReplicas;
     }
 
-    private Create(String collection, String config, String shards, int numReplicas) {
+    private Create(String collection, String config, String shards, int numNrtReplicas) {
       super(CollectionAction.CREATE, SolrIdentifierValidator.validateCollectionName(collection));
       this.configName = config;
-      this.replicationFactor = numReplicas;
+      this.nrtReplicas = numNrtReplicas;
       this.shards = shards;
       this.routerName = ImplicitDocRouter.NAME;
     }
@@ -410,9 +438,11 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     public Create setNumShards(Integer numShards) {this.numShards = numShards; return this; }
     public Create setMaxShardsPerNode(Integer numShards) { this.maxShardsPerNode = numShards; return this; }
     public Create setAutoAddReplicas(boolean autoAddReplicas) { this.autoAddReplicas = autoAddReplicas; return this; }
-    public Create setRealtimeReplicas(Integer realtimeReplicas) { this.realtimeReplicas = realtimeReplicas; return this;}
-    @Deprecated
-    public Create setReplicationFactor(Integer repl) { this.replicationFactor = repl; return this; }
+    public Create setNrtReplicas(Integer nrtReplicas) { this.nrtReplicas = nrtReplicas; return this;}
+    public Create setTlogReplicas(Integer tlogReplicas) { this.tlogReplicas = tlogReplicas; return this;}
+    public Create setPullReplicas(Integer pullReplicas) { this.pullReplicas = pullReplicas; return this;}
+
+    public Create setReplicationFactor(Integer repl) { this.nrtReplicas = repl; return this; }
     public Create setStateFormat(Integer stateFormat) { this.stateFormat = stateFormat; return this; }
     public Create setRule(String... s){ this.rule = s; return this; }
     public Create setSnitch(String... s){ this.snitch = s; return this; }
@@ -423,9 +453,13 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     public String getShards() { return  shards; }
     public Integer getNumShards() { return numShards; }
     public Integer getMaxShardsPerNode() { return maxShardsPerNode; }
-    public Integer getReplicationFactor() { return replicationFactor; }
+    
+    public Integer getReplicationFactor() { return getNumNrtReplicas(); }
+    public Integer getNumNrtReplicas() { return nrtReplicas; }
     public Boolean getAutoAddReplicas() { return autoAddReplicas; }
-    public Integer getRealtimeReplicas() { return realtimeReplicas; }
+    public Integer getNumTlogReplicas() {return tlogReplicas;}
+    public Integer getNumPullReplicas() {return pullReplicas;}
+
     public Integer getStateFormat() { return stateFormat; }
     
     /**
@@ -506,21 +540,25 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       if (routerField != null) {
         params.set("router.field", routerField);
       }
-      if (replicationFactor != null) {
-        params.set( "replicationFactor", replicationFactor);
+      if (nrtReplicas != null) {
+        params.set( "replicationFactor", nrtReplicas);// Keep both for compatibility?
+        params.set( ZkStateReader.NRT_REPLICAS, nrtReplicas);
       }
       if (autoAddReplicas != null) {
         params.set(ZkStateReader.AUTO_ADD_REPLICAS, autoAddReplicas);
       }
-      if (realtimeReplicas != null) {
-        params.set(ZkStateReader.REALTIME_REPLICAS, realtimeReplicas);
-      }
       if(properties != null) {
         addProperties(params, properties);
       }
       if (stateFormat != null) {
         params.set(DocCollection.STATE_FORMAT, stateFormat);
       }
+      if (pullReplicas != null) {
+        params.set(ZkStateReader.PULL_REPLICAS, pullReplicas);
+      }
+      if (tlogReplicas != null) {
+        params.set(ZkStateReader.TLOG_REPLICAS, tlogReplicas);
+      }
       if(rule != null) params.set("rule", rule);
       if(snitch != null) params.set("snitch", snitch);
       return params;
@@ -1641,19 +1679,26 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
 
 
   }
-
+  
   /**
-   * Returns a SolrRequest to add a replica to a shard in a collection
+   * Returns a SolrRequest to add a replica of type {@link org.apache.solr.common.cloud.Replica.Type#NRT} to a shard in a collection
    */
   public static AddReplica addReplicaToShard(String collection, String shard) {
-    return new AddReplica(collection, shard, null);
+    return addReplicaToShard(collection, shard, Replica.Type.NRT);
+  }
+
+  /**
+   * Returns a SolrRequest to add a replica of the specified type to a shard in a collection
+   */
+  public static AddReplica addReplicaToShard(String collection, String shard, Replica.Type replicaType) {
+    return new AddReplica(collection, shard, null, replicaType);
   }
 
   /**
    * Returns a SolrRequest to add a replica to a collection using a route key
    */
   public static AddReplica addReplicaByRouteKey(String collection, String routeKey) {
-    return new AddReplica(collection, null, routeKey);
+    return new AddReplica(collection, null, routeKey, Replica.Type.NRT);
   }
 
   // ADDREPLICA request
@@ -1666,6 +1711,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     protected String instanceDir;
     protected String dataDir;
     protected Properties properties;
+    protected Replica.Type type;
 
     /**
      * @deprecated Use {@link #addReplicaByRouteKey(String, String)} or {@link #addReplicaToShard(String, String)}
@@ -1675,11 +1721,12 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       super(CollectionAction.ADDREPLICA);
     }
 
-    private AddReplica(String collection, String shard, String routeKey) {
+    private AddReplica(String collection, String shard, String routeKey, Replica.Type type) {
       super(CollectionAction.ADDREPLICA);
       this.collection = collection;
       this.shard = shard;
       this.routeKey = routeKey;
+      this.type = type;
     }
 
     public Properties getProperties() {
@@ -1753,6 +1800,11 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       this.asyncId = id;
       return this;
     }
+    
+    public AddReplica setType(Replica.Type type) {
+      this.type = type;
+      return this;
+    }
 
     @Override
     public SolrParams getParams() {
@@ -1778,6 +1830,9 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       if (dataDir != null)  {
         params.add("dataDir", dataDir);
       }
+      if (type != null) {
+        params.add(ZkStateReader.REPLICA_TYPE, type.name());
+      }
       if (properties != null) {
         addProperties(params, properties);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
index d89b2f6..5c3f895 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
@@ -18,6 +18,7 @@ package org.apache.solr.common.cloud;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
@@ -33,8 +34,10 @@ import org.noggit.JSONWriter;
 
 import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
-import static org.apache.solr.common.cloud.ZkStateReader.REALTIME_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
+import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
 
 /**
  * Models a Collection in zookeeper (but that Java name is obviously taken, hence "DocCollection")
@@ -58,10 +61,11 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
   private final String znode;
 
   private final Integer replicationFactor;
+  private final Integer numNrtReplicas;
+  private final Integer numTlogReplicas;
+  private final Integer numPullReplicas;
   private final Integer maxShardsPerNode;
   private final Boolean autoAddReplicas;
-  private final Integer realtimeReplicas;
-
 
   public DocCollection(String name, Map<String, Slice> slices, Map<String, Object> props, DocRouter router) {
     this(name, slices, props, router, Integer.MAX_VALUE, ZkStateReader.CLUSTER_STATE);
@@ -83,14 +87,13 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
     this.nodeNameLeaderReplicas = new HashMap<>();
     this.nodeNameReplicas = new HashMap<>();
     this.replicationFactor = (Integer) verifyProp(props, REPLICATION_FACTOR);
+    this.numNrtReplicas = (Integer) verifyProp(props, NRT_REPLICAS);
+    this.numTlogReplicas = (Integer) verifyProp(props, TLOG_REPLICAS);
+    this.numPullReplicas = (Integer) verifyProp(props, PULL_REPLICAS);
     this.maxShardsPerNode = (Integer) verifyProp(props, MAX_SHARDS_PER_NODE);
     Boolean autoAddReplicas = (Boolean) verifyProp(props, AUTO_ADD_REPLICAS);
     this.autoAddReplicas = autoAddReplicas == null ? Boolean.FALSE : autoAddReplicas;
-    Integer realtimeReplicas = (Integer) verifyProp(props, REALTIME_REPLICAS);
-    this.realtimeReplicas = realtimeReplicas == null ? -1 : realtimeReplicas;
-    if (this.realtimeReplicas != -1 && this.realtimeReplicas != 1) {
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Invalid realtimeReplicas must be 1 or -1, found:" + this.realtimeReplicas);
-    }
+    
     verifyProp(props, RULE);
     verifyProp(props, SNITCH);
     Iterator<Map.Entry<String, Slice>> iter = slices.entrySet().iterator();
@@ -133,7 +136,9 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
     switch (propName) {
       case MAX_SHARDS_PER_NODE:
       case REPLICATION_FACTOR:
-      case REALTIME_REPLICAS:
+      case NRT_REPLICAS:
+      case PULL_REPLICAS:
+      case TLOG_REPLICAS:
         return Integer.parseInt(o.toString());
       case AUTO_ADD_REPLICAS:
         return Boolean.parseBoolean(o.toString());
@@ -234,10 +239,6 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
     return maxShardsPerNode;
   }
 
-  public int getRealtimeReplicas() {
-    return realtimeReplicas;
-  }
-
   public String getZNode(){
     return znode;
   }
@@ -311,6 +312,14 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
     }
     return replicas;
   }
+  
+  public List<Replica> getReplicas(EnumSet<Replica.Type> s) {
+    List<Replica> replicas = new ArrayList<>();
+    for (Slice slice : this) {
+      replicas.addAll(slice.getReplicas(s));
+    }
+    return replicas;
+  }
 
   /**
    * Get the shardId of a core on a specific node
@@ -333,4 +342,25 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
     return super.equals(that) && Objects.equals(this.znode, other.znode) && this.znodeVersion == other.znodeVersion;
   }
 
+  /**
+   * @return the number of replicas of type {@link org.apache.solr.common.cloud.Replica.Type#NRT} this collection was created with
+   */
+  public Integer getNumNrtReplicas() {
+    return numNrtReplicas;
+  }
+
+  /**
+   * @return the number of replicas of type {@link org.apache.solr.common.cloud.Replica.Type#TLOG} this collection was created with
+   */
+  public Integer getNumTlogReplicas() {
+    return numTlogReplicas;
+  }
+
+  /**
+   * @return the number of replicas of type {@link org.apache.solr.common.cloud.Replica.Type#PULL} this collection was created with
+   */
+  public Integer getNumPullReplicas() {
+    return numPullReplicas;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
index 4968cf2..b7655be 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
@@ -84,9 +84,31 @@ public class Replica extends ZkNodeProps {
     }
   }
 
+  public enum Type {
+    /**
+     * Writes updates to transaction log and indexes locally. Replicas of type {@link Type#NRT} support NRT (soft commits) and RTG. 
+     * Any {@link Type#NRT} replica can become a leader. A shard leader will forward updates to all active {@link Type#NRT} and
+     * {@link Type#TLOG} replicas. 
+     */
+    NRT,
+    /**
+     * Writes to transaction log, but not to index, uses replication. Any {@link Type#TLOG} replica can become leader (by first
+     * applying all local transaction log elements). If a replica is of type {@link Type#TLOG} but is also the leader, it will behave 
+     * as a {@link Type#NRT}. A shard leader will forward updates to all active {@link Type#NRT} and {@link Type#TLOG} replicas.
+     */
+    TLOG,
+    /**
+     * Doesn’t index or writes to transaction log. Just replicates from {@link Type#NRT} or {@link Type#TLOG} replicas. {@link Type#PULL}
+     * replicas can’t become shard leaders (i.e., if there are only pull replicas in the collection at some point, updates will fail
+     * same as if there is no leaders, queries continue to work), so they don’t even participate in elections.
+     */
+    PULL
+  }
+
   private final String name;
   private final String nodeName;
   private final State state;
+  private final Type type;
 
   public Replica(String name, Map<String,Object> propMap) {
     super(propMap);
@@ -98,6 +120,12 @@ public class Replica extends ZkNodeProps {
       this.state = State.ACTIVE;                         //Default to ACTIVE
       propMap.put(ZkStateReader.STATE_PROP, state.toString());
     }
+    String typeString = (String)propMap.get(ZkStateReader.REPLICA_TYPE);
+    if (typeString == null) {
+      this.type = Type.NRT;
+    } else {
+      this.type = Type.valueOf(typeString);
+    }
 
   }
 
@@ -129,6 +157,10 @@ public class Replica extends ZkNodeProps {
   public boolean isActive(Set<String> liveNodes) {
     return liveNodes.contains(this.nodeName) && this.state == State.ACTIVE;
   }
+  
+  public Type getType() {
+    return this.type;
+  }
 
   @Override
   public String toString() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java b/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
index bd3bafd..477c1b1 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
@@ -18,6 +18,7 @@ package org.apache.solr.common.cloud;
 
 import java.util.Collection;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
@@ -27,6 +28,7 @@ import java.util.Map;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
+import org.apache.solr.common.cloud.Replica.Type;
 import org.noggit.JSONUtil;
 import org.noggit.JSONWriter;
 
@@ -161,7 +163,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
     // add the replicas *after* the other properties (for aesthetics, so it's easy to find slice properties in the JSON output)
     this.replicas = replicas != null ? replicas : makeReplicas((Map<String,Object>)propMap.get(REPLICAS));
     propMap.put(REPLICAS, this.replicas);
-
+    
     Map<String, Object> rules = (Map<String, Object>) propMap.get("routingRules");
     if (rules != null) {
       this.routingRules = new HashMap<>();
@@ -202,7 +204,10 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
 
   private Replica findLeader() {
     for (Replica replica : replicas.values()) {
-      if (replica.getStr(LEADER) != null) return replica;
+      if (replica.getStr(LEADER) != null) {
+        assert replica.getType() == Type.TLOG || replica.getType() == Type.NRT: "Pull replica should not become leader!";
+        return replica;
+      }
     }
     return null;
   }
@@ -215,7 +220,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
   }
 
   /**
-   * Gets the list of replicas for this slice.
+   * Gets the list of all replicas for this slice.
    */
   public Collection<Replica> getReplicas() {
     return replicas.values();
@@ -227,6 +232,13 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
   public List<Replica> getReplicas(Predicate<Replica> pred) {
     return replicas.values().stream().filter(pred).collect(Collectors.toList());
   }
+  
+  /**
+   * Gets the list of replicas that have a type present in s
+   */
+  public List<Replica> getReplicas(EnumSet<Replica.Type> s) {
+    return this.getReplicas(r->s.contains(r.getType()));
+  }
 
   /**
    * Get the map of coreNodeName to replicas for this slice.
@@ -238,7 +250,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
   public Map<String,Replica> getReplicasCopy() {
     return new LinkedHashMap<>(replicas);
   }
-
+  
   public Replica getLeader() {
     return leader;
   }
@@ -272,4 +284,5 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
   public void write(JSONWriter jsonWriter) {
     jsonWriter.write(propMap);
   }
+  
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 7cefbee..158b53a 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -23,6 +23,7 @@ import java.net.URLDecoder;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -39,6 +40,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
 
 import org.apache.solr.common.Callable;
 import org.apache.solr.common.SolrException;
@@ -96,7 +98,9 @@ public class ZkStateReader implements Closeable {
   public static final String MAX_SHARDS_PER_NODE = "maxShardsPerNode";
   public static final String AUTO_ADD_REPLICAS = "autoAddReplicas";
   public static final String MAX_CORES_PER_NODE = "maxCoresPerNode";
-  public static final String REALTIME_REPLICAS = "realtimeReplicas";
+  public static final String PULL_REPLICAS = "pullReplicas";
+  public static final String NRT_REPLICAS = "nrtReplicas";
+  public static final String TLOG_REPLICAS = "tlogReplicas";
 
   public static final String ROLES = "/roles.json";
 
@@ -106,6 +110,8 @@ public class ZkStateReader implements Closeable {
   public static final String LEGACY_CLOUD = "legacyCloud";
 
   public static final String URL_SCHEME = "urlScheme";
+  
+  public static final String REPLICA_TYPE = "type";
 
 
   /** A view of the current state of all collections; combines all the different state sources into a single view. */
@@ -780,6 +786,12 @@ public class ZkStateReader implements Closeable {
   
   public List<ZkCoreNodeProps> getReplicaProps(String collection, String shardId, String thisCoreNodeName,
       Replica.State mustMatchStateFilter, Replica.State mustNotMatchStateFilter) {
+    //TODO: We don't need all these getReplicaProps method overloading. Also, it's odd that the default is to return replicas of type TLOG and NRT only
+    return getReplicaProps(collection, shardId, thisCoreNodeName, mustMatchStateFilter, null, EnumSet.of(Replica.Type.TLOG,  Replica.Type.NRT));
+  }
+  
+  public List<ZkCoreNodeProps> getReplicaProps(String collection, String shardId, String thisCoreNodeName,
+      Replica.State mustMatchStateFilter, Replica.State mustNotMatchStateFilter, final EnumSet<Replica.Type> acceptReplicaType) {
     assert thisCoreNodeName != null;
     ClusterState clusterState = this.clusterState;
     if (clusterState == null) {
@@ -798,7 +810,7 @@ public class ZkStateReader implements Closeable {
     
     Map<String,Replica> shardMap = replicas.getReplicasMap();
     List<ZkCoreNodeProps> nodes = new ArrayList<>(shardMap.size());
-    for (Entry<String,Replica> entry : shardMap.entrySet()) {
+    for (Entry<String,Replica> entry : shardMap.entrySet().stream().filter((e)->acceptReplicaType.contains(e.getValue().getType())).collect(Collectors.toList())) {
       ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue());
       
       String coreNodeName = entry.getValue().getName();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
index f3e0d7e..3fc6157 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
@@ -128,6 +128,11 @@ public abstract class CoreAdminParams
    */
   public static final String NEW_COLLECTION = "newCollection";
 
+  /**
+   * Tells the CoreAdminHandler that the new Core will be a replica of a particular {@link org.apache.solr.common.cloud.Replica.Type}
+   */
+  public static final String REPLICA_TYPE = "replicaType";
+
   public enum CoreAdminAction {
     STATUS(true),
     UNLOAD,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/solrj/src/test/org/apache/solr/client/solrj/CollectionAdminRequestRequiredParamsTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/CollectionAdminRequestRequiredParamsTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/CollectionAdminRequestRequiredParamsTest.java
index 50f2d18..0f9b147 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/CollectionAdminRequestRequiredParamsTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/CollectionAdminRequestRequiredParamsTest.java
@@ -23,6 +23,8 @@ import java.util.Set;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.SolrParams;
 
@@ -74,6 +76,13 @@ public class CollectionAdminRequestRequiredParamsTest extends LuceneTestCase {
             .setRouteKey("route")
             .setCollectionName("collection");
     assertContainsParams(request.getParams(), ACTION, COLLECTION, ShardParams._ROUTE_);
+    
+    // with type parameter
+    request = new CollectionAdminRequest.AddReplica()
+            .setShardName("shard")
+            .setCollectionName("collection")
+            .setType(Replica.Type.NRT);
+    assertContainsParams(request.getParams(), ACTION, COLLECTION, SHARD, ZkStateReader.REPLICA_TYPE);
   }
   
   public void testAddReplicaProp() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
index 8c6eb60..ed778a6 100644
--- a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
@@ -451,7 +451,7 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
     return buildUrl(port, context);
   }
 
-  protected void addFields(SolrInputDocument doc, Object... fields) {
+  protected static void addFields(SolrInputDocument doc, Object... fields) {
     for (int i = 0; i < fields.length; i += 2) {
       doc.addField((String) (fields[i]), fields[i + 1]);
     }
@@ -1123,6 +1123,7 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
     coreProperties.setProperty("config", "${solrconfig:solrconfig.xml}");
     coreProperties.setProperty("schema", "${schema:schema.xml}");
     coreProperties.setProperty("coreNodeName", "${coreNodeName:}");
+    coreProperties.setProperty("replicaType", "${replicaType:}");
 
     writeCoreProperties(jettyHome.toPath().resolve("cores").resolve("collection1"), coreProperties, "collection1");
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2fc41d56/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index cd49357..a55e2ea 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -778,8 +778,8 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
    * is set.
    */
   public static void deleteCore() {
-    log.info("###deleteCore" );
     if (h != null) {
+      log.info("###deleteCore" );
       // If the test case set up Zk, it should still have it as available,
       // otherwise the core close will just be unnecessarily delayed.
       CoreContainer cc = h.getCoreContainer();
@@ -2123,9 +2123,17 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
     SolrDocumentList list1 = (SolrDocumentList) expected;
     SolrDocumentList list2 = (SolrDocumentList) actual;
 
-    if(Float.compare(list1.getMaxScore(), list2.getMaxScore()) != 0 || list1.getNumFound() != list2.getNumFound() ||
-        list1.getStart() != list2.getStart()) {
+    if (list1.getMaxScore() == null) {
+      if (list2.getMaxScore() != null) {
+        return false;
+      } 
+    } else if (list2.getMaxScore() == null) {
       return false;
+    } else {
+      if (Float.compare(list1.getMaxScore(), list2.getMaxScore()) != 0 || list1.getNumFound() != list2.getNumFound() ||
+          list1.getStart() != list2.getStart()) {
+        return false;
+      }
     }
     for(int i=0; i<list1.getNumFound(); i++) {
       if(!compareSolrDocument(list1.get(i), list2.get(i))) {
@@ -2425,24 +2433,27 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
     }
     return result;
   }
-
-  protected void waitForWarming() throws InterruptedException {
-    RefCounted<SolrIndexSearcher> registeredSearcher = h.getCore().getRegisteredSearcher();
-    RefCounted<SolrIndexSearcher> newestSearcher = h.getCore().getNewestSearcher(false);
-    ;
+  
+  protected static void waitForWarming(SolrCore core) throws InterruptedException {
+    RefCounted<SolrIndexSearcher> registeredSearcher = core.getRegisteredSearcher();
+    RefCounted<SolrIndexSearcher> newestSearcher = core.getNewestSearcher(false);
     while (registeredSearcher == null || registeredSearcher.get() != newestSearcher.get()) {
       if (registeredSearcher != null) {
         registeredSearcher.decref();
       }
       newestSearcher.decref();
       Thread.sleep(50);
-      registeredSearcher = h.getCore().getRegisteredSearcher();
-      newestSearcher = h.getCore().getNewestSearcher(false);
+      registeredSearcher = core.getRegisteredSearcher();
+      newestSearcher = core.getNewestSearcher(false);
     }
     registeredSearcher.decref();
     newestSearcher.decref();
   }
 
+  protected void waitForWarming() throws InterruptedException {
+    waitForWarming(h.getCore());
+  }
+
   @BeforeClass
   public static void chooseMPForMP() throws Exception {
     if (random().nextBoolean()) {