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/24 01:03:56 UTC

[1/5] lucene-solr:jira/SOLR-445: SOLR-445: more testing of DBQ mixed with failures

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/SOLR-445 cc2cd23ca -> da3ea40e8


SOLR-445: more testing of DBQ mixed with failures

(trying to staticly recreate a random failure i haven't fully figured out yet)


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

Branch: refs/heads/jira/SOLR-445
Commit: 956d9a592a0a6e9c9d7c8244a4289f4cbf5d5012
Parents: ae22181
Author: Chris Hostetter <ho...@apache.org>
Authored: Tue Mar 22 16:07:10 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Wed Mar 23 15:23:13 2016 -0700

----------------------------------------------------------------------
 .../cloud/TestTolerantUpdateProcessorCloud.java | 25 ++++++++++++++++++--
 1 file changed, 23 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/956d9a59/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
index 7a9e676..6767158 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
@@ -775,7 +775,7 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
     
     UpdateResponse rsp = null;
     
-    // add 2 docs to each shard
+    // 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")),
@@ -856,7 +856,28 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
       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);
+                      
   }
 
   /**


[5/5] lucene-solr:jira/SOLR-445: SOLR-8890: generalized whitelist of param names DUP will use when forwarding requests, usage in SOLR-445

Posted by ho...@apache.org.
SOLR-8890: generalized whitelist of param names DUP will use when forwarding requests, usage in SOLR-445


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

Branch: refs/heads/jira/SOLR-445
Commit: da3ea40e80189c7c2bbd8114a99c72a64262786b
Parents: a468655
Author: Chris Hostetter <ho...@apache.org>
Authored: Wed Mar 23 14:33:29 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Wed Mar 23 17:01:11 2016 -0700

----------------------------------------------------------------------
 .../processor/DistributedUpdateProcessor.java   | 19 ++++++++++++++-----
 .../DistributedUpdateProcessorFactory.java      | 20 ++++++++++++++++++++
 .../TolerantUpdateProcessorFactory.java         |  3 ++-
 3 files changed, 36 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da3ea40e/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 0614c6c..5f4e4f1 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
@@ -92,7 +92,8 @@ import org.slf4j.LoggerFactory;
 // NOT mt-safe... create a new processor for each add thread
 // TODO: we really should not wait for distrib after local? unless a certain replication factor is asked for
 public class DistributedUpdateProcessor extends UpdateRequestProcessor {
-  
+
+  final static String PARAM_WHITELIST_CTX_KEY = DistributedUpdateProcessor.class + "PARAM_WHITELIST_CTX_KEY";
   public static final String DISTRIB_FROM_SHARD = "distrib.from.shard";
   public static final String DISTRIB_FROM_COLLECTION = "distrib.from.collection";
   public static final String DISTRIB_FROM_PARENT = "distrib.from.parent";
@@ -292,6 +293,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
 
     this.req = req;
     
+    // this should always be used - see filterParams
+    DistributedUpdateProcessorFactory.addParamToDistributedRequestWhitelist
+      (this.req, UpdateParams.UPDATE_CHAIN, TEST_DISTRIB_SKIP_SERVERS);
+    
     CoreDescriptor coreDesc = req.getCore().getCoreDescriptor();
     
     this.zkEnabled  = coreDesc.getCoreContainer().isZooKeeperAware();
@@ -1207,12 +1212,16 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     }
   }
 
+  /** @see DistributedUpdateProcessorFactory#addParamToDistributedRequestWhitelist */
   protected ModifiableSolrParams filterParams(SolrParams params) {
     ModifiableSolrParams fparams = new ModifiableSolrParams();
-    passParam(params, fparams, UpdateParams.UPDATE_CHAIN);
-    passParam(params, fparams, TEST_DISTRIB_SKIP_SERVERS);
-    // nocommit ... generalize this...
-    passParam(params, fparams, "maxErrors");
+    
+    Set<String> whitelist = (Set<String>) this.req.getContext().get(PARAM_WHITELIST_CTX_KEY);
+    assert null != whitelist : "whitelist can't be null, constructor adds to it";
+
+    for (String p : whitelist) {
+      passParam(params, fparams, p);
+    }
     return fparams;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da3ea40e/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java
index 4b64dec..6446b1a 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java
@@ -16,6 +16,9 @@
  */
 package org.apache.solr.update.processor;
 
+import java.util.Set;
+import java.util.TreeSet;
+
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
@@ -28,6 +31,23 @@ import org.apache.solr.response.SolrQueryResponse;
 public class DistributedUpdateProcessorFactory 
   extends UpdateRequestProcessorFactory 
   implements DistributingUpdateProcessorFactory {
+
+  /**
+   * By default, the {@link DistributedUpdateProcessor} is extremely conservative in the list of request 
+   * params that will be copied/included when updates are forwarded to other nodes.  This method may be 
+   * used by any {@link UpdateRequestProcessorFactory#getInstance} call to annotate a 
+   * SolrQueryRequest with the names of parameters that should also be forwarded.
+   */
+  public static void addParamToDistributedRequestWhitelist(final SolrQueryRequest req, final String... paramNames) {
+    Set<String> whitelist = (Set<String>) req.getContext().get(DistributedUpdateProcessor.PARAM_WHITELIST_CTX_KEY);
+    if (null == whitelist) {
+      whitelist = new TreeSet<String>();
+      req.getContext().put(DistributedUpdateProcessor.PARAM_WHITELIST_CTX_KEY, whitelist);
+    }
+    for (String p : paramNames) {
+      whitelist.add(p);
+    }
+  }
   
   @Override
   public void init(NamedList args) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/da3ea40e/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessorFactory.java
index e7f5653..8cd3500 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessorFactory.java
@@ -129,7 +129,8 @@ public class TolerantUpdateProcessorFactory extends UpdateRequestProcessorFactor
     if (DistribPhase.FROMLEADER.equals(distribPhase)) {
       return next;
     }
-    
+
+    DistributedUpdateProcessorFactory.addParamToDistributedRequestWhitelist(req, MAX_ERRORS_PARAM);
     int maxErrors = req.getParams().getInt(MAX_ERRORS_PARAM, defaultMaxErrors);
     if (maxErrors < -1) {
       throw new SolrException(ErrorCode.BAD_REQUEST, "'"+MAX_ERRORS_PARAM + "' must either be non-negative, or -1 to indicate 'unlimiited': " + maxErrors);


[2/5] lucene-solr:jira/SOLR-445: SOLR-445: fix silly test bug

Posted by ho...@apache.org.
SOLR-445: fix silly test bug


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

Branch: refs/heads/jira/SOLR-445
Commit: ae22181193dcb24707f7255f0132a2a0a85bf300
Parents: cc2cd23
Author: Chris Hostetter <ho...@apache.org>
Authored: Tue Mar 22 15:26:16 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Wed Mar 23 15:23:13 2016 -0700

----------------------------------------------------------------------
 .../cloud/TestTolerantUpdateProcessorCloud.java | 21 +++++++++++---------
 1 file changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae221811/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
index 236213e..7a9e676 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
@@ -793,8 +793,8 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
                                delIErr(docId1, "version conflict"),
                                addErr(docId22,"not_a_num"));
     
-    // attempt a request containing 5 errors of various types (add, delI, delQ)
-    for (String maxErrors : new String[] {"5", "-1", "100"}) {
+    // 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,
@@ -813,23 +813,26 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
                                  addErr(docId22,"bogus_val"));
     }
     
-    // attempt a request containing 5 errors of various types (add, delI, delQ) .. 1 too many
+    // 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", "4",
+                          "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
-      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());
+      
+      // 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();
@@ -849,7 +852,7 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
         actualKnownErrs.add(err);
       }
       assertEquals("wrong number of errors in metadata: " + remoteErrMetadata.toString(),
-                   5, actualKnownErrsCount);
+                   4, actualKnownErrsCount);
       assertEquals("at least one dup error in metadata: " + remoteErrMetadata.toString(),
                    actualKnownErrsCount, actualKnownErrs.size());
     }


[3/5] lucene-solr:jira/SOLR-445: SOLR-445: harden checks in random test; add isoluated cloud test demonstrating bug random test found; add nocommit hack to DUP to work around test failure for now

Posted by ho...@apache.org.
SOLR-445: harden checks in random test; add isoluated cloud test demonstrating bug random test found; add nocommit hack to DUP to work around test failure for now

(SOLR-8890 to fix a better way)


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

Branch: refs/heads/jira/SOLR-445
Commit: 2622eac2915ee210cfffd1969ef5dd8e2030e5cf
Parents: 956d9a5
Author: Chris Hostetter <ho...@apache.org>
Authored: Wed Mar 23 10:18:42 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Wed Mar 23 17:00:37 2016 -0700

----------------------------------------------------------------------
 .../processor/DistributedUpdateProcessor.java   |  2 +
 .../cloud/TestTolerantUpdateProcessorCloud.java | 32 ++++++++++
 .../TestTolerantUpdateProcessorRandomCloud.java | 61 ++++++++++++++++++--
 3 files changed, 90 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2622eac2/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 0c7836e..0614c6c 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
@@ -1211,6 +1211,8 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     ModifiableSolrParams fparams = new ModifiableSolrParams();
     passParam(params, fparams, UpdateParams.UPDATE_CHAIN);
     passParam(params, fparams, TEST_DISTRIB_SKIP_SERVERS);
+    // nocommit ... generalize this...
+    passParam(params, fparams, "maxErrors");
     return fparams;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2622eac2/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
index 6767158..054c074 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
@@ -743,6 +743,38 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
                       // // 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
+                      );
+
   }
 
   //

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2622eac2/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
index 6e5daeb..8d45707 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorRandomCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorRandomCloud.java
@@ -43,7 +43,11 @@ 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;
@@ -100,7 +104,8 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
     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();
@@ -124,7 +129,8 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
       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);
     
@@ -143,6 +149,8 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
     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);
@@ -246,7 +254,8 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
       final UpdateResponse rsp = req.process(client);
       assertUpdateTolerantErrors(client.toString() + " => " + expectedErrors.toString(), rsp,
                                  expectedErrors.toArray(new ExpectedErr[expectedErrors.size()]));
-      log.info("END: {}", expectedDocIds.cardinality());
+               
+      log.info("END ITER #{}, expecting #docs: {}", i, expectedDocIds.cardinality());
 
       assertEquals("post update commit failed?", 0, CLOUD_CLIENT.commit().getStatus());
       
@@ -257,8 +266,23 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
         log.info("sleeping to give searchers a chance to re-open #" + j);
         Thread.sleep(200);
       }
-      assertEquals("cloud client doc count doesn't match bitself cardinality",
-                   expectedDocIds.cardinality(), countDocs(CLOUD_CLIENT));
+
+      // 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());
     }
   }
 
@@ -324,7 +348,34 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
     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;
+  }
 }


[4/5] lucene-solr:jira/SOLR-445: SOLR-445: randomized testing of the 'doc missing unique key' code path

Posted by ho...@apache.org.
SOLR-445: randomized testing of the 'doc missing unique key' code path


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

Branch: refs/heads/jira/SOLR-445
Commit: a4686553712a0d01dc2d6853038c4cca2caee63f
Parents: 2622eac
Author: Chris Hostetter <ho...@apache.org>
Authored: Wed Mar 23 11:34:21 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Wed Mar 23 17:01:11 2016 -0700

----------------------------------------------------------------------
 .../TestTolerantUpdateProcessorRandomCloud.java | 30 +++++++++++++-------
 1 file changed, 19 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a4686553/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
index 8d45707..b3f0423 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorRandomCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorRandomCloud.java
@@ -173,19 +173,27 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
         
         if (random().nextBoolean()) {
           // add a doc
-          final int id_i = randomUnsetBit(random(), docsAffectedThisRequest, maxDocId);
-          final String id = "id_"+id_i;
-          docsAffectedThisRequest.set(id_i);
-          if (causeError) {
-            expectedErrors.add(addErr(id));
+          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 {
-            expectedDocIds.set(id_i);
-          }
-          final String val = causeError ? "bogus_val" : (""+TestUtil.nextInt(random(), 42, 666));
-          req.add(doc(f("id",id),
+            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)));
-          log.info("ADD: {} = {}", id, val);
+                      f("foo_i", val));
+          }
+          req.add(doc);
+          log.info("ADD: {} = {}", id, doc);
         } else {
           // delete something
           if (random().nextBoolean()) {