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/12 01:27:16 UTC

[35/50] [abbrv] lucene-solr git commit: SOLR-445: cleaned up some low haning nocommits, and added a few more based on a review of the current code

SOLR-445: cleaned up some low haning nocommits, and added a few more based on a review of the current code

in particularly updated a few test comments to note that improvements are blocked by SOLR-8830


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

Branch: refs/heads/jira/SOLR-445
Commit: b24fb0274f5736d1a733f227458ed6fa54865fcc
Parents: c37d5a8
Author: Chris Hostetter <ho...@apache.org>
Authored: Thu Mar 10 15:50:00 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Thu Mar 10 15:50:00 2016 -0700

----------------------------------------------------------------------
 .../processor/DistributedUpdateProcessor.java   | 27 ++++++++++++++++----
 .../processor/TolerantUpdateProcessor.java      | 13 +++-------
 .../TolerantUpdateProcessorFactory.java         |  4 +--
 .../DistribTolerantUpdateProcessorTest.java     | 27 ++++++++------------
 .../cloud/TestTolerantUpdateProcessorCloud.java |  6 ++---
 .../processor/TolerantUpdateProcessorTest.java  | 23 ++++++++---------
 .../solr/client/solrj/impl/CloudSolrClient.java |  5 ++--
 7 files changed, 53 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b24fb027/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 6b650fb..105d3ff 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
@@ -923,10 +923,6 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
 
     
     if (0 < errorsForClient.size()) {
-      // nocommit: slight intentional change here: throwing instead of using setException directly
-      // nocommit: sanity check that doesn't break any other assumptions?
-      //
-      // nocommit: if 1==errorsForClient.size() should we throw it directly? ... would mean changes for catching logic in TolerantUP.finish()
       throw new DistributedUpdatesAsyncException(errorsForClient);
     }
   }
@@ -1706,7 +1702,26 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
       super(buildCode(errors), buildMsg(errors), null);
       this.errors = errors;
 
-      // nocommit: can/should we try to merge the ((SolrException)Error.e).getMetadata() into this.getMetadata() ?
+      // nocommit: the code below is useful for preserving things like "root-error-class"
+      // nocommit: but wreaks havoc on ToleranteUpdateProcessor's exception annotating.
+      //
+      // nocommit: before enabling the code below, we need to make ToleranteUpdateProcessor 
+      // nocommit: smart enough to remove metadata it cares about before adding it (and others) back
+      //
+      // // create a merged copy of the metadata from all wrapped exceptions
+      // NamedList<String> metadata = new NamedList<String>();
+      // for (Error error : errors) {
+      //   if (error.e instanceof SolrException) {
+      //     SolrException e = (SolrException) error.e;
+      //     NamedList<String> eMeta = e.getMetadata();
+      //     if (null != eMeta) {
+      //       metadata.addAll(eMeta);
+      //     }
+      //   }
+      // }
+      // if (0 < metadata.size()) {
+      //   this.setMetadata(metadata);
+      // }
     }
 
     /** Helper method for constructor */
@@ -1721,6 +1736,8 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
         if (result != error.statusCode ) {
           // ...otherwise use sensible default
           return ErrorCode.SERVER_ERROR.code;
+          // nocommit: don't short circut - check them all...
+          // nocommit: ...even if not all same, use 400 if all 4xx, else use 500
         }
       }
       return result;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b24fb027/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessor.java
index 30e5f80..a858e92 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessor.java
@@ -89,7 +89,6 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
   private final int maxErrors;
   
   private final SolrQueryRequest req;
-  private final SolrQueryResponse rsp; // nocommit: needed?
   private ZkController zkController;
 
   /**
@@ -116,7 +115,6 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
   //
   // So as a kludge, we keep track of them for deduping against identical remote failures
   //
-  // :nocommit: probably need to use this for "commit" as well?
   private Set<ToleratedUpdateError> knownDBQErrors = new HashSet<>();
         
   private final FirstErrTracker firstErrTracker = new FirstErrTracker();
@@ -126,7 +124,6 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
     super(next);
     assert maxErrors >= 0;
       
-    this.rsp = rsp; // nocommit: needed?
     header = rsp.getResponseHeader();
     this.maxErrors = maxErrors;
     this.req = req;
@@ -177,16 +174,12 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
   @Override
   public void processDelete(DeleteUpdateCommand cmd) throws IOException {
     
-    // nocommit: do we need special delById => isLeader(id) vs delByQ => isAnyLeader logic?
-      
     try {
       
       super.processDelete(cmd);
       
     } catch (Throwable t) { // nocommit: OOM trap
       firstErrTracker.caught(t);
-
-      // nocommit: do we need isLeader type logic like processAdd ? does processAdd even need it?
       
       ToleratedUpdateError err = new ToleratedUpdateError(cmd.isDeleteById() ? CmdType.DELID : CmdType.DELQ,
                                                           cmd.isDeleteById() ? cmd.id : cmd.query,
@@ -208,8 +201,8 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
   }
 
   
-  // nocommit: what about processCommit and other UpdateProcessor methods?
-  // nocommit: ...at a minimum use firstErrTracker to catch & rethrow so finish can annotate
+  // nocommit: override processCommit and other UpdateProcessor methods
+  // nocommit: ...use firstErrTracker to catch & rethrow so finish can annotate
 
   @Override
   public void finish() throws IOException {
@@ -299,7 +292,7 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
     return field.getType().indexedToReadable(ref, new CharsRefBuilder()).toString();
   }
 
-  // nocommit: javadocs ... also: sanity check this method is even accurate
+  // nocommit: 1) is this method even needed? 2) is this method correct? 3) javadocs
   private boolean isLeader(AddUpdateCommand cmd) {
     if(!cmd.getReq().getCore().getCoreDescriptor().getCoreContainer().isZooKeeperAware())
       return true;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b24fb027/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 45f5fcb..1338908 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
@@ -112,9 +112,9 @@ public class TolerantUpdateProcessorFactory extends UpdateRequestProcessorFactor
       maxErrors = this.defaultMaxErrors;
     }
 
-    // nocommit: support maxErrors < 0 to mean the same as Integer.MAX_VALUE
+    // nocommit: support maxErrors < 0 to mean the same as Integer.MAX_VALUE (add test)
     
-    // nocommit: if (effective) maxErrors==0, then bypass this processor completley?
+    // NOTE: even if 0==maxErrors, we still inject processor into chain so respones has expected header info
     return new TolerantUpdateProcessor(req, rsp, next, maxErrors, distribPhase);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b24fb027/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java
index bebe642..5555e31 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java
@@ -35,6 +35,7 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.junit.Test;
 
+// nocommit: should we just remove this class? does it give us anything not covered by TestTolerantUpdateProcessorCloud?
 public class DistribTolerantUpdateProcessorTest extends AbstractFullDistribZkTestBase {
   
 
@@ -90,15 +91,14 @@ public class DistribTolerantUpdateProcessorTest extends AbstractFullDistribZkTes
       fail("Expecting exception");
     } catch (SolrException e) {
       // we can't make any reliable assertions about the error message, because
-      // it varies based on how the request was routed
-      // nocommit: can we tighten this any more?
+      // it varies based on how the request was routed -- see SOLR-8830
       assertTrue("not the type of error we were expecting: " + e.toString(),
                  400 <= e.code() && e.code() < 500);
     }
-    assertUSucceedsWithErrors("tolerant-chain-max-errors-10",
-                              new SolrInputDocument[]{ invalidDoc,
-                                                      sdoc("id", 4, "text", "the brown fox") },
-                              null, 1, "1");
+    assertAddsSucceedWithErrors("tolerant-chain-max-errors-10",
+                                new SolrInputDocument[]{ invalidDoc,
+                                                         sdoc("id", 4, "text", "the brown fox") },
+                                null, "1");
     commit();
 
     ModifiableSolrParams query = new ModifiableSolrParams();
@@ -109,15 +109,9 @@ public class DistribTolerantUpdateProcessorTest extends AbstractFullDistribZkTes
 
   }
 
-  // nocommit: redesign so that we can assert errors of diff types besides "add" (ie: deletes) 
-  private void assertUSucceedsWithErrors(String chain, SolrInputDocument[] docs,
-                                         SolrParams requestParams,
-                                         int numErrors,
-                                         String... idsShouldFail) throws Exception {
-    
-    // nocommit: retire numErrors from this method sig ... trappy
-    assertEquals("bad test, idsShouldFail.length doesn't match numErrors",
-                 numErrors, idsShouldFail.length);
+  private void assertAddsSucceedWithErrors(String chain, SolrInputDocument[] docs,
+                                            SolrParams requestParams,
+                                            String... idsShouldFail) throws Exception {
     
     ModifiableSolrParams newParams = new ModifiableSolrParams(requestParams);
     newParams.set("update.chain", chain);
@@ -132,8 +126,7 @@ public class DistribTolerantUpdateProcessorTest extends AbstractFullDistribZkTes
     Set<String> addErrorIdsExpected = new HashSet<String>(Arrays.asList(idsShouldFail));
     
     for (SimpleOrderedMap<String> err : errors) {
-      // nocommit: support other types
-      assertEquals("nocommit: error type not handled yet by this method",
+      assertEquals("error type not handled yet by this method",
                    "ADD", err.get("type"));
       
       String id = err.get("id");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b24fb027/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 93200d8..7d8d769 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
@@ -525,8 +525,7 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
       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
-      // nocommit: verify that we can't do an e.getMessage() substring check
+      // 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
@@ -600,8 +599,7 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
       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
-      // nocommit: verify that we can't do an e.getMessage() substring check
+      // 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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b24fb027/solr/core/src/test/org/apache/solr/update/processor/TolerantUpdateProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/TolerantUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/TolerantUpdateProcessorTest.java
index 7470ea0..7f77ad5 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/TolerantUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/TolerantUpdateProcessorTest.java
@@ -99,6 +99,9 @@ public class TolerantUpdateProcessorTest extends UpdateProcessorTestBase {
         ,"//result[@numFound='0']");
     super.tearDown();
   }
+
+  // nocommit: add reflection based test to ensure processor overrides all methods & uses firstErrTracker
+ 
   
   @Test
   public void testValidAdds() throws IOException {
@@ -128,7 +131,7 @@ public class TolerantUpdateProcessorTest extends UpdateProcessorTestBase {
       //expected
       assertTrue(e.getMessage().contains("Document is missing mandatory uniqueKey field"));
     }
-    assertAddsSucceedWithErrors("tolerant-chain-max-errors-10", Arrays.asList(new SolrInputDocument[]{invalidDoc}), null, 1, "(unknown)");
+    assertAddsSucceedWithErrors("tolerant-chain-max-errors-10", Arrays.asList(new SolrInputDocument[]{invalidDoc}), null, "(unknown)");
     
     //a valid doc
     SolrInputDocument validDoc = doc(field("id", 1f, "1"), field("text", 1f, "the quick brown fox"));
@@ -147,7 +150,7 @@ public class TolerantUpdateProcessorTest extends UpdateProcessorTestBase {
         ,"//result[@numFound='0']");
     
     
-    assertAddsSucceedWithErrors("tolerant-chain-max-errors-10", Arrays.asList(new SolrInputDocument[]{invalidDoc, validDoc}), null, 1, "(unknown)");
+    assertAddsSucceedWithErrors("tolerant-chain-max-errors-10", Arrays.asList(new SolrInputDocument[]{invalidDoc, validDoc}), null, "(unknown)");
     assertU(commit());
     
     // verify that the good document made it in. 
@@ -170,7 +173,7 @@ public class TolerantUpdateProcessorTest extends UpdateProcessorTestBase {
     assertQ(req("q","id:3")
         ,"//result[@numFound='0']");
     
-    assertAddsSucceedWithErrors("tolerant-chain-max-errors-10", Arrays.asList(new SolrInputDocument[]{invalidDoc, validDoc}), null, 1, "2");
+    assertAddsSucceedWithErrors("tolerant-chain-max-errors-10", Arrays.asList(new SolrInputDocument[]{invalidDoc, validDoc}), null, "2");
     assertU(commit());
     
     // The valid document was indexed
@@ -187,7 +190,7 @@ public class TolerantUpdateProcessorTest extends UpdateProcessorTestBase {
   public void testMaxErrorsDefault() throws IOException {
     try {
       // by default the TolerantUpdateProcessor accepts all errors, so this batch should succeed with 10 errors.
-      assertAddsSucceedWithErrors("tolerant-chain-max-errors-not-set", docs, null, 10, badIds);
+      assertAddsSucceedWithErrors("tolerant-chain-max-errors-not-set", docs, null, badIds);
     } catch(Exception e) {
       fail("Shouldn't get an exception for this batch: " + e.getMessage());
     }
@@ -200,7 +203,7 @@ public class TolerantUpdateProcessorTest extends UpdateProcessorTestBase {
     ModifiableSolrParams requestParams = new ModifiableSolrParams();
     requestParams.add("maxErrors", "10");
     // still OK
-    assertAddsSucceedWithErrors("tolerant-chain-max-errors-not-set", docs, requestParams, 10, badIds);
+    assertAddsSucceedWithErrors("tolerant-chain-max-errors-not-set", docs, requestParams, badIds);
     assertU(commit());
     assertQ(req("q","*:*")
         ,"//result[@numFound='10']");
@@ -212,7 +215,7 @@ public class TolerantUpdateProcessorTest extends UpdateProcessorTestBase {
     requestParams.add("maxErrors", "5");
     try {
       // should fail
-      assertAddsSucceedWithErrors("tolerant-chain-max-errors-not-set", docs, requestParams, 10, badIds);
+      assertAddsSucceedWithErrors("tolerant-chain-max-errors-not-set", docs, requestParams, badIds);
       fail("Expecting exception");
     } catch (SolrException e) {
       assertTrue(e.getMessage(),
@@ -234,7 +237,7 @@ public class TolerantUpdateProcessorTest extends UpdateProcessorTestBase {
     requestParams.add("maxErrors", "0");
     try {
       // should fail
-      assertAddsSucceedWithErrors("tolerant-chain-max-errors-10", smallBatch, requestParams, 1, "1");
+      assertAddsSucceedWithErrors("tolerant-chain-max-errors-10", smallBatch, requestParams, "1");
       fail("Expecting exception");
     } catch (SolrException e) {
       assertTrue(e.getMessage().contains("ERROR: [doc=1] Error adding field 'weight'='b' msg=For input string: \"b\""));
@@ -351,13 +354,9 @@ public class TolerantUpdateProcessorTest extends UpdateProcessorTestBase {
   
   private void assertAddsSucceedWithErrors(String chain,
                                            final Collection<SolrInputDocument> docs,
-                                           SolrParams requestParams, int numErrors,
+                                           SolrParams requestParams, 
                                            String... idsShouldFail) throws IOException {
 
-    // nocommit: retire numErrors from this method sig ... trappy
-    assertEquals("bad test, idsShouldFail.length doesn't match numErrors",
-                 numErrors, idsShouldFail.length);
-    
     SolrQueryResponse response = add(chain, requestParams, docs);
     
     @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b24fb027/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index febb56f..9a34976 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -778,9 +778,10 @@ public class CloudSolrClient extends SolrClient {
       cheader.add("errors", toleratedErrors);
       if (maxToleratedErrors < toleratedErrors.size()) {
         // cumulative errors are too high, we need to throw a client exception w/correct metadata
-        
+
+        // nocommit: refactor & reuse DistributedUpdatesAsyncException
         NamedList metadata = new NamedList<String>();
-        SolrException toThrow = new SolrException(ErrorCode.BAD_REQUEST, "nocommit: need better msg");
+        SolrException toThrow = new SolrException(ErrorCode.BAD_REQUEST, "nocommit: better msg from DUAE");
         toThrow.setMetadata(metadata);
         for (SimpleOrderedMap<String> err : toleratedErrors) {
           ToleratedUpdateError te = ToleratedUpdateError.parseMap(err);