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/02/25 02:58:09 UTC

[49/50] [abbrv] lucene-solr git commit: SOLR-445: replace errors map with List and tweak public so we can differentiate errors of diff types

SOLR-445: replace errors map with List<KnownErr> and tweak public so we can differentiate errors of diff types

for example: an error on deleteById for docId1 vs an error on add for docId1


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

Branch: refs/heads/jira/SOLR-445
Commit: 4ce376fa0f1e6acc84744582ddba6dfe9fd6f11a
Parents: 08bcb76
Author: Chris Hostetter <ho...@apache.org>
Authored: Wed Feb 24 12:10:19 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Wed Feb 24 12:10:19 2016 -0700

----------------------------------------------------------------------
 .../processor/TolerantUpdateProcessor.java      | 89 ++++++++++----------
 .../DistribTolerantUpdateProcessorTest.java     | 27 +++++-
 .../cloud/TestTolerantUpdateProcessorCloud.java | 28 ++++--
 .../processor/TolerantUpdateProcessorTest.java  | 68 +++++++++------
 4 files changed, 134 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4ce376fa/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 60fb387..65b682d 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
@@ -17,6 +17,10 @@
 package org.apache.solr.update.processor;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRefBuilder;
@@ -84,11 +88,10 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
   private ZkController zkController;
 
   /**
-   * Map of errors that occurred in this batch, keyed by unique key. The value is also a Map so that
-   * for each error the output is a key value pair.
+   * Known errors that occurred in this batch, in order encountered (may not be the same as the 
+   * order the commands were originally executed in due to the async distributed updates).
    */
-  // nocommit: why not just SimpleOrderedMap<String> ?
-  private final SimpleOrderedMap<SimpleOrderedMap<String>> errors = new SimpleOrderedMap<>();; 
+  private final List<KnownErr> knownErrors = new ArrayList<KnownErr>();
 
   private final FirstErrTracker firstErrTracker = new FirstErrTracker();
   private final DistribPhase distribPhase;
@@ -106,7 +109,7 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
     
     this.zkController = this.req.getCore().getCoreDescriptor().getCoreContainer().getZkController();
 
-    // nocommit: assert existence of uniqueKey & record for future processAdd+processError calls
+    // nocommit: assert existence of uniqueKey & record for future processAdd+processAddError calls
   }
   
   @Override
@@ -124,8 +127,8 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
       firstErrTracker.caught(t);
       
       if (isLeader || distribPhase.equals(DistribPhase.NONE)) {
-        processError(getPrintableId(id, cmd.getReq().getSchema().getUniqueKeyField()), t);
-        if (errors.size() > maxErrors) {
+        processAddError(getPrintableId(id, cmd.getReq().getSchema().getUniqueKeyField()), t);
+        if (knownErrors.size() > maxErrors) {
           firstErrTracker.throwFirst();
         }
       } else {
@@ -134,33 +137,22 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
     }
   }
   
-  // nocommit: need to subclass & handle & count errors during processDelete
   
-  // nocommit: what about processCommit and other methods? ...
+  // nocommit: what about processCommit and processDelete and other UP methods? ...
   // nocommit: ...at a minimum use firstErrTracker to catch & rethrow so finish can annotate
 
   // nocommit: refactor this method away
-  protected void processError(CharSequence id, Throwable error) {
-    processError(id, error.getMessage());
+  protected void processAddError(CharSequence id, Throwable error) {
+    processAddError(id, error.getMessage());
   }
   
   /** 
    * Logs an error for the given id, and buffers it up to be 
    * included in the response header 
    */
-  protected void processError(CharSequence id, CharSequence error) {
-    // nocommit: refactor so we can track delete errors diff from adds .. are we going to track deletes?
-
-    // nocommit: what's the point of the nested map?
-    SimpleOrderedMap<String> errorMap = new SimpleOrderedMap<>();
-    errorMap.add("message", error.toString());
-    addError(id.toString(), errorMap);
-  }
-  
-  /** Add an error to the list that's going to be returned to the user */
-  protected void addError(String id, SimpleOrderedMap<String> map) {
-    log.debug("Adding error for : {}", id);
-    errors.add(id, map);
+  protected void processAddError(CharSequence id, CharSequence error) {
+  // nocommit: need refactor the KnownErr wrapping up so this method can handle deletes & commits as well
+    knownErrors.add(new KnownErr(CmdType.ADD, id.toString(), error.toString()));
   }
 
   @Override
@@ -203,7 +195,7 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
           }
           
           if (err.type.equals(CmdType.ADD)) { // nocommit: generalize this to work with any CmdType
-            processError(err.id, err.errorValue);
+            processAddError(err.id, err.errorValue);
           } else {
             log.error("found remote error metadata we can't handle key: " + err);
             assert false : "found remote error metadata we can't handle key: " + err;
@@ -213,20 +205,20 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
     }
 
     // good or bad populate the response header
-    if (0 < errors.size()) { // nocommit: we should just always set errors, even if empty?
+    if (0 < knownErrors.size()) { // nocommit: we should just always set errors, even if empty?
       
-      header.add("numErrors", errors.size()); // nocommit: eliminate from response, client can count
-      header.add("errors", errors);
+      header.add("numErrors", knownErrors.size()); // nocommit: eliminate from response, client can count
+      header.add("errors", KnownErr.formatForResponseHeader(knownErrors));
     } else {
       header.add("numErrors", 0); // nocommit: eliminate from response, client can count
     }
 
     // annotate any error that might be thrown (or was already thrown)
-    firstErrTracker.annotate(errors);
+    firstErrTracker.annotate(knownErrors);
 
     // decide if we have hit a situation where we know an error needs to be thrown.
     
-    if ((DistribPhase.TOLEADER.equals(distribPhase) ? 0 : maxErrors) < errors.size()) {
+    if ((DistribPhase.TOLEADER.equals(distribPhase) ? 0 : maxErrors) < knownErrors.size()) {
       // NOTE: even if maxErrors wasn't exceeeded, we need to throw an error when we have any errors if we're
       // a leader that was forwarded to by another node so that the forwarding node knows we encountered some
       // problems and can aggregate the results
@@ -319,7 +311,7 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
      * Annotates the first exception (which may already have been thrown, or be thrown in the future) with 
      * the metadata from this update processor.  For use in {@link TolerantUpdateProcessor#finish}
      */
-    public void annotate(SimpleOrderedMap<SimpleOrderedMap<String>> errors) {
+    public void annotate(List<KnownErr> errors) {
 
       if (null == first) {
         return; // no exception to annotate
@@ -327,15 +319,14 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
       
       assert null != errors : "how do we have an exception to annotate w/o any errors?";
       
-      NamedList<String> errMetadata = first.getMetadata();
-      if (null == errMetadata) { // obnoxious
-        errMetadata = new NamedList<String>();
-        first.setMetadata(errMetadata);
+      NamedList<String> firstErrMetadata = first.getMetadata();
+      if (null == firstErrMetadata) { // obnoxious
+        firstErrMetadata = new NamedList<String>();
+        first.setMetadata(firstErrMetadata);
       }
 
-      for (int i = 0; i < errors.size(); i++) {
-        KnownErr err = new KnownErr(CmdType.ADD, errors.getName(i), errors.getVal(i).get("message"));
-        errMetadata.add(err.getMetadataKey(), err.getMetadataValue());
+      for (KnownErr ke : errors) {
+        firstErrMetadata.add(ke.getMetadataKey(), ke.getMetadataValue());
       }
     }
     
@@ -351,12 +342,22 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
    * Helper class for dealing with SolrException metadata (String) keys 
    */
   public static final class KnownErr {
-    // nocommit: switch metadata key parsing/writting to use this class
-    // nocommit: switch error counting to use instances of this class
     
     private final static String META_PRE =  TolerantUpdateProcessor.class.getName() + "--";
     private final static int META_PRE_LEN = META_PRE.length();
 
+    /** returns a map of simple objects suitable for putting in a SolrQueryResponse */
+    public static List<SimpleOrderedMap<String>> formatForResponseHeader(List<KnownErr> errs) {
+      List<SimpleOrderedMap<String>> result = new ArrayList<>(errs.size());
+      for (KnownErr e : errs) {
+        SimpleOrderedMap<String> entry = new SimpleOrderedMap<String>();
+        entry.add("type", e.type.toString());
+        entry.add("id", e.id);
+        entry.add("message", e.errorValue);
+        result.add(entry);
+      }
+      return result;
+    }
     
     /** returns a KnownErr instance if this metadataKey is one we care about, else null */
     public static KnownErr parseMetadataIfKnownErr(String metadataKey, String metadataVal) {
@@ -372,24 +373,24 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
     public final CmdType type;
     /** may be null depending on type */
     public final String id;
-    public final String errorValue;
+    public final String errorValue; // nocommit: refactor: rename errMessage?
     
     public KnownErr(CmdType type, String id, String errorValue) {
       this.type = type;
       assert null != type;
       
-      this.id = id;
       assert null != id;
+      this.id = id;
       
-      this.errorValue = errorValue;
       assert null != errorValue;
+      this.errorValue = errorValue;
     }
     
     public String getMetadataKey() {
       return META_PRE + type + ":" + id;
     }
     public String getMetadataValue() {
-      return errorValue;
+      return errorValue.toString();
     }
     public String toString() {
       return getMetadataKey() + "=>" + getMetadataValue();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4ce376fa/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 a307382..2c545be 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java
@@ -17,6 +17,11 @@
 package org.apache.solr.cloud;
 
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.HashSet;
+import java.util.Set;
+
 
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -104,6 +109,7 @@ 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,
@@ -112,12 +118,27 @@ public class DistribTolerantUpdateProcessorTest extends AbstractFullDistribZkTes
     newParams.set("update.chain", chain);
     UpdateResponse response = indexDoc(newParams, docs);
     @SuppressWarnings("unchecked")
-    SimpleOrderedMap<Object> errors = (SimpleOrderedMap<Object>) response.getResponseHeader().get("errors");
+    List<SimpleOrderedMap<String>> errors = (List<SimpleOrderedMap<String>>)
+      response.getResponseHeader().get("errors");
     assertNotNull("Null errors in response: " + response.toString(), errors);
+
+    assertEquals("number of errors in response: " + response.toString(), ids.length, errors.size());
+    
+    // nocommit: retire numErrors, we've already checked errors.size()
     assertEquals("Wrong numErrors in response: " + response.toString(),
                  numErrors, response.getResponseHeader().get("numErrors"));
-    for (String id : ids) {
-      assertNotNull("Id " + id + " not found in errors list: " + response.toString(), errors.get(id));
+    
+    Set<String> addErrorIdsExpected = new HashSet<String>(Arrays.asList(ids));
+    
+    for (SimpleOrderedMap<String> err : errors) {
+      // nocommit: support other types
+      assertEquals("nocommit: error type not handled yet",
+                   "ADD", err.get("type"));
+      
+      String id = err.get("id");
+      assertNotNull("null err id", id);
+      assertTrue("unexpected id in errors list: " + response.toString(),
+                 addErrorIdsExpected.contains(id));
     }
     
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4ce376fa/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 7a2bced..0cfcd17 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
@@ -22,7 +22,10 @@ import java.net.URL;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 import org.apache.lucene.util.LuceneTestCase;
@@ -496,21 +499,34 @@ public class TestTolerantUpdateProcessorCloud extends LuceneTestCase {
 
   }
 
-  // nocommit: refactor into multiple methods, some of which can check tolerant deletions as well?
+  // nocommit: redesign so that we can assert errors of diff types besides "add" (ie: deletes) 
   public static void assertUpdateTolerantErrors(String assertionMsgPrefix,
                                                 UpdateResponse response,
                                                 String... errorIdsExpected) {
 
     @SuppressWarnings("unchecked")
-    SimpleOrderedMap<Object> errors = (SimpleOrderedMap<Object>) response.getResponseHeader().get("errors");
+    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(),
                  errorIdsExpected.length, errors.size());
-    for (String id : errorIdsExpected) {
-      assertNotNull(assertionMsgPrefix + ": Id " + id + " not found in errors: " + errors.toString(),
-                    errors.get(id));
+
+    Set<String> addErrorIdsExpected = new HashSet<String>(Arrays.asList(errorIdsExpected));
+
+    for (SimpleOrderedMap<String> err : errors) {
+      String assertErrPre = assertionMsgPrefix + ": " + err.toString();
+      
+      // nocommit: support other types
+      assertEquals(assertErrPre + " ... nocommit: this err type not handled yet",
+                   "ADD", err.get("type"));
+      
+      String id = err.get("id");
+      assertNotNull(assertErrPre + " ... null id", id);
+      assertTrue(assertErrPre + " ... unexpected id", addErrorIdsExpected.contains(id));
+
     }
-    
+
+    // nocommit: retire numErrors, we've already checked errors.size()
     assertEquals(assertionMsgPrefix + ": numErrors: " + response.toString(),
                  errorIdsExpected.length, response.getResponseHeader().get("numErrors"));
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4ce376fa/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 9d0b8dc..08da0c1 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
@@ -21,7 +21,9 @@ import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import javax.xml.xpath.XPathExpressionException;
 
@@ -283,8 +285,8 @@ public class TolerantUpdateProcessorTest extends UpdateProcessorTestBase {
     response = update("tolerant-chain-max-errors-10", adoc("text", "the quick brown fox"));
     assertNull(BaseTestHarness.validateXPath(response, "//int[@name='status']=0",
         "//int[@name='numErrors']=1",
-        "//lst[@name='errors']/lst[@name='(unknown)']",
-        "//lst[@name='errors']/lst[@name='(unknown)']/str[@name='message']/text()='Document is missing mandatory uniqueKey field: id'"));
+        "//arr[@name='errors']/lst/str[@name='id']/text()='(unknown)'",
+        "//arr[@name='errors']/lst/str[@name='message']/text()='Document is missing mandatory uniqueKey field: id'"));
     
     response = update("tolerant-chain-max-errors-10", adoc("text", "the quick brown fox"));
     StringWriter builder = new StringWriter();
@@ -296,26 +298,26 @@ public class TolerantUpdateProcessorTest extends UpdateProcessorTestBase {
     response = update("tolerant-chain-max-errors-10", builder.toString());
     assertNull(BaseTestHarness.validateXPath(response, "//int[@name='status']=0",
         "//int[@name='numErrors']=10",
-        "not(//lst[@name='errors']/lst[@name='0'])",
-        "//lst[@name='errors']/lst[@name='1']",
-        "not(//lst[@name='errors']/lst[@name='2'])",
-        "//lst[@name='errors']/lst[@name='3']",
-        "not(//lst[@name='errors']/lst[@name='4'])",
-        "//lst[@name='errors']/lst[@name='5']",
-        "not(//lst[@name='errors']/lst[@name='6'])",
-        "//lst[@name='errors']/lst[@name='7']",
-        "not(//lst[@name='errors']/lst[@name='8'])",
-        "//lst[@name='errors']/lst[@name='9']",
-        "not(//lst[@name='errors']/lst[@name='10'])",
-        "//lst[@name='errors']/lst[@name='11']",
-        "not(//lst[@name='errors']/lst[@name='12'])",
-        "//lst[@name='errors']/lst[@name='13']",
-        "not(//lst[@name='errors']/lst[@name='14'])",
-        "//lst[@name='errors']/lst[@name='15']",
-        "not(//lst[@name='errors']/lst[@name='16'])",
-        "//lst[@name='errors']/lst[@name='17']",
-        "not(//lst[@name='errors']/lst[@name='18'])",
-        "//lst[@name='errors']/lst[@name='19']"));
+        "not(//arr[@name='errors']/lst/str[@name='id']/text()='0')",
+        "//arr[@name='errors']/lst/str[@name='id']/text()='1'",
+        "not(//arr[@name='errors']/lst/str[@name='id']/text()='2')",
+        "//arr[@name='errors']/lst/str[@name='id']/text()='3'",
+        "not(//arr[@name='errors']/lst/str[@name='id']/text()='4')",
+        "//arr[@name='errors']/lst/str[@name='id']/text()='5'",
+        "not(//arr[@name='errors']/lst/str[@name='id']/text()='6')",
+        "//arr[@name='errors']/lst/str[@name='id']/text()='7'",
+        "not(//arr[@name='errors']/lst/str[@name='id']/text()='8')",
+        "//arr[@name='errors']/lst/str[@name='id']/text()='9'",
+        "not(//arr[@name='errors']/lst/str[@name='id']/text()='10')",
+        "//arr[@name='errors']/lst/str[@name='id']/text()='11'",
+        "not(//arr[@name='errors']/lst/str[@name='id']/text()='12')",
+        "//arr[@name='errors']/lst/str[@name='id']/text()='13'",
+        "not(//arr[@name='errors']/lst/str[@name='id']/text()='14')",
+        "//arr[@name='errors']/lst/str[@name='id']/text()='15'",
+        "not(//arr[@name='errors']/lst/str[@name='id']/text()='16')",
+        "//arr[@name='errors']/lst/str[@name='id']/text()='17'",
+        "not(//arr[@name='errors']/lst/str[@name='id']/text()='18')",
+        "//arr[@name='errors']/lst/str[@name='id']/text()='19'"));
     
   }
 
@@ -368,15 +370,31 @@ public class TolerantUpdateProcessorTest extends UpdateProcessorTestBase {
     }
   }
   
+  // nocommit: redesign so that we can assert errors of diff types besides "add" (ie: deletes) 
   private void assertUSucceedsWithErrors(String chain, final Collection<SolrInputDocument> docs, SolrParams requestParams, int numErrors, String... ids) throws IOException {
     SolrQueryResponse response = add(chain, requestParams, docs);
+    
     @SuppressWarnings("unchecked")
-    SimpleOrderedMap<Object> errors = (SimpleOrderedMap<Object>) response.getResponseHeader().get("errors");
+    List<SimpleOrderedMap<String>> errors = (List<SimpleOrderedMap<String>>)
+      response.getResponseHeader().get("errors");
     assertNotNull(errors);
+
+    assertEquals("number of errors", ids.length, errors.size());
+    
+    // nocommit: retire numErrors, we've already checked errors.size()
     assertEquals(numErrors, response.getResponseHeader().get("numErrors"));
     
-    for(String id:ids) {
-      assertNotNull("Id " + id + " not found in errors list", errors.get(id));
+    Set<String> addErrorIdsExpected = new HashSet<String>(Arrays.asList(ids));
+
+    for (SimpleOrderedMap<String> err : errors) {
+      // nocommit: support other types
+      assertEquals("nocommit: error type not handled yet",
+                   "ADD", err.get("type"));
+      
+      String id = err.get("id");
+      assertNotNull("null err id", id);
+      assertTrue("unexpected id", addErrorIdsExpected.contains(id));
+
     }
   }