You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2016/03/25 22:09:23 UTC

[2/4] lucene-solr:branch_6x: SOLR-445: new ToleranteUpdateProcessorFactory to support skipping update commands that cause failures when sending multiple updates in a single request.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b6eacb8/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
new file mode 100644
index 0000000..054c074
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
@@ -0,0 +1,1065 @@
+/*
+ * 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.File;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.SolrInputField;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.ToleratedUpdateError;
+import org.apache.solr.common.ToleratedUpdateError.CmdType;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.util.RevertDefaultThreadHandlerRule;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test of TolerantUpdateProcessor using a MiniSolrCloud.  Updates (that include failures which 
+ * should be tolerated) are explicitly tested against various initial nodes to confirm correct 
+ * behavior regardless of routing.
+ *
+ * <p>
+ * <b>NOTE:</b> This test sets up a static instance of MiniSolrCloud with a single collection 
+ * and several clients pointed at specific nodes. These are all re-used across multiple test methods, 
+ * and assumes that the state of the cluster is healthy.
+ * </p>
+ *
+ */
+public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final int NUM_SHARDS = 2; 
+  private static final int REPLICATION_FACTOR = 2; 
+  private static final int NUM_SERVERS = 5; 
+  
+  private static final String COLLECTION_NAME = "test_col";
+  
+  /** A basic client for operations at the cloud level, default collection will be set */
+  private static CloudSolrClient CLOUD_CLIENT;
+
+  /** A client for talking directly to the leader of shard1 */
+  private static HttpSolrClient S_ONE_LEADER_CLIENT;
+  
+  /** A client for talking directly to the leader of shard2 */
+  private static HttpSolrClient S_TWO_LEADER_CLIENT;
+
+  /** A client for talking directly to a passive replica of shard1 */
+  private static HttpSolrClient S_ONE_NON_LEADER_CLIENT;
+  
+  /** A client for talking directly to a passive replica of shard2 */
+  private static HttpSolrClient S_TWO_NON_LEADER_CLIENT;
+
+  /** A client for talking directly to a node that has no piece of the collection */
+  private static HttpSolrClient NO_COLLECTION_CLIENT;
+  
+  /** id field doc routing prefix for shard1 */
+  private static final String S_ONE_PRE = "abc!";
+  
+  /** id field doc routing prefix for shard2 */
+  private static final String S_TWO_PRE = "XYZ!";
+  
+  @BeforeClass
+  private static void createMiniSolrCloudCluster() throws Exception {
+    
+    final String configName = "solrCloudCollectionConfig";
+    final File configDir = new File(TEST_HOME() + File.separator + "collection1" + File.separator + "conf");
+
+    configureCluster(NUM_SERVERS)
+      .addConfig(configName, configDir.toPath())
+      .configure();
+    assertSpinLoopAllJettyAreRunning(cluster);
+    
+    Map<String, String> collectionProperties = new HashMap<>();
+    collectionProperties.put("config", "solrconfig-distrib-update-processor-chains.xml");
+    collectionProperties.put("schema", "schema15.xml"); // string id for doc routing prefix
+
+    assertNotNull(cluster.createCollection(COLLECTION_NAME, NUM_SHARDS, REPLICATION_FACTOR,
+                                           configName, null, null, collectionProperties));
+    
+    CLOUD_CLIENT = cluster.getSolrClient();
+    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+    
+    ZkStateReader zkStateReader = CLOUD_CLIENT.getZkStateReader();
+    AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION_NAME, zkStateReader, true, true, 330);
+
+
+    // really hackish way to get a URL for specific nodes based on shard/replica hosting
+    // inspired by TestMiniSolrCloudCluster
+    HashMap<String, String> urlMap = new HashMap<>();
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+      URL jettyURL = jetty.getBaseUrl();
+      String nodeKey = jettyURL.getHost() + ":" + jettyURL.getPort() + jettyURL.getPath().replace("/","_");
+      urlMap.put(nodeKey, jettyURL.toString());
+    }
+    zkStateReader.updateClusterState();
+    ClusterState clusterState = zkStateReader.getClusterState();
+    for (Slice slice : clusterState.getSlices(COLLECTION_NAME)) {
+      String shardName = slice.getName();
+      Replica leader = slice.getLeader();
+      assertNotNull("slice has null leader: " + slice.toString(), leader);
+      assertNotNull("slice leader has null node name: " + slice.toString(), leader.getNodeName());
+      String leaderUrl = urlMap.remove(leader.getNodeName());
+      assertNotNull("could not find URL for " + shardName + " leader: " + leader.getNodeName(),
+                    leaderUrl);
+      assertEquals("expected two total replicas for: " + slice.getName(),
+                   2, slice.getReplicas().size());
+      
+      String passiveUrl = null;
+      
+      for (Replica replica : slice.getReplicas()) {
+        if ( ! replica.equals(leader)) {
+          passiveUrl = urlMap.remove(replica.getNodeName());
+          assertNotNull("could not find URL for " + shardName + " replica: " + replica.getNodeName(),
+                        passiveUrl);
+        }
+      }
+      assertNotNull("could not find URL for " + shardName + " replica", passiveUrl);
+
+      if (shardName.equals("shard1")) {
+        S_ONE_LEADER_CLIENT = new HttpSolrClient(leaderUrl + "/" + COLLECTION_NAME + "/");
+        S_ONE_NON_LEADER_CLIENT = new HttpSolrClient(passiveUrl + "/" + COLLECTION_NAME + "/");
+      } else if (shardName.equals("shard2")) {
+        S_TWO_LEADER_CLIENT = new HttpSolrClient(leaderUrl + "/" + COLLECTION_NAME + "/");
+        S_TWO_NON_LEADER_CLIENT = new HttpSolrClient(passiveUrl + "/" + COLLECTION_NAME + "/");
+      } else {
+        fail("unexpected shard: " + shardName);
+      }
+    }
+    assertEquals("Should be exactly one server left (nost hosting either shard)", 1, urlMap.size());
+    NO_COLLECTION_CLIENT = new HttpSolrClient(urlMap.values().iterator().next() +
+                                              "/" + COLLECTION_NAME + "/");
+    
+    assertNotNull(S_ONE_LEADER_CLIENT);
+    assertNotNull(S_TWO_LEADER_CLIENT);
+    assertNotNull(S_ONE_NON_LEADER_CLIENT);
+    assertNotNull(S_TWO_NON_LEADER_CLIENT);
+    assertNotNull(NO_COLLECTION_CLIENT);
+
+    // sanity check that our S_ONE_PRE & S_TWO_PRE really do map to shard1 & shard2 with default routing
+    assertEquals(0, CLOUD_CLIENT.add(doc(f("id", S_ONE_PRE + random().nextInt()),
+                                         f("expected_shard_s", "shard1"))).getStatus());
+    assertEquals(0, CLOUD_CLIENT.add(doc(f("id", S_TWO_PRE + random().nextInt()),
+                                         f("expected_shard_s", "shard2"))).getStatus());
+    assertEquals(0, CLOUD_CLIENT.commit().getStatus());
+    SolrDocumentList docs = CLOUD_CLIENT.query(params("q", "*:*",
+                                                      "fl","id,expected_shard_s,[shard]")).getResults();
+    assertEquals(2, docs.getNumFound());
+    assertEquals(2, docs.size());
+    for (SolrDocument doc : docs) {
+      String expected = COLLECTION_NAME + "_" + doc.getFirstValue("expected_shard_s") + "_replica";
+      String docShard = doc.getFirstValue("[shard]").toString();
+      assertTrue("shard routing prefixes don't seem to be aligned anymore, " +
+                 "did someone change the default routing rules? " +
+                 "and/or the the default core name rules? " +
+                 "and/or the numShards used by this test? ... " +
+                 "couldn't find " + expected + " as substring of [shard] == '" + docShard +
+                 "' ... for docId == " + doc.getFirstValue("id"),
+                 docShard.contains(expected));
+    }
+  }
+  
+  @Before
+  private void clearCollection() throws Exception {
+    assertEquals(0, CLOUD_CLIENT.deleteByQuery("*:*").getStatus());
+    assertEquals(0, CLOUD_CLIENT.commit().getStatus());
+  }
+
+  public void testSanity() throws Exception {
+    
+    // verify some basic sanity checking of indexing & querying across the collection
+    // w/o using our custom update processor chain
+    
+    assertEquals(0, CLOUD_CLIENT.add(doc(f("id", S_ONE_PRE + "1"),
+                                         f("foo_i", 42))).getStatus());
+    assertEquals(0, CLOUD_CLIENT.add(doc(f("id", S_TWO_PRE + "2"),
+                                         f("foo_i", 66))).getStatus());
+    assertEquals(0, CLOUD_CLIENT.commit().getStatus());
+
+    for (SolrClient c : Arrays.asList(S_ONE_LEADER_CLIENT, S_TWO_LEADER_CLIENT,
+                                      S_ONE_NON_LEADER_CLIENT, S_TWO_NON_LEADER_CLIENT,
+                                      NO_COLLECTION_CLIENT, CLOUD_CLIENT)) {
+      assertQueryDocIds(c, true, S_ONE_PRE + "1",  S_TWO_PRE + "2");
+      assertQueryDocIds(c, false, "id_not_exists");
+
+      // verify adding 2 broken docs causes a clint exception
+      try {
+        UpdateResponse rsp = update(params(),
+                                    doc(f("id", S_ONE_PRE + "X"), f("foo_i", "bogus_val_X")),
+                                    doc(f("id", S_TWO_PRE + "Y"), f("foo_i", "bogus_val_Y"))
+                                    ).process(c);
+        fail("did not get a top level exception when more then 10 docs failed: " + rsp.toString());
+      } catch (SolrException e) {
+        assertEquals("not the type of error we were expecting ("+e.code()+"): " + e.toString(),
+                     400, e.code());
+      }
+        
+      // verify malformed deleteByQuerys fail
+      try {
+        UpdateResponse rsp = update(params()).deleteByQuery("foo_i:not_a_num").process(c);
+        fail("sanity check for malformed DBQ didn't fail: " + rsp.toString());
+      } catch (SolrException e) {
+        assertEquals("not the expected DBQ failure: " + e.getMessage(), 400, e.code());
+      }
+      
+      // verify oportunistic concurrency deletions fail as we expect when docs are / aren't present
+      for (UpdateRequest r : new UpdateRequest[] {
+          update(params("commit", "true")).deleteById(S_ONE_PRE + "1", -1L),
+          update(params("commit", "true")).deleteById(S_TWO_PRE + "2", -1L),
+          update(params("commit", "true")).deleteById("id_not_exists",  1L)    }) {
+        try {
+          UpdateResponse rsp = r.process(c);
+          fail("sanity check for oportunistic concurrency delete didn't fail: "
+               + r.toString() + " => " + rsp.toString());
+        } catch (SolrException e) {
+          assertEquals("not the expected oportunistic concurrency failure code: "
+                       + r.toString() + " => " + e.getMessage(), 409, e.code());
+        }
+      }
+    }
+  }
+
+  //
+  public void testVariousDeletesViaCloudClient() throws Exception {
+    testVariousDeletes(CLOUD_CLIENT);
+  }
+  public void testVariousDeletesViaShard1LeaderClient() throws Exception {
+    testVariousDeletes(S_ONE_LEADER_CLIENT);
+  }
+  public void testVariousDeletesViaShard2LeaderClient() throws Exception {
+    testVariousDeletes(S_TWO_LEADER_CLIENT);
+  }
+  public void testVariousDeletesViaShard1NonLeaderClient() throws Exception {
+    testVariousDeletes(S_ONE_NON_LEADER_CLIENT);
+  }
+  public void testVariousDeletesViaShard2NonLeaderClient() throws Exception {
+    testVariousDeletes(S_TWO_NON_LEADER_CLIENT);
+  }
+  public void testVariousDeletesViaNoCollectionClient() throws Exception {
+    testVariousDeletes(NO_COLLECTION_CLIENT);
+  }
+  
+  protected static void testVariousDeletes(SolrClient client) throws Exception {
+    assertNotNull("client not initialized", client);
+
+    // 2 docs, one on each shard
+    final String docId1 = S_ONE_PRE + "42";
+    final String docId2 = S_TWO_PRE + "666";
+    
+    UpdateResponse rsp = null;
+    
+    // add 1 doc to each shard
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true"),
+                 doc(f("id", docId1), f("foo_i", "2001")),
+                 doc(f("id", docId2), f("foo_i", "1976"))).process(client);
+    assertEquals(0, rsp.getStatus());
+
+    // attempt to delete individual doc id(s) that should fail because of oportunistic concurrency constraints
+    for (String id : new String[] { docId1, docId2 }) {
+      rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                          "commit", "true")).deleteById(id, -1L).process(client);
+      assertEquals(0, rsp.getStatus());
+      assertUpdateTolerantErrors("failed oportunistic concurrent delId="+id, rsp,
+                                 delIErr(id));
+    }
+    
+    // multiple failed deletes from the same shard (via oportunistic concurrent w/ bogus ids)
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true")
+                 ).deleteById(S_ONE_PRE + "X", +1L).deleteById(S_ONE_PRE + "Y", +1L).process(client);
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantErrors("failed oportunistic concurrent delete by id for 2 bogus docs", rsp,
+                               delIErr(S_ONE_PRE + "X"), delIErr(S_ONE_PRE + "Y"));
+    assertQueryDocIds(client, true, docId1, docId2);
+    
+    // multiple failed deletes from the diff shards due to oportunistic concurrency constraints
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true")).deleteById(docId2, -1L).deleteById(docId1, -1L).process(client);
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantErrors("failed oportunistic concurrent delete by id for 2 docs", rsp,
+                               delIErr(docId1), delIErr(docId2));
+    assertQueryDocIds(client, true, docId1, docId2);
+
+    // deleteByQuery using malformed query (fail)
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true")).deleteByQuery("bogus_field:foo").process(client);
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantErrors("failed oportunistic concurrent delete by query", rsp,
+                               delQErr("bogus_field:foo"));
+    assertQueryDocIds(client, true, docId1, docId2);
+
+    // mix 2 deleteByQuery, one malformed (fail), one that doesn't match anything (ok)
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true")
+                 ).deleteByQuery("bogus_field:foo").deleteByQuery("foo_i:23").process(client);
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantErrors("failed oportunistic concurrent delete by query", rsp,
+                               delQErr("bogus_field:foo"));
+    assertQueryDocIds(client, true, docId1, docId2);
+    
+    // mix 2 deleteById using _version_=-1, one for real doc1 (fail), one for bogus id (ok)
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true")
+                 ).deleteById(docId1, -1L).deleteById("bogus", -1L).process(client);
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantErrors("failed oportunistic concurrent delete by id: exists", rsp,
+                               delIErr(docId1));
+    assertQueryDocIds(client, true, docId1, docId2);
+    
+    // mix 2 deleteById using _version_=1, one for real doc1 (ok, deleted), one for bogus id (fail)
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true")
+                 ).deleteById(docId1, +1L).deleteById("bogusId", +1L).process(client);
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantErrors("failed oportunistic concurrent delete by id: bogus", rsp,
+                               delIErr("bogusId"));
+    assertQueryDocIds(client, false, docId1);
+    assertQueryDocIds(client, true, docId2);
+    
+    // mix 2 deleteByQuery, one malformed (fail), one that alctaully removes some docs (ok)
+    assertQueryDocIds(client, true, docId2);
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true")
+                 ).deleteByQuery("bogus_field:foo").deleteByQuery("foo_i:1976").process(client);
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantErrors("failed oportunistic concurrent delete by query", rsp,
+                               delQErr("bogus_field:foo"));
+    assertQueryDocIds(client, false, docId2);
+
+  }
+
+  
+  //
+  public void testVariousAddsViaCloudClient() throws Exception {
+    testVariousAdds(CLOUD_CLIENT);
+  }
+  public void testVariousAddsViaShard1LeaderClient() throws Exception {
+    testVariousAdds(S_ONE_LEADER_CLIENT);
+  }
+  public void testVariousAddsViaShard2LeaderClient() throws Exception {
+    testVariousAdds(S_TWO_LEADER_CLIENT);
+  }
+  public void testVariousAddsViaShard1NonLeaderClient() throws Exception {
+    testVariousAdds(S_ONE_NON_LEADER_CLIENT);
+  }
+  public void testVariousAddsViaShard2NonLeaderClient() throws Exception {
+    testVariousAdds(S_TWO_NON_LEADER_CLIENT);
+  }
+  public void testVariousAddsViaNoCollectionClient() throws Exception {
+    testVariousAdds(NO_COLLECTION_CLIENT);
+  }
+
+  protected static void testVariousAdds(SolrClient client) throws Exception {
+    assertNotNull("client not initialized", client);
+    
+    UpdateResponse rsp = null;
+
+    // 2 docs that are both on shard1, the first one should fail
+    for (int maxErrors : new int[] { -1, 2, 47, 10 }) {
+      // regardless of which of these maxErrors values we use, behavior should be the same...
+      rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                          "maxErrors", ""+maxErrors,
+                          "commit", "true"),
+                   doc(f("id", S_ONE_PRE + "42"), f("foo_i", "bogus_value")),
+                   doc(f("id", S_ONE_PRE + "666"), f("foo_i", "1976"))).process(client);
+      
+      assertEquals(0, rsp.getStatus());
+      assertUpdateTolerantAddErrors("single shard, 1st doc should fail", rsp, S_ONE_PRE + "42");
+      assertEquals(0, client.commit().getStatus());
+      assertQueryDocIds(client, false, S_ONE_PRE + "42");
+      assertQueryDocIds(client, true, S_ONE_PRE + "666");
+
+      // ...only diff should be that we get an accurate report of the effective maxErrors
+      assertEquals(maxErrors, rsp.getResponseHeader().get("maxErrors"));
+    }
+    
+    // 2 docs that are both on shard1, the second one should fail
+    
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-not-set",
+                        "commit", "true"),
+                 doc(f("id", S_ONE_PRE + "55"), f("foo_i", "1976")),
+                 doc(f("id", S_ONE_PRE + "77"), f("foo_i", "bogus_val"))).process(client);
+    
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantAddErrors("single shard, 2nd doc should fail", rsp, S_ONE_PRE + "77");
+    assertQueryDocIds(client, false, S_ONE_PRE + "77");
+    assertQueryDocIds(client, true, S_ONE_PRE + "666", S_ONE_PRE + "55");
+    // since maxErrors is unset, we should get an "unlimited" value back
+    assertEquals(-1, rsp.getResponseHeader().get("maxErrors"));
+
+    // clean slate
+    assertEquals(0, client.deleteByQuery("*:*").getStatus());
+
+    // 2 docs on 2 diff shards, first of which should fail
+    
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true"),
+                 doc(f("id", S_ONE_PRE + "42"), f("foo_i", "bogus_value")),
+                 doc(f("id", S_TWO_PRE + "666"), f("foo_i", "1976"))).process(client);
+    
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantAddErrors("two shards, 1st doc should fail", rsp, S_ONE_PRE + "42");
+    assertEquals(0, client.commit().getStatus());
+    assertQueryDocIds(client, false, S_ONE_PRE + "42");
+    assertQueryDocIds(client, true, S_TWO_PRE + "666");
+    
+    // 2 docs on 2 diff shards, second of which should fail
+
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true"),
+                 doc(f("id", S_ONE_PRE + "55"), f("foo_i", "1976")),
+                 doc(f("id", S_TWO_PRE + "77"), f("foo_i", "bogus_val"))).process(client);
+    
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantAddErrors("two shards, 2nd doc should fail", rsp, S_TWO_PRE + "77");
+    assertQueryDocIds(client, false, S_TWO_PRE + "77");
+    assertQueryDocIds(client, true, S_TWO_PRE + "666", S_ONE_PRE + "55");
+
+    // clean slate
+    assertEquals(0, client.deleteByQuery("*:*").getStatus());
+
+    // many docs from diff shards, 1 from each shard should fail
+    
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true"),
+                 doc(f("id", S_ONE_PRE + "11")),
+                 doc(f("id", S_TWO_PRE + "21")),
+                 doc(f("id", S_ONE_PRE + "12")),
+                 doc(f("id", S_TWO_PRE + "22"), f("foo_i", "bogus_val")),
+                 doc(f("id", S_ONE_PRE + "13")),
+                 doc(f("id", S_TWO_PRE + "23")),
+                 doc(f("id", S_ONE_PRE + "14")),
+                 doc(f("id", S_TWO_PRE + "24")),
+                 doc(f("id", S_ONE_PRE + "15"), f("foo_i", "bogus_val")),
+                 doc(f("id", S_TWO_PRE + "25")),
+                 doc(f("id", S_ONE_PRE + "16")),
+                 doc(f("id", S_TWO_PRE + "26"))).process(client);
+    
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantAddErrors("many docs, 1 from each shard should fail", rsp,
+                                  S_ONE_PRE + "15",
+                                  S_TWO_PRE + "22");
+    assertQueryDocIds(client, false, S_TWO_PRE + "22", S_ONE_PRE + "15");
+    assertQueryDocIds(client, true,
+                      S_ONE_PRE + "11", S_TWO_PRE + "21", S_ONE_PRE + "12",
+                      S_ONE_PRE + "13", S_TWO_PRE + "23", S_ONE_PRE + "14", S_TWO_PRE + "24",
+                      S_TWO_PRE + "25", S_ONE_PRE + "16", S_TWO_PRE + "26");
+
+    // clean slate
+    assertEquals(0, client.deleteByQuery("*:*").getStatus());
+
+    // many docs from diff shards, 1 from each shard should fail and 1 w/o uniqueKey
+    
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true"),
+                 doc(f("id", S_ONE_PRE + "11")),
+                 doc(f("id", S_TWO_PRE + "21")),
+                 doc(f("id", S_ONE_PRE + "12")),
+                 doc(f("id", S_TWO_PRE + "22"), f("foo_i", "bogus_val")),
+                 doc(f("id", S_ONE_PRE + "13")),
+                 doc(f("id", S_TWO_PRE + "23")),
+                 doc(f("foo_i", "42")),          // no "id"
+                 doc(f("id", S_ONE_PRE + "14")),
+                 doc(f("id", S_TWO_PRE + "24")),
+                 doc(f("id", S_ONE_PRE + "15"), f("foo_i", "bogus_val")),
+                 doc(f("id", S_TWO_PRE + "25")),
+                 doc(f("id", S_ONE_PRE + "16")),
+                 doc(f("id", S_TWO_PRE + "26"))).process(client);
+    
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantAddErrors("many docs, 1 from each shard (+ no id) should fail", rsp,
+                                  S_ONE_PRE + "15",
+                                  "(unknown)",
+                                  S_TWO_PRE + "22");
+    assertQueryDocIds(client, false, S_TWO_PRE + "22", S_ONE_PRE + "15");
+    assertQueryDocIds(client, true,
+                      S_ONE_PRE + "11", S_TWO_PRE + "21", S_ONE_PRE + "12",
+                      S_ONE_PRE + "13", S_TWO_PRE + "23", S_ONE_PRE + "14", S_TWO_PRE + "24",
+                      S_TWO_PRE + "25", S_ONE_PRE + "16", S_TWO_PRE + "26");
+
+    // clean slate
+    assertEquals(0, client.deleteByQuery("*:*").getStatus());
+    
+    // many docs from diff shards, more then 10 (total) should fail
+
+    try {
+      rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                          "commit", "true"),
+                   doc(f("id", S_ONE_PRE + "11")),
+                   doc(f("id", S_TWO_PRE + "21"), f("foo_i", "bogus_val")),
+                   doc(f("id", S_ONE_PRE + "12")),
+                   doc(f("id", S_TWO_PRE + "22"), f("foo_i", "bogus_val")),
+                   doc(f("id", S_ONE_PRE + "13")),
+                   doc(f("id", S_TWO_PRE + "23"), f("foo_i", "bogus_val")),
+                   doc(f("id", S_ONE_PRE + "14"), f("foo_i", "bogus_val")),
+                   doc(f("id", S_TWO_PRE + "24")),
+                   doc(f("id", S_ONE_PRE + "15"), f("foo_i", "bogus_val")),
+                   doc(f("id", S_TWO_PRE + "25")),
+                   doc(f("id", S_ONE_PRE + "16"), f("foo_i", "bogus_val")),
+                   doc(f("id", S_TWO_PRE + "26"), f("foo_i", "bogus_val")),
+                   doc(f("id", S_ONE_PRE + "17")),
+                   doc(f("id", S_TWO_PRE + "27")),
+                   doc(f("id", S_ONE_PRE + "18"), f("foo_i", "bogus_val")),
+                   doc(f("id", S_TWO_PRE + "28"), f("foo_i", "bogus_val")),
+                   doc(f("id", S_ONE_PRE + "19"), f("foo_i", "bogus_val")),
+                   doc(f("id", S_TWO_PRE + "29"), f("foo_i", "bogus_val")),
+                   doc(f("id", S_ONE_PRE + "10")), // may be skipped, more then 10 fails
+                   doc(f("id", S_TWO_PRE + "20"))  // may be skipped, more then 10 fails
+                   ).process(client);
+      
+      fail("did not get a top level exception when more then 10 docs failed: " + rsp.toString());
+    } catch (SolrException e) {
+      // we can't make any reliable assertions about the error message, because
+      // it varies based on how the request was routed -- see SOLR-8830
+      assertEquals("not the type of error we were expecting ("+e.code()+"): " + e.toString(),
+                   // NOTE: we always expect a 400 because we know that's what we would get from these types of errors
+                   // on a single node setup -- a 5xx type error isn't something we should have triggered
+                   400, e.code());
+
+      // verify that the Exceptions metadata can tell us what failed.
+      NamedList<String> remoteErrMetadata = e.getMetadata();
+      assertNotNull("no metadata in: " + e.toString(), remoteErrMetadata);
+      Set<ToleratedUpdateError> actualKnownErrs
+        = new LinkedHashSet<ToleratedUpdateError>(remoteErrMetadata.size());
+      int actualKnownErrsCount = 0;
+      for (int i = 0; i < remoteErrMetadata.size(); i++) {
+        ToleratedUpdateError err =
+          ToleratedUpdateError.parseMetadataIfToleratedUpdateError(remoteErrMetadata.getName(i),
+                                                                   remoteErrMetadata.getVal(i));
+        if (null == err) {
+          // some metadata unrelated to this update processor
+          continue;
+        }
+        actualKnownErrsCount++;
+        actualKnownErrs.add(err);
+      }
+      assertEquals("wrong number of errors in metadata: " + remoteErrMetadata.toString(),
+                   11, actualKnownErrsCount);
+      assertEquals("at least one dup error in metadata: " + remoteErrMetadata.toString(),
+                   actualKnownErrsCount, actualKnownErrs.size());
+      for (ToleratedUpdateError err : actualKnownErrs) {
+        assertEquals("only expected type of error is ADD: " + err,
+                     CmdType.ADD, err.getType());
+        assertTrue("failed err msg didn't match expected value: " + err,
+                   err.getMessage().contains("bogus_val"));
+      }
+    }
+    assertEquals(0, client.commit().getStatus()); // need to force since update didn't finish
+    assertQueryDocIds(client, false
+                      // explicitly failed
+                      , S_TWO_PRE + "21", S_TWO_PRE + "22", S_TWO_PRE + "23", S_ONE_PRE + "14"
+                      , S_ONE_PRE + "15", S_ONE_PRE + "16", S_TWO_PRE + "26", S_ONE_PRE + "18"
+                      , S_TWO_PRE + "28", S_ONE_PRE + "19", S_TWO_PRE + "29"
+                      //
+                      // // we can't assert for sure these docs were skipped
+                      // // depending on shard we hit, they may have been added async before errors were exceeded
+                      // , S_ONE_PRE + "10", S_TWO_PRE + "20" // skipped
+                      );
+    assertQueryDocIds(client, true,
+                      S_ONE_PRE + "11", S_ONE_PRE + "12", S_ONE_PRE + "13", S_TWO_PRE + "24",
+                      S_TWO_PRE + "25", S_ONE_PRE + "17", S_TWO_PRE + "27");
+    
+    // clean slate
+    assertEquals(0, client.deleteByQuery("*:*").getStatus());
+    
+    // many docs from diff shards, more then 10 from a single shard (two) should fail
+
+    try {
+      ArrayList<SolrInputDocument> docs = new ArrayList<SolrInputDocument>(30);
+      docs.add(doc(f("id", S_ONE_PRE + "z")));
+      docs.add(doc(f("id", S_TWO_PRE + "z")));
+      docs.add(doc(f("id", S_ONE_PRE + "y")));
+      docs.add(doc(f("id", S_TWO_PRE + "y")));
+      for (int i = 0; i < 11; i++) {
+        docs.add(doc(f("id", S_ONE_PRE + i)));
+        docs.add(doc(f("id", S_TWO_PRE + i), f("foo_i", "bogus_val")));
+      }
+      docs.add(doc(f("id", S_ONE_PRE + "x"))); // may be skipped, more then 10 fails
+      docs.add(doc(f("id", S_TWO_PRE + "x"))); // may be skipped, more then 10 fails
+          
+      rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                          "commit", "true"),
+                   docs.toArray(new SolrInputDocument[docs.size()])).process(client);
+      
+      fail("did not get a top level exception when more then 10 docs failed: " + rsp.toString());
+    } catch (SolrException e) {
+      // we can't make any reliable assertions about the error message, because
+      // it varies based on how the request was routed -- see SOLR-8830
+      assertEquals("not the type of error we were expecting ("+e.code()+"): " + e.toString(),
+                   // NOTE: we always expect a 400 because we know that's what we would get from these types of errors
+                   // on a single node setup -- a 5xx type error isn't something we should have triggered
+                   400, e.code());
+
+      // verify that the Exceptions metadata can tell us what failed.
+      NamedList<String> remoteErrMetadata = e.getMetadata();
+      assertNotNull("no metadata in: " + e.toString(), remoteErrMetadata);
+      Set<ToleratedUpdateError> actualKnownErrs
+        = new LinkedHashSet<ToleratedUpdateError>(remoteErrMetadata.size());
+      int actualKnownErrsCount = 0;
+      for (int i = 0; i < remoteErrMetadata.size(); i++) {
+        ToleratedUpdateError err =
+          ToleratedUpdateError.parseMetadataIfToleratedUpdateError(remoteErrMetadata.getName(i),
+                                                                   remoteErrMetadata.getVal(i));
+        if (null == err) {
+          // some metadata unrelated to this update processor
+          continue;
+        }
+        actualKnownErrsCount++;
+        actualKnownErrs.add(err);
+      }
+      assertEquals("wrong number of errors in metadata: " + remoteErrMetadata.toString(),
+                   11, actualKnownErrsCount);
+      assertEquals("at least one dup error in metadata: " + remoteErrMetadata.toString(),
+                   actualKnownErrsCount, actualKnownErrs.size());
+      for (ToleratedUpdateError err : actualKnownErrs) {
+        assertEquals("only expected type of error is ADD: " + err,
+                     CmdType.ADD, err.getType());
+        assertTrue("failed id had unexpected prefix: " + err,
+                   err.getId().startsWith(S_TWO_PRE));
+        assertTrue("failed err msg didn't match expected value: " + err,
+                   err.getMessage().contains("bogus_val"));
+      }
+           
+    }
+    assertEquals(0, client.commit().getStatus()); // need to force since update didn't finish
+    assertQueryDocIds(client, true
+                      , S_ONE_PRE + "z", S_ONE_PRE + "y", S_TWO_PRE + "z", S_TWO_PRE + "y" // first
+                      //
+                      , S_ONE_PRE + "0", S_ONE_PRE + "1", S_ONE_PRE + "2", S_ONE_PRE + "3", S_ONE_PRE + "4"
+                      , S_ONE_PRE + "5", S_ONE_PRE + "6", S_ONE_PRE + "7", S_ONE_PRE + "8", S_ONE_PRE + "9"
+                      );
+    assertQueryDocIds(client, false
+                      // explicitly failed
+                      , S_TWO_PRE + "0", S_TWO_PRE + "1", S_TWO_PRE + "2", S_TWO_PRE + "3", S_TWO_PRE + "4"
+                      , S_TWO_PRE + "5", S_TWO_PRE + "6", S_TWO_PRE + "7", S_TWO_PRE + "8", S_TWO_PRE + "9"
+                      //
+                      // // we can't assert for sure these docs were skipped
+                      // // depending on shard we hit, they may have been added async before errors were exceeded
+                      // , S_ONE_PRE + "x", S_TWO_PRE + "x", // skipped
+                      );
+
+    // clean slate
+    assertEquals(0, client.deleteByQuery("*:*").getStatus());
+    
+    // many docs from diff shards, more then 10 don't have any uniqueKey specified
+
+    try {
+      ArrayList<SolrInputDocument> docs = new ArrayList<SolrInputDocument>(30);
+      docs.add(doc(f("id", S_ONE_PRE + "z")));
+      docs.add(doc(f("id", S_TWO_PRE + "z")));
+      docs.add(doc(f("id", S_ONE_PRE + "y")));
+      docs.add(doc(f("id", S_TWO_PRE + "y")));
+      for (int i = 0; i < 11; i++) {
+        // no "id" field
+        docs.add(doc(f("foo_i", "" + i)));
+      }
+      docs.add(doc(f("id", S_ONE_PRE + "x"))); // may be skipped, more then 10 fails
+      docs.add(doc(f("id", S_TWO_PRE + "x"))); // may be skipped, more then 10 fails
+          
+      rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                          "commit", "true"),
+                   docs.toArray(new SolrInputDocument[docs.size()])).process(client);
+      
+      fail("did not get a top level exception when more then 10 docs mising uniqueKey: " + rsp.toString());
+    } catch (SolrException e) {
+      // we can't make any reliable assertions about the error message, because
+      // it varies based on how the request was routed -- see SOLR-8830
+      assertEquals("not the type of error we were expecting ("+e.code()+"): " + e.toString(),
+                   // NOTE: we always expect a 400 because we know that's what we would get from these types of errors
+                   // on a single node setup -- a 5xx type error isn't something we should have triggered
+                   400, e.code());
+
+      // verify that the Exceptions metadata can tell us what failed.
+      NamedList<String> remoteErrMetadata = e.getMetadata();
+      assertNotNull("no metadata in: " + e.toString(), remoteErrMetadata);
+      int actualKnownErrsCount = 0;
+      for (int i = 0; i < remoteErrMetadata.size(); i++) {
+        ToleratedUpdateError err =
+          ToleratedUpdateError.parseMetadataIfToleratedUpdateError(remoteErrMetadata.getName(i),
+                                                                   remoteErrMetadata.getVal(i));
+        if (null == err) {
+          // some metadata unrelated to this update processor
+          continue;
+        }
+        actualKnownErrsCount++;
+        assertEquals("only expected type of error is ADD: " + err,
+                     CmdType.ADD, err.getType());
+        assertTrue("failed id didn't match 'unknown': " + err,
+                   err.getId().contains("unknown"));
+      }
+      assertEquals("wrong number of errors in metadata: " + remoteErrMetadata.toString(),
+                   11, actualKnownErrsCount);
+    }
+    assertEquals(0, client.commit().getStatus()); // need to force since update didn't finish
+    assertQueryDocIds(client, true
+                      , S_ONE_PRE + "z", S_ONE_PRE + "y", S_TWO_PRE + "z", S_TWO_PRE + "y" // first
+                      // // we can't assert for sure these docs were skipped or added
+                      // // depending on shard we hit, they may have been added async before errors were exceeded
+                      // , S_ONE_PRE + "x", S_TWO_PRE + "x" // skipped
+                      );
+
+    // clean slate
+    assertEquals(0, client.deleteByQuery("*:*").getStatus());
+    
+    // many docs from diff shards, more then 10 from a single shard (two) should fail but
+    // request should still succeed because of maxErrors=-1 param
+
+    ArrayList<SolrInputDocument> docs = new ArrayList<SolrInputDocument>(30);
+    ArrayList<ExpectedErr> expectedErrs = new ArrayList<ExpectedErr>(30);
+    docs.add(doc(f("id", S_ONE_PRE + "z")));
+    docs.add(doc(f("id", S_TWO_PRE + "z")));
+    docs.add(doc(f("id", S_ONE_PRE + "y")));
+    docs.add(doc(f("id", S_TWO_PRE + "y")));
+    for (int i = 0; i < 11; i++) {
+      docs.add(doc(f("id", S_ONE_PRE + i)));
+      docs.add(doc(f("id", S_TWO_PRE + i), f("foo_i", "bogus_val")));
+      expectedErrs.add(addErr(S_TWO_PRE + i));
+    }
+    docs.add(doc(f("id", S_ONE_PRE + "x"))); 
+    docs.add(doc(f("id", S_TWO_PRE + "x"))); 
+    
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "maxErrors", "-1",
+                        "commit", "true"),
+                 docs.toArray(new SolrInputDocument[docs.size()])).process(client);
+    assertUpdateTolerantErrors("many docs from shard2 fail, but req should succeed", rsp,
+                               expectedErrs.toArray(new ExpectedErr[expectedErrs.size()]));
+    assertQueryDocIds(client, true
+                      , S_ONE_PRE + "z", S_ONE_PRE + "y", S_TWO_PRE + "z", S_TWO_PRE + "y" // first
+                      , S_ONE_PRE + "x", S_TWO_PRE + "x" // later
+                      );
+
+  }
+
+  //
+  public void testAddsMixedWithDeletesViaCloudClient() throws Exception {
+    testAddsMixedWithDeletes(CLOUD_CLIENT);
+  }
+  public void testAddsMixedWithDeletesViaShard1LeaderClient() throws Exception {
+    testAddsMixedWithDeletes(S_ONE_LEADER_CLIENT);
+  }
+  public void testAddsMixedWithDeletesViaShard2LeaderClient() throws Exception {
+    testAddsMixedWithDeletes(S_TWO_LEADER_CLIENT);
+  }
+  public void testAddsMixedWithDeletesViaShard1NonLeaderClient() throws Exception {
+    testAddsMixedWithDeletes(S_ONE_NON_LEADER_CLIENT);
+  }
+  public void testAddsMixedWithDeletesViaShard2NonLeaderClient() throws Exception {
+    testAddsMixedWithDeletes(S_TWO_NON_LEADER_CLIENT);
+  }
+  public void testAddsMixedWithDeletesViaNoCollectionClient() throws Exception {
+    testAddsMixedWithDeletes(NO_COLLECTION_CLIENT);
+  }
+  
+  protected static void testAddsMixedWithDeletes(SolrClient client) throws Exception {
+    assertNotNull("client not initialized", client);
+
+    // 3 doc ids, exactly one on shard1
+    final String docId1  = S_ONE_PRE + "42";
+    final String docId21 = S_TWO_PRE + "42";
+    final String docId22 = S_TWO_PRE + "666";
+    
+    UpdateResponse rsp = null;
+    
+    // add 2 docs, one to each shard
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true"),
+                 doc(f("id", docId1), f("foo_i", "2001")),
+                 doc(f("id", docId21), f("foo_i", "1976"))).process(client);
+    assertEquals(0, rsp.getStatus());
+
+    // add failure on shard2, delete failure on shard1
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true"),
+                 doc(f("id", docId22), f("foo_i", "not_a_num")))
+      .deleteById(docId1, -1L)
+      .process(client);
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantErrors("shard2 add fail, shard1 delI fail", rsp,
+                               delIErr(docId1, "version conflict"),
+                               addErr(docId22,"not_a_num"));
+    
+    // attempt a request containing 4 errors of various types (add, delI, delQ)
+    for (String maxErrors : new String[] {"4", "-1", "100"}) {
+      // for all of these maxErrors values, the overall request should still succeed
+      rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                          "maxErrors", maxErrors,
+                          "commit", "true"),
+                   doc(f("id", docId22), f("foo_i", "bogus_val")))
+        .deleteById(docId1, -1L)
+        .deleteByQuery("malformed:[")
+        .deleteById(docId21, -1L)
+        .process(client);
+      
+      assertEquals(0, rsp.getStatus());
+      assertUpdateTolerantErrors("failed variety of updates", rsp,
+                                 delIErr(docId1, "version conflict"),
+                                 delQErr("malformed:[", "SyntaxError"),
+                                 delIErr(docId21,"version conflict"),
+                                 addErr(docId22,"bogus_val"));
+    }
+    
+    // attempt a request containing 4 errors of various types (add, delI, delQ) .. 1 too many
+    try {
+      rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                          "maxErrors", "3",
+                          "commit", "true"),
+                   doc(f("id", docId22), f("foo_i", "bogus_val")))
+        .deleteById(docId1, -1L)
+        .deleteByQuery("malformed:[")
+        .deleteById(docId21, -1L)
+        .process(client);
+      fail("did not get a top level exception when more then 4 updates failed: " + rsp.toString());
+    } catch (SolrException e) {
+      // we can't make any reliable assertions about the error message, because
+      // it varies based on how the request was routed -- see SOLR-8830
+      
+      // likewise, we can't make a firm(er) assertion about the response code...
+      assertTrue("not the type of error we were expecting ("+e.code()+"): " + e.toString(),
+                 // should be one these 2 depending on order that the async errors were hit...
+                 // on a single node setup -- a 5xx type error isn't something we should have triggered
+                 400 == e.code() || 409 == e.code());
+
+      // verify that the Exceptions metadata can tell us what failed.
+      NamedList<String> remoteErrMetadata = e.getMetadata();
+      assertNotNull("no metadata in: " + e.toString(), remoteErrMetadata);
+      Set<ToleratedUpdateError> actualKnownErrs
+        = new LinkedHashSet<ToleratedUpdateError>(remoteErrMetadata.size());
+      int actualKnownErrsCount = 0;
+      for (int i = 0; i < remoteErrMetadata.size(); i++) {
+        ToleratedUpdateError err =
+          ToleratedUpdateError.parseMetadataIfToleratedUpdateError(remoteErrMetadata.getName(i),
+                                                                   remoteErrMetadata.getVal(i));
+        if (null == err) {
+          // some metadata unrelated to this update processor
+          continue;
+        }
+        actualKnownErrsCount++;
+        actualKnownErrs.add(err);
+      }
+      assertEquals("wrong number of errors in metadata: " + remoteErrMetadata.toString(),
+                   4, actualKnownErrsCount);
+      assertEquals("at least one dup error in metadata: " + remoteErrMetadata.toString(),
+                   actualKnownErrsCount, actualKnownErrs.size());
+    }
+
+    // sanity check our 2 existing docs are still here
+    assertQueryDocIds(client, true, docId1, docId21);
+    assertQueryDocIds(client, false, docId22);
+
+    // tolerate some failures along with a DELQ that should succeed
+    rsp = update(params("update.chain", "tolerant-chain-max-errors-10",
+                        "commit", "true"),
+                 doc(f("id", docId22), f("foo_i", "not_a_num")))
+      .deleteById(docId1, -1L)
+      .deleteByQuery("zot_i:[42 to gibberish...")
+      .deleteByQuery("foo_i:[50 TO 2000}")
+      .process(client);
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantErrors("mix fails with one valid DELQ", rsp,
+                               delIErr(docId1, "version conflict"),
+                               delQErr("zot_i:[42 to gibberish..."),
+                               addErr(docId22,"not_a_num"));
+    // one of our previous docs should have been deleted now
+    assertQueryDocIds(client, true, docId1);
+    assertQueryDocIds(client, false, docId21, docId22);
+                      
+  }
+
+  /**
+   * HACK: Loops over every Jetty instance in the specified MiniSolrCloudCluster to see if they are running,
+   * and sleeps small increments until they all report that they are, or a max num iters is reached
+   * 
+   * (work around for SOLR-8862.  Maybe something like this should be promoted into MiniSolrCloudCluster's 
+   * start() method? or SolrCloudTestCase's configureCluster?)
+   */
+  public static void assertSpinLoopAllJettyAreRunning(MiniSolrCloudCluster cluster) throws InterruptedException {
+    // NOTE: idealy we could use an ExecutorService that tried to open Sockets (with a long timeout)
+    // to each of the jetty instances in parallel w/o any sleeping -- but since they pick their ports
+    // dynamically and don't report them until/unless the server is up, that won't neccessarily do us
+    // any good.
+    final int numServers = cluster.getJettySolrRunners().size();
+    int numRunning = 0;
+    for (int i = 5; 0 <= i; i--) {
+      numRunning = 0;
+      for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+        if (jetty.isRunning()) {
+          numRunning++;
+        }
+      }
+      if (numServers == numRunning) {
+        return;
+      } else if (0 == i) {
+        // give up
+        break;
+      }
+      // the more nodes we're waiting on, the longer we should try to sleep (within reason)
+      Thread.sleep(Math.min((numServers - numRunning) * 100, 1000));
+    }
+    assertEquals("giving up waiting for all jetty instances to be running",
+                 numServers, numRunning);
+  }
+  
+  /** Asserts that the UpdateResponse contains the specified expectedErrs and no others */
+  public static void assertUpdateTolerantErrors(String assertionMsgPrefix,
+                                                UpdateResponse response,
+                                                ExpectedErr... expectedErrs) {
+    @SuppressWarnings("unchecked")
+    List<SimpleOrderedMap<String>> errors = (List<SimpleOrderedMap<String>>)
+      response.getResponseHeader().get("errors");
+    
+    assertNotNull(assertionMsgPrefix + ": Null errors: " + response.toString(), errors);
+    assertEquals(assertionMsgPrefix + ": Num error ids: " + errors.toString(),
+                 expectedErrs.length, errors.size());
+
+    for (SimpleOrderedMap<String> err : errors) {
+      String assertErrPre = assertionMsgPrefix + ": " + err.toString();
+
+      String id = err.get("id");
+      assertNotNull(assertErrPre + " ... null id", id);
+      String type = err.get("type");
+      assertNotNull(assertErrPre + " ... null type", type);
+      String message = err.get("message");
+      assertNotNull(assertErrPre + " ... null message", message);
+
+      // inefficient scan, but good nough for the size of sets we're dealing with
+      boolean found = false;
+      for (ExpectedErr expected : expectedErrs) {
+        if (expected.type.equals(type) && expected.id.equals(id)
+            && (null == expected.msgSubStr || message.contains(expected.msgSubStr))) {
+          found = true;
+          break;
+        }
+      }
+      assertTrue(assertErrPre + " ... unexpected err in: " + response.toString(), found);
+
+    }
+  }
+  
+  /** convinience method when the only type of errors you expect are 'add' errors */
+  public static void assertUpdateTolerantAddErrors(String assertionMsgPrefix,
+                                                   UpdateResponse response,
+                                                   String... errorIdsExpected) {
+    ExpectedErr[] expected = new ExpectedErr[errorIdsExpected.length];
+    for (int i = 0; i < expected.length; i++) {
+      expected[i] = addErr(errorIdsExpected[i]);
+    }
+    assertUpdateTolerantErrors(assertionMsgPrefix, response, expected);
+  }
+
+  /** 
+   * Asserts that the specified document ids do/do-not exist in the index, using both the specified client, 
+   * and the CLOUD_CLIENT 
+   */
+  public static void assertQueryDocIds(SolrClient client, boolean shouldExist, String... ids) throws Exception {
+    for (String id : ids) {
+      assertEquals(client.toString() + " should " + (shouldExist ? "" : "not ") + "find id: " + id,
+                   (shouldExist ? 1 : 0),
+                   CLOUD_CLIENT.query(params("q", "{!term f=id}" + id)).getResults().getNumFound());
+    }
+    if (! CLOUD_CLIENT.equals(client) ) {
+      assertQueryDocIds(CLOUD_CLIENT, shouldExist, ids);
+    }
+  }
+  
+  public static UpdateRequest update(SolrParams params, SolrInputDocument... docs) {
+    UpdateRequest r = new UpdateRequest();
+    r.setParams(new ModifiableSolrParams(params));
+    r.add(Arrays.asList(docs));
+    return r;
+  }
+  
+  public static SolrInputDocument doc(SolrInputField... fields) {
+    SolrInputDocument doc = new SolrInputDocument();
+    for (SolrInputField f : fields) {
+      doc.put(f.getName(), f);
+    }
+    return doc;
+  }
+  
+  public static SolrInputField f(String fieldName, Object... values) {
+    SolrInputField f = new SolrInputField(fieldName);
+    f.setValue(values, 1.0F);
+    return f;
+  }
+
+  /** simple helper struct */
+  public static final class ExpectedErr {
+    final String type;
+    final String id;
+    final String msgSubStr; // ignored if null
+
+    public ExpectedErr(String type, String id, String msgSubStr) {
+      this.type = type;
+      this.id = id;
+      this.msgSubStr = msgSubStr;
+    }
+    public String toString() {
+      return "type=<"+type+">,id=<"+id+">,msgSubStr=<"+msgSubStr+">";
+    }
+  }
+  public static ExpectedErr addErr(String id, String msgSubStr) {
+    return new ExpectedErr("ADD", id, msgSubStr);
+  }
+  public static ExpectedErr delIErr(String id, String msgSubStr) {
+    return new ExpectedErr("DELID", id, msgSubStr);
+  }
+  public static ExpectedErr delQErr(String id, String msgSubStr) {
+    return new ExpectedErr("DELQ", id, msgSubStr);
+  }  
+  public static ExpectedErr addErr(String id) {
+    return addErr(id, null);
+  }
+  public static ExpectedErr delIErr(String id) {
+    return delIErr(id, null);
+  }
+  public static ExpectedErr delQErr(String id) {
+    return delQErr(id, null);
+  }  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b6eacb8/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorRandomCloud.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorRandomCloud.java b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorRandomCloud.java
new file mode 100644
index 0000000..b3f0423
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorRandomCloud.java
@@ -0,0 +1,389 @@
+/*
+ * 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.File;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import static org.apache.solr.cloud.TestTolerantUpdateProcessorCloud.assertUpdateTolerantErrors;
+import static org.apache.solr.cloud.TestTolerantUpdateProcessorCloud.addErr;
+import static org.apache.solr.cloud.TestTolerantUpdateProcessorCloud.delIErr;
+import static org.apache.solr.cloud.TestTolerantUpdateProcessorCloud.delQErr;
+import static org.apache.solr.cloud.TestTolerantUpdateProcessorCloud.f;
+import static org.apache.solr.cloud.TestTolerantUpdateProcessorCloud.update;
+import static org.apache.solr.cloud.TestTolerantUpdateProcessorCloud.ExpectedErr;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import static org.apache.solr.common.params.CursorMarkParams.CURSOR_MARK_PARAM;
+import static org.apache.solr.common.params.CursorMarkParams.CURSOR_MARK_NEXT;
+import static org.apache.solr.common.params.CursorMarkParams.CURSOR_MARK_START;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.SolrInputField;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.ToleratedUpdateError;
+import org.apache.solr.common.ToleratedUpdateError.CmdType;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.util.RevertDefaultThreadHandlerRule;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test of TolerantUpdateProcessor using a randomized MiniSolrCloud.
+ * Reuses some utility methods in {@link TestTolerantUpdateProcessorCloud}
+ *
+ * <p>
+ * <b>NOTE:</b> This test sets up a static instance of MiniSolrCloud with a single collection 
+ * and several clients pointed at specific nodes. These are all re-used across multiple test methods, 
+ * and assumes that the state of the cluster is healthy between tests.
+ * </p>
+ *
+ */
+public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String COLLECTION_NAME = "test_col";
+  
+  /** A basic client for operations at the cloud level, default collection will be set */
+  private static CloudSolrClient CLOUD_CLIENT;
+  /** one HttpSolrClient for each server */
+  private static List<SolrClient> NODE_CLIENTS;
+
+  @BeforeClass
+  private static void createMiniSolrCloudCluster() throws Exception {
+    
+    final String configName = "solrCloudCollectionConfig";
+    final File configDir = new File(TEST_HOME() + File.separator + "collection1" + File.separator + "conf");
+
+    final int numShards = TestUtil.nextInt(random(), 2, TEST_NIGHTLY ? 5 : 3);
+    final int repFactor = TestUtil.nextInt(random(), 2, TEST_NIGHTLY ? 5 : 3);
+    // at least one server won't have any replicas
+    final int numServers = 1 + (numShards * repFactor);
+
+    log.info("Configuring cluster: servers={}, shards={}, repfactor={}", numServers, numShards, repFactor);
+    configureCluster(numServers)
+      .addConfig(configName, configDir.toPath())
+      .configure();
+
+    TestTolerantUpdateProcessorCloud.assertSpinLoopAllJettyAreRunning(cluster);
+    
+    Map<String, String> collectionProperties = new HashMap<>();
+    collectionProperties.put("config", "solrconfig-distrib-update-processor-chains.xml");
+    collectionProperties.put("schema", "schema15.xml"); // string id 
+
+
+    assertNotNull(cluster.createCollection(COLLECTION_NAME, numShards, repFactor,
+                                           configName, null, null, collectionProperties));
+    
+    CLOUD_CLIENT = cluster.getSolrClient();
+    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+
+    NODE_CLIENTS = new ArrayList<SolrClient>(numServers);
+    
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+      URL jettyURL = jetty.getBaseUrl();
+      NODE_CLIENTS.add(new HttpSolrClient(jettyURL.toString() + "/" + COLLECTION_NAME + "/"));
+    }
+    assertEquals(numServers, NODE_CLIENTS.size());
+    
+    ZkStateReader zkStateReader = CLOUD_CLIENT.getZkStateReader();
+    AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION_NAME, zkStateReader, true, true, 330);
+    
+  }
+  
+  @Before
+  private void deleteAllDocs() throws Exception {
+    assertEquals(0, update(params("commit","true")).deleteByQuery("*:*").process(CLOUD_CLIENT).getStatus());
+    assertEquals("index should be empty", 0L, countDocs(CLOUD_CLIENT));
+  }
+  
+  public void testRandomUpdates() throws Exception {
+    final int maxDocId = atLeast(10000);
+    final BitSet expectedDocIds = new BitSet(maxDocId+1);
+    
+    final int numIters = atLeast(50);
+    for (int i = 0; i < numIters; i++) {
+
+      log.info("BEGIN ITER #{}", i);
+      
+      final UpdateRequest req = update(params("maxErrors","-1",
+                                              "update.chain", "tolerant-chain-max-errors-10"));
+      final int numCmds = TestUtil.nextInt(random(), 1, 20);
+      final List<ExpectedErr> expectedErrors = new ArrayList<ExpectedErr>(numCmds);
+      int expectedErrorsCount = 0;
+      // it's ambigious/confusing which order mixed DELQ + ADD  (or ADD and DELI for the same ID)
+      // in the same request wll be processed by various clients, so we keep things simple
+      // and ensure that no single doc Id is affected by more then one command in the same request
+      final BitSet docsAffectedThisRequest = new BitSet(maxDocId+1);
+      for (int cmdIter = 0; cmdIter < numCmds; cmdIter++) {
+        if ((maxDocId / 2) < docsAffectedThisRequest.cardinality()) {
+          // we're already mucking with more then half the docs in the index
+          break;
+        }
+
+        final boolean causeError = random().nextBoolean();
+        if (causeError) {
+          expectedErrorsCount++;
+        }
+        
+        if (random().nextBoolean()) {
+          // add a doc
+          String id = null;
+          SolrInputDocument doc = null;
+          if (causeError && (0 == TestUtil.nextInt(random(), 0, 21))) {
+            doc = doc(f("foo_s","no unique key"));
+            expectedErrors.add(addErr("(unknown)"));
+          } else {
+            final int id_i = randomUnsetBit(random(), docsAffectedThisRequest, maxDocId);
+            docsAffectedThisRequest.set(id_i);
+            id = "id_"+id_i;
+            if (causeError) {
+              expectedErrors.add(addErr(id));
+            } else {
+              expectedDocIds.set(id_i);
+            }
+            final String val = causeError ? "bogus_val" : (""+TestUtil.nextInt(random(), 42, 666));
+            doc = doc(f("id",id),
+                      f("id_i", id_i),
+                      f("foo_i", val));
+          }
+          req.add(doc);
+          log.info("ADD: {} = {}", id, doc);
+        } else {
+          // delete something
+          if (random().nextBoolean()) {
+            // delete by id
+            final int id_i = randomUnsetBit(random(), docsAffectedThisRequest, maxDocId);
+            final String id = "id_"+id_i;
+            final boolean docExists = expectedDocIds.get(id_i);
+            docsAffectedThisRequest.set(id_i);
+            long versionConstraint = docExists ? 1 : -1;
+            if (causeError) {
+              versionConstraint = -1 * versionConstraint;
+              expectedErrors.add(delIErr(id));
+            } else {
+              // if doc exists it will legitimately be deleted
+              expectedDocIds.clear(id_i);
+            }
+            req.deleteById(id, versionConstraint);
+            log.info("DEL: {} = {}", id, causeError ? "ERR" : "OK" );
+          } else {
+            // delete by query
+            final String q;
+            if (causeError) {
+              // even though our DBQ is gibberish that's going to fail, record a docId as affected
+              // so that we don't generate the same random DBQ and get redundent errors
+              // (problematic because of how DUP forwarded DBQs have to have their errors deduped by TUP)
+              final int id_i = randomUnsetBit(random(), docsAffectedThisRequest, maxDocId);
+              docsAffectedThisRequest.set(id_i);
+              q = "foo_i:["+id_i+" TO ....giberish";
+              expectedErrors.add(delQErr(q));
+            } else {
+              // ensure our DBQ is only over a range of docs not already affected
+              // by any other cmds in this request
+              final int rangeAxis = randomUnsetBit(random(), docsAffectedThisRequest, maxDocId);
+              final int loBound = docsAffectedThisRequest.previousSetBit(rangeAxis);
+              final int hiBound = docsAffectedThisRequest.nextSetBit(rangeAxis);
+              final int lo = TestUtil.nextInt(random(), loBound+1, rangeAxis);
+              final int hi = TestUtil.nextInt(random(), rangeAxis,
+                                              // bound might be negative if no set bits above axis
+                                              (hiBound < 0) ? maxDocId : hiBound-1);
+
+              if (lo != hi) {
+                assert lo < hi : "lo="+lo+" hi="+hi;
+                // NOTE: clear & set are exclusive of hi, so we use "}" in range query accordingly
+                q = "id_i:[" + lo + " TO " + hi + "}";
+                expectedDocIds.clear(lo, hi);
+                docsAffectedThisRequest.set(lo, hi);
+              } else {
+                // edge case: special case DBQ of one doc
+                assert (lo == rangeAxis && hi == rangeAxis) : "lo="+lo+" axis="+rangeAxis+" hi="+hi;
+                q = "id_i:[" + lo + " TO " + lo + "]"; // have to be inclusive of both ends
+                expectedDocIds.clear(lo);
+                docsAffectedThisRequest.set(lo);
+              }
+            }
+            req.deleteByQuery(q);
+            log.info("DEL: {}", q);
+          }
+        }
+      }
+      assertEquals("expected error count sanity check: " + req.toString(),
+                   expectedErrorsCount, expectedErrors.size());
+        
+      final SolrClient client = random().nextBoolean() ? CLOUD_CLIENT
+        : NODE_CLIENTS.get(TestUtil.nextInt(random(), 0, NODE_CLIENTS.size()-1));
+      
+      final UpdateResponse rsp = req.process(client);
+      assertUpdateTolerantErrors(client.toString() + " => " + expectedErrors.toString(), rsp,
+                                 expectedErrors.toArray(new ExpectedErr[expectedErrors.size()]));
+               
+      log.info("END ITER #{}, expecting #docs: {}", i, expectedDocIds.cardinality());
+
+      assertEquals("post update commit failed?", 0, CLOUD_CLIENT.commit().getStatus());
+      
+      for (int j = 0; j < 5; j++) {
+        if (expectedDocIds.cardinality() == countDocs(CLOUD_CLIENT)) {
+          break;
+        }
+        log.info("sleeping to give searchers a chance to re-open #" + j);
+        Thread.sleep(200);
+      }
+
+      // check the index contents against our expecationts
+      final BitSet actualDocIds = allDocs(CLOUD_CLIENT, maxDocId);
+      if ( expectedDocIds.cardinality() != actualDocIds.cardinality() ) {
+        log.error("cardinality missmatch: expected {} BUT actual {}",
+                  expectedDocIds.cardinality(),
+                  actualDocIds.cardinality());
+      }
+      final BitSet x = (BitSet) actualDocIds.clone();
+      x.xor(expectedDocIds);
+      for (int b = x.nextSetBit(0); 0 <= b; b = x.nextSetBit(b+1)) {
+        final boolean expectedBit = expectedDocIds.get(b);
+        final boolean actualBit = actualDocIds.get(b);
+        log.error("bit #"+b+" mismatch: expected {} BUT actual {}", expectedBit, actualBit);
+      }
+      assertEquals(x.cardinality() + " mismatched bits",
+                   expectedDocIds.cardinality(), actualDocIds.cardinality());
+    }
+  }
+
+  /** sanity check that randomUnsetBit works as expected 
+   * @see #randomUnsetBit
+   */
+  public void testSanityRandomUnsetBit() {
+    final int max = atLeast(100);
+    BitSet bits = new BitSet(max+1);
+    for (int i = 0; i <= max; i++) {
+      assertFalse("how is bitset already full? iter="+i+" card="+bits.cardinality()+"/max="+max,
+                  bits.cardinality() == max+1);
+      final int nextBit = randomUnsetBit(random(), bits, max);
+      assertTrue("nextBit shouldn't be negative yet: " + nextBit,
+                 0 <= nextBit);
+      assertTrue("nextBit can't exceed max: " + nextBit,
+                 nextBit <= max);
+      assertFalse("expect unset: " + nextBit, bits.get(nextBit));
+      bits.set(nextBit);
+    }
+    
+    assertEquals("why isn't bitset full?", max+1, bits.cardinality());
+
+    final int firstClearBit = bits.nextClearBit(0);
+    assertTrue("why is there a clear bit? = " + firstClearBit,
+               max < firstClearBit);
+    assertEquals("why is a bit set above max?",
+                 -1, bits.nextSetBit(max+1));
+    
+    assertEquals("wrong nextBit at end of all iters", -1,
+                 randomUnsetBit(random(), bits, max));
+    assertEquals("wrong nextBit at redundent end of all iters", -1,
+                 randomUnsetBit(random(), bits, max));
+  }
+  
+  public static SolrInputDocument doc(SolrInputField... fields) {
+    // SolrTestCaseJ4 has same method name, prevents static import from working
+    return TestTolerantUpdateProcessorCloud.doc(fields);
+  }
+
+  /**
+   * Given a BitSet, returns a random bit that is currently false, or -1 if all bits are true.
+   * NOTE: this method is not fair.
+   */
+  public static final int randomUnsetBit(Random r, BitSet bits, final int max) {
+    // NOTE: don't forget, BitSet will grow automatically if not careful
+    if (bits.cardinality() == max+1) {
+      return -1;
+    }
+    final int candidate = TestUtil.nextInt(r, 0, max);
+    if (bits.get(candidate)) {
+      final int lo = bits.previousClearBit(candidate);
+      final int hi = bits.nextClearBit(candidate);
+      if (lo < 0 && max < hi) {
+        fail("how the hell did we not short circut out? card="+bits.cardinality()+"/size="+bits.size());
+      } else if (lo < 0) {
+        return hi;
+      } else if (max < hi) {
+        return lo;
+      } // else...
+      return ((candidate - lo) < (hi - candidate)) ? lo : hi;
+    }
+    return candidate;
+  }
+
+  /** returns the numFound from a *:* query */
+  public static final long countDocs(SolrClient c) throws Exception {
+    return c.query(params("q","*:*","rows","0")).getResults().getNumFound();
+  }
+
+  /** uses a Cursor to iterate over every doc in the index, recording the 'id_i' value in a BitSet */
+  private static final BitSet allDocs(final SolrClient c, final int maxDocIdExpected) throws Exception {
+    BitSet docs = new BitSet(maxDocIdExpected+1);
+    String cursorMark = CURSOR_MARK_START;
+    int docsOnThisPage = Integer.MAX_VALUE;
+    while (0 < docsOnThisPage) {
+      final SolrParams p = params("q","*:*",
+                                  "rows","100",
+                                  // note: not numeric, but we don't actual care about the order
+                                  "sort", "id asc",
+                                  CURSOR_MARK_PARAM, cursorMark);
+      QueryResponse rsp = c.query(p);
+      cursorMark = rsp.getNextCursorMark();
+      docsOnThisPage = 0;
+      for (SolrDocument doc : rsp.getResults()) {
+        docsOnThisPage++;
+        int id_i = ((Integer)doc.get("id_i")).intValue();
+        assertTrue("found id_i bigger then expected "+maxDocIdExpected+": " + id_i,
+                   id_i <= maxDocIdExpected);
+        docs.set(id_i);
+      }
+      cursorMark = rsp.getNextCursorMark();
+    }
+    return docs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b6eacb8/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestBadConfig.java b/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
index 637c975..31361be 100644
--- a/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
+++ b/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
@@ -96,4 +96,9 @@ public class TestBadConfig extends AbstractBadConfigTestBase {
     assertConfigs("bad-solrconfig-unexpected-schema-attribute.xml", "schema-minimal.xml",
                   "Unexpected arg(s): {bogusParam=bogusValue}");
   }
+
+  public void testTolerantUpdateProcessorNoUniqueKey() throws Exception {
+    assertConfigs("solrconfig-tolerant-update-minimal.xml", "schema-minimal.xml",
+                  "requires a schema that includes a uniqueKey field");
+  }
 }