You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2016/02/19 19:43:43 UTC

[1/3] lucene-solr git commit: SOLR-445: hossman's feb 5 2016 patch

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/SOLR-445 [created] fd12a5b9f


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a9da7ae/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
new file mode 100644
index 0000000..15116cd
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/update/processor/TolerantUpdateProcessorTest.java
@@ -0,0 +1,381 @@
+/*
+ * 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.update.processor;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import javax.xml.xpath.XPathExpressionException;
+
+import org.apache.solr.client.solrj.util.ClientUtils;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.servlet.DirectSolrConnection;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.util.BaseTestHarness;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.xml.sax.SAXException;
+
+public class TolerantUpdateProcessorTest extends UpdateProcessorTestBase {
+  
+  /**
+   * List of valid + invalid documents
+   */
+  private static List<SolrInputDocument> docs = null;
+  /**
+   * IDs of the invalid documents in <code>docs</code>
+   */
+  private static String[] badIds = null;
+  
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig-update-processor-chains.xml", "schema12.xml");
+  }
+  
+  @AfterClass
+  public static void tearDownClass() {
+    docs = null;
+    badIds = null;
+  }
+  
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    //expected exception messages
+    ignoreException("Error adding field");
+    ignoreException("Document is missing mandatory uniqueKey field");
+    if (docs == null) {
+      docs = new ArrayList<>(20);
+      badIds = new String[10];
+      for(int i = 0; i < 10;i++) {
+        // a valid document
+        docs.add(doc(field("id", 1f, String.valueOf(2*i)), field("weight", 1f, i)));
+        // ... and an invalid one
+        docs.add(doc(field("id", 1f, String.valueOf(2*i+1)), field("weight", 1f, "b")));
+        badIds[i] = String.valueOf(2*i+1);
+      }
+    }
+    
+  }
+  
+  @Override
+  public void tearDown() throws Exception {
+    resetExceptionIgnores();
+    assertU(delQ("*:*"));
+    assertU(commit());
+    assertQ(req("q","*:*")
+        ,"//result[@numFound='0']");
+    super.tearDown();
+  }
+  
+  @Test
+  public void testValidAdds() throws IOException {
+    SolrInputDocument validDoc = doc(field("id", 1f, "1"), field("text", 1f, "the quick brown fox"));
+    add("tolerant-chain-max-errors-10", null, validDoc);
+    
+    validDoc = doc(field("id", 1f, "2"), field("text", 1f, "the quick brown fox"));
+    add("tolerant-chain-max-errors-not-set", null, validDoc);
+    
+    assertU(commit());
+    assertQ(req("q","*:*")
+        ,"//result[@numFound='2']");
+    assertQ(req("q","id:1")
+        ,"//result[@numFound='1']");
+    assertQ(req("q","id:2")
+        ,"//result[@numFound='1']");
+  }
+  
+  @Test
+  public void testInvalidAdds() throws IOException {
+    SolrInputDocument invalidDoc = doc(field("text", 1f, "the quick brown fox")); //no id
+    try {
+      // This doc should fail without being tolerant
+      add("not-tolerant", null, invalidDoc);
+      fail("Expecting exception");
+    } catch (Exception e) {
+      //expected
+      assertTrue(e.getMessage().contains("Document is missing mandatory uniqueKey field"));
+    }
+    assertUSucceedsWithErrors("tolerant-chain-max-errors-10", Arrays.asList(new SolrInputDocument[]{invalidDoc}), null, 1, "(unknown)");
+    
+    //a valid doc
+    SolrInputDocument validDoc = doc(field("id", 1f, "1"), field("text", 1f, "the quick brown fox"));
+    
+    try {
+      // This batch should fail without being tolerant
+      add("not-tolerant", null, Arrays.asList(new SolrInputDocument[]{invalidDoc, validDoc}));
+      fail("Expecting exception");
+    } catch (Exception e) {
+      //expected
+      assertTrue(e.getMessage().contains("Document is missing mandatory uniqueKey field"));
+    }
+    
+    assertU(commit());
+    assertQ(req("q","id:1")
+        ,"//result[@numFound='0']");
+    
+    
+    assertUSucceedsWithErrors("tolerant-chain-max-errors-10", Arrays.asList(new SolrInputDocument[]{invalidDoc, validDoc}), null, 1, "(unknown)");
+    assertU(commit());
+    
+    // verify that the good document made it in. 
+    assertQ(req("q","id:1")
+        ,"//result[@numFound='1']");
+    
+    invalidDoc = doc(field("id", 1f, "2"), field("weight", 1f, "aaa"));
+    validDoc = doc(field("id", 1f, "3"), field("weight", 1f, "3"));
+    
+    try {
+      // This batch should fail without being tolerant
+      add("not-tolerant", null, Arrays.asList(new SolrInputDocument[]{invalidDoc, validDoc})); //no id
+      fail("Expecting exception");
+    } catch (Exception e) {
+      //expected
+      assertTrue(e.getMessage().contains("Error adding field"));
+    }
+    
+    assertU(commit());
+    assertQ(req("q","id:3")
+        ,"//result[@numFound='0']");
+    
+    assertUSucceedsWithErrors("tolerant-chain-max-errors-10", Arrays.asList(new SolrInputDocument[]{invalidDoc, validDoc}), null, 1, "2");
+    assertU(commit());
+    
+    // The valid document was indexed
+    assertQ(req("q","id:3")
+        ,"//result[@numFound='1']");
+    
+    // The invalid document was NOT indexed
+    assertQ(req("q","id:2")
+        ,"//result[@numFound='0']");
+    
+  }
+  
+  @Test
+  public void testMaxErrorsDefault() throws IOException {
+    try {
+      // by default the TolerantUpdateProcessor accepts all errors, so this batch should succeed with 10 errors.
+      assertUSucceedsWithErrors("tolerant-chain-max-errors-not-set", docs, null, 10, badIds);
+    } catch(Exception e) {
+      fail("Shouldn't get an exception for this batch: " + e.getMessage());
+    }
+    assertU(commit());
+    assertQ(req("q","*:*")
+        ,"//result[@numFound='10']");
+  }
+  
+  public void testMaxErrorsSucceed() throws IOException {
+    ModifiableSolrParams requestParams = new ModifiableSolrParams();
+    requestParams.add("maxErrors", "10");
+    // still OK
+    assertUSucceedsWithErrors("tolerant-chain-max-errors-not-set", docs, requestParams, 10, badIds);
+    assertU(commit());
+    assertQ(req("q","*:*")
+        ,"//result[@numFound='10']");
+  }
+  
+  @Test
+  public void testMaxErrorsThrowsException() throws IOException {
+    ModifiableSolrParams requestParams = new ModifiableSolrParams();
+    requestParams.add("maxErrors", "5");
+    try {
+      // should fail
+      assertUSucceedsWithErrors("tolerant-chain-max-errors-not-set", docs, requestParams, 10, badIds);
+      fail("Expecting exception");
+    } catch (SolrException e) {
+      assertTrue(e.getMessage(),
+                 e.getMessage().contains("ERROR: [doc=1] Error adding field 'weight'='b' msg=For input string: \"b\""));
+    }
+    //the first good documents made it to the index
+    assertU(commit());
+    assertQ(req("q","*:*")
+        ,"//result[@numFound='6']");
+  }
+
+  // nocommit: need a testMaxErrorsNegative (ie: infinite)
+  
+  @Test
+  public void testMaxErrors0() throws IOException {
+    //make the TolerantUpdateProcessor intolerant
+    List<SolrInputDocument> smallBatch = docs.subList(0, 2);
+    ModifiableSolrParams requestParams = new ModifiableSolrParams();
+    requestParams.add("maxErrors", "0");
+    try {
+      // should fail
+      assertUSucceedsWithErrors("tolerant-chain-max-errors-10", smallBatch, requestParams, 1, "1");
+      fail("Expecting exception");
+    } catch (SolrException e) {
+      assertTrue(e.getMessage().contains("ERROR: [doc=1] Error adding field 'weight'='b' msg=For input string: \"b\""));
+    }
+    //the first good documents made it to the index
+    assertU(commit());
+    assertQ(req("q","*:*")
+        ,"//result[@numFound='1']");
+  }
+  
+  @Test
+  public void testInvalidDelete() throws XPathExpressionException, SAXException {
+    ignoreException("undefined field invalidfield");
+    String response = update("tolerant-chain-max-errors-10", adoc("id", "1", "text", "the quick brown fox"));
+    assertNull(BaseTestHarness.validateXPath(response, "//int[@name='status']=0",
+        "//int[@name='numErrors']=0"));
+    
+    response = update("tolerant-chain-max-errors-10", delQ("invalidfield:1"));
+    assertNull(BaseTestHarness.validateXPath(response, "//int[@name='status']=0",
+        "//int[@name='numErrors']=1",
+        "//lst[@name='errors']/lst[@name='invalidfield:1']",
+        "//lst[@name='errors']/lst[@name='invalidfield:1']/str[@name='message']/text()='undefined field invalidfield'"));
+  }
+  
+  @Test
+  public void testValidDelete() throws XPathExpressionException, SAXException {
+    ignoreException("undefined field invalidfield");
+    String response = update("tolerant-chain-max-errors-10", adoc("id", "1", "text", "the quick brown fox"));
+    assertNull(BaseTestHarness.validateXPath(response, "//int[@name='status']=0",
+        "//int[@name='numErrors']=0"));
+    assertU(commit());
+    assertQ(req("q","*:*")
+        ,"//result[@numFound='1']");
+    
+    response = update("tolerant-chain-max-errors-10", delQ("id:1"));
+    assertNull(BaseTestHarness.validateXPath(response, "//int[@name='status']=0",
+        "//int[@name='numErrors']=0"));
+    assertU(commit());
+    assertQ(req("q","*:*")
+        ,"//result[@numFound='0']");
+  }
+  
+  @Test
+  public void testResponse() throws SAXException, XPathExpressionException, IOException {
+    String response = update("tolerant-chain-max-errors-10", adoc("id", "1", "text", "the quick brown fox"));
+    assertNull(BaseTestHarness.validateXPath(response, "//int[@name='status']=0",
+        "//int[@name='numErrors']=0"));
+    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'"));
+    
+    response = update("tolerant-chain-max-errors-10", adoc("text", "the quick brown fox"));
+    StringWriter builder = new StringWriter();
+    builder.append("<add>");
+    for (SolrInputDocument doc:docs) {
+      ClientUtils.writeXML(doc, builder);
+    }
+    builder.append("</add>");
+    response = update("tolerant-chain-max-errors-10", builder.toString());
+    assertNull(BaseTestHarness.validateXPath(response, "//int[@name='status']=0",
+        "//int[@name='numErrors']=10",
+        "//int[@name='numAdds']=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']"));
+    
+  }
+  
+  public String update(String chain, String xml) {
+    DirectSolrConnection connection = new DirectSolrConnection(h.getCore());
+    SolrRequestHandler handler = h.getCore().getRequestHandler("/update");
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.add("update.chain", chain);
+    try {
+      return connection.request(handler, params, xml);
+    } catch (SolrException e) {
+      throw (SolrException)e;
+    } catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+    }
+  }
+  
+  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");
+    assertNotNull(errors);
+    assertEquals(numErrors, response.getResponseHeader().get("numErrors"));
+    assertEquals(docs.size() - numErrors, response.getResponseHeader().get("numAdds"));
+    
+    for(String id:ids) {
+      assertNotNull("Id " + id + " not found in errors list", errors.get(id));
+    }
+  }
+  
+  protected SolrQueryResponse add(final String chain, SolrParams requestParams, final SolrInputDocument doc) throws IOException {
+    return add(chain, requestParams, Arrays.asList(new SolrInputDocument[]{doc}));
+  }
+  
+  protected SolrQueryResponse add(final String chain, SolrParams requestParams, final Collection<SolrInputDocument> docs) throws IOException {
+    
+    SolrCore core = h.getCore();
+    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
+    assertNotNull("No Chain named: " + chain, pc);
+    
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    rsp.add("responseHeader", new SimpleOrderedMap<Object>());
+    
+    if(requestParams == null) {
+      requestParams = new ModifiableSolrParams();
+    }
+    
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, requestParams);
+    try {
+      UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
+      for(SolrInputDocument doc:docs) {
+        AddUpdateCommand cmd = new AddUpdateCommand(req);
+        cmd.solrDoc = doc;
+        processor.processAdd(cmd);
+      }
+      processor.finish();
+      
+    } finally {
+      req.close();
+    }
+    return rsp;
+  }
+  
+}
\ No newline at end of file


[2/3] lucene-solr git commit: SOLR-445: hossman's feb 5 2016 patch

Posted by ma...@apache.org.
SOLR-445: hossman's feb 5 2016 patch


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

Branch: refs/heads/jira/SOLR-445
Commit: 3a9da7ae576f35e742ec54a72da2d4224066bb63
Parents: c4d3179
Author: markrmiller <ma...@apache.org>
Authored: Fri Feb 19 08:57:13 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Fri Feb 19 13:37:47 2016 -0500

----------------------------------------------------------------------
 .../apache/solr/update/SolrCmdDistributor.java  |  39 +-
 .../processor/DistributedUpdateProcessor.java   | 112 +++-
 .../processor/TolerantUpdateProcessor.java      | 390 +++++++++++++
 .../TolerantUpdateProcessorFactory.java         | 121 ++++
 ...lrconfig-distrib-update-processor-chains.xml |  84 +++
 .../conf/solrconfig-update-processor-chains.xml |  17 +
 .../DistribTolerantUpdateProcessorTest.java     | 146 +++++
 .../cloud/TestTolerantUpdateProcessorCloud.java | 566 +++++++++++++++++++
 .../processor/TolerantUpdateProcessorTest.java  | 381 +++++++++++++
 9 files changed, 1811 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a9da7ae/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java b/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
index 0244b0e..a99952d 100644
--- a/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
+++ b/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
@@ -21,6 +21,7 @@ import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.BinaryResponseParser;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient; // jdoc
 import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.common.SolrException;
@@ -55,7 +56,8 @@ public class SolrCmdDistributor {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   
   private StreamingSolrClients clients;
-  
+  private boolean finished = false; // see finish()
+
   private int retryPause = 500;
   private int maxRetriesOnForward = MAX_RETRIES_ON_FORWARD;
   
@@ -86,6 +88,9 @@ public class SolrCmdDistributor {
   
   public void finish() {    
     try {
+      assert ! finished : "lifecycle sanity check";
+      finished = true;
+      
       blockAndDoRetries();
     } finally {
       clients.shutdown();
@@ -134,7 +139,7 @@ public class SolrCmdDistributor {
             
             SolrException.log(SolrCmdDistributor.log, "forwarding update to "
                 + oldNodeUrl + " failed - retrying ... retries: "
-                + err.req.retries + " " + err.req.cmdString + " params:"
+                + err.req.retries + " " + err.req.cmd.toString() + " params:"
                 + err.req.uReq.getParams() + " rsp:" + rspCode, err.e);
             try {
               Thread.sleep(retryPause);
@@ -183,7 +188,7 @@ public class SolrCmdDistributor {
         uReq.deleteByQuery(cmd.query);
       }
       
-      submit(new Req(cmd.toString(), node, uReq, sync), false);
+      submit(new Req(cmd, node, uReq, sync), false);
     }
   }
   
@@ -196,14 +201,13 @@ public class SolrCmdDistributor {
   }
   
   public void distribAdd(AddUpdateCommand cmd, List<Node> nodes, ModifiableSolrParams params, boolean synchronous, RequestReplicationTracker rrt) throws IOException {  
-    String cmdStr = cmd.toString();
     for (Node node : nodes) {
       UpdateRequest uReq = new UpdateRequest();
       if (cmd.isLastDocInBatch)
         uReq.lastDocInBatch();
       uReq.setParams(params);
       uReq.add(cmd.solrDoc, cmd.commitWithin, cmd.overwrite);
-      submit(new Req(cmdStr, node, uReq, synchronous, rrt, cmd.pollQueueTime), false);
+      submit(new Req(cmd, node, uReq, synchronous, rrt, cmd.pollQueueTime), false);
     }
     
   }
@@ -222,12 +226,12 @@ public class SolrCmdDistributor {
     log.debug("Distrib commit to: {} params: {}", nodes, params);
     
     for (Node node : nodes) {
-      submit(new Req(cmd.toString(), node, uReq, false), true);
+      submit(new Req(cmd, node, uReq, false), true);
     }
     
   }
 
-  private void blockAndDoRetries() {
+  public void blockAndDoRetries() {
     clients.blockUntilFinished();
     
     // wait for any async commits to complete
@@ -268,7 +272,7 @@ public class SolrCmdDistributor {
     if (log.isDebugEnabled()) {
       log.debug("sending update to "
           + req.node.getUrl() + " retry:"
-          + req.retries + " " + req.cmdString + " params:" + req.uReq.getParams());
+          + req.retries + " " + req.cmd + " params:" + req.uReq.getParams());
     }
     
     if (isCommit) {
@@ -310,26 +314,26 @@ public class SolrCmdDistributor {
     public UpdateRequest uReq;
     public int retries;
     public boolean synchronous;
-    public String cmdString;
+    public UpdateCommand cmd;
     public RequestReplicationTracker rfTracker;
     public int pollQueueTime;
 
-    public Req(String cmdString, Node node, UpdateRequest uReq, boolean synchronous) {
-      this(cmdString, node, uReq, synchronous, null, 0);
+    public Req(UpdateCommand cmd, Node node, UpdateRequest uReq, boolean synchronous) {
+      this(cmd, node, uReq, synchronous, null, 0);
     }
     
-    public Req(String cmdString, Node node, UpdateRequest uReq, boolean synchronous, RequestReplicationTracker rfTracker, int pollQueueTime) {
+    public Req(UpdateCommand cmd, Node node, UpdateRequest uReq, boolean synchronous, RequestReplicationTracker rfTracker, int pollQueueTime) {
       this.node = node;
       this.uReq = uReq;
       this.synchronous = synchronous;
-      this.cmdString = cmdString;
+      this.cmd = cmd;
       this.rfTracker = rfTracker;
       this.pollQueueTime = pollQueueTime;
     }
     
     public String toString() {
       StringBuilder sb = new StringBuilder();
-      sb.append("SolrCmdDistributor$Req: cmd=").append(String.valueOf(cmdString));
+      sb.append("SolrCmdDistributor$Req: cmd=").append(cmd.toString());
       sb.append("; node=").append(String.valueOf(node));
       return sb.toString();
     }
@@ -378,6 +382,13 @@ public class SolrCmdDistributor {
   public static class Error {
     public Exception e;
     public int statusCode = -1;
+
+    /**
+     * NOTE: This is the request that happened to be executed when this error was <b>triggered</b> the error, 
+     * but because of how {@link StreamingSolrClients} uses {@link ConcurrentUpdateSolrClient} it might not 
+     * actaully be the request that <b>caused</b> the error -- multiple requests are merged &amp; processed as 
+     * a sequential batch.
+     */
     public Req req;
     
     public String toString() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a9da7ae/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 d0e72db..20de47c 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
@@ -221,6 +221,8 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   
   public static final String COMMIT_END_POINT = "commit_end_point";
   public static final String LOG_REPLAY = "log_replay";
+
+  private boolean finished = false; // see finish()
   
   private final SolrQueryRequest req;
   private final SolrQueryResponse rsp;
@@ -787,38 +789,30 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     cmdDistrib.finish();    
     List<Error> errors = cmdDistrib.getErrors();
     // TODO - we may need to tell about more than one error...
-    
-    // if it's a forward, any fail is a problem - 
-    // otherwise we assume things are fine if we got it locally
-    // until we start allowing min replication param
-    if (errors.size() > 0) {
-      // if one node is a RetryNode, this was a forward request
-      if (errors.get(0).req.node instanceof RetryNode) {
-        rsp.setException(errors.get(0).e);
-      } else {
-        if (log.isWarnEnabled()) {
-          for (Error error : errors) {
-            log.warn("Error sending update to " + error.req.node.getBaseUrl(), error.e);
-          }
-        }
-      }
-      // else
-      // for now we don't error - we assume if it was added locally, we
-      // succeeded 
-    }
-   
-    
-    // if it is not a forward request, for each fail, try to tell them to
-    // recover - the doc was already added locally, so it should have been
-    // legit
 
+    List<Error> errorsForClient = new ArrayList<>(errors.size());
+    
     for (final SolrCmdDistributor.Error error : errors) {
       
       if (error.req.node instanceof RetryNode) {
-        // we don't try to force a leader to recover
-        // when we cannot forward to it
+        // if it's a forward, any fail is a problem - 
+        // otherwise we assume things are fine if we got it locally
+        // until we start allowing min replication param
+        errorsForClient.add(error);
         continue;
       }
+
+      // else...
+      
+      // for now we don't error - we assume if it was added locally, we
+      // succeeded 
+      if (log.isWarnEnabled()) {
+        log.warn("Error sending update to " + error.req.node.getBaseUrl(), error.e);
+      }
+      
+      // Since it is not a forward request, for each fail, try to tell them to
+      // recover - the doc was already added locally, so it should have been
+      // legit
        
       DistribPhase phase =
           DistribPhase.parseParam(error.req.uReq.getParams().get(DISTRIB_UPDATE_PARAM));       
@@ -838,8 +832,8 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
         // let's just fail this request and let the client retry? or just call processAdd again?
         log.error("On "+cloudDesc.getCoreNodeName()+", replica "+replicaUrl+
             " now thinks it is the leader! Failing the request to let the client retry! "+error.e);
-        rsp.setException(error.e);
-        break;
+        errorsForClient.add(error);
+        break; // nocommit: why not continue?
       }
 
       String collection = null;
@@ -924,7 +918,16 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
       rsp.getResponseHeader().add(UpdateRequest.REPFACT, replicationTracker.getAchievedRf());
       rsp.getResponseHeader().add(UpdateRequest.MIN_REPFACT, replicationTracker.minRf);
       replicationTracker = null;
-    }    
+    }
+
+    
+    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);
+    }
   }
 
  
@@ -1373,7 +1376,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
       }
 
       if (someReplicas)  {
-        cmdDistrib.finish();
+        cmdDistrib.blockAndDoRetries();
       }
     }
 
@@ -1618,7 +1621,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
             zkController.getBaseUrl(), req.getCore().getName()));
         if (nodes != null) {
           cmdDistrib.distribCommit(cmd, nodes, params);
-          finish();
+          cmdDistrib.blockAndDoRetries();
         }
       }
     }
@@ -1645,6 +1648,9 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   
   @Override
   public void finish() throws IOException {
+    assert ! finished : "lifecycle sanity check";
+    finished = true;
+    
     if (zkEnabled) doFinish();
     
     if (next != null && nodes == null) next.finish();
@@ -1692,4 +1698,48 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     // definitely not the leader.  Otherwise assume we are.
     return DistribPhase.FROMLEADER != phase;
   }
+
+  public static final class DistributedUpdatesAsyncException extends SolrException {
+    public final List<Error> errors;
+    public DistributedUpdatesAsyncException(List<Error> errors) {
+      super(buildCode(errors), buildMsg(errors), null);
+      this.errors = errors;
+
+      // nocommit: can/should we try to merge the ((SolrException)Error.e).getMetadata() into this.getMetadata() ?
+    }
+
+    /** Helper method for constructor */
+    private static final int buildCode(List<Error> errors) {
+      assert null != errors;
+      assert 0 < errors.size();
+      
+      // if they are all the same, then we use that...
+      int result = errors.get(0).statusCode;
+      for (Error error : errors) {
+        log.trace("REMOTE ERROR: {}", error);
+        if (result != error.statusCode ) {
+          // ...otherwise use sensible default
+          return ErrorCode.SERVER_ERROR.code;
+        }
+      }
+      return result;
+    }
+    
+    /** Helper method for constructor */
+    private static final String buildMsg(List<Error> errors) {
+      assert null != errors;
+      assert 0 < errors.size();
+      
+      if (1 == errors.size()) {
+        return "Async exception during distributed update: " + errors.get(0).e.getMessage();
+      } else {
+        StringBuilder buf = new StringBuilder(errors.size() + " Async exceptions during distributed update: ");
+        for (Error error : errors) {
+          buf.append("\n");
+          buf.append(error.e.getMessage());
+        }
+        return buf.toString();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a9da7ae/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
new file mode 100644
index 0000000..67684e9
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessor.java
@@ -0,0 +1,390 @@
+/*
+ * 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.update.processor;
+
+import java.io.IOException;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CharsRefBuilder;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.params.ShardParams;
+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.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.DeleteUpdateCommand;
+import org.apache.solr.update.SolrCmdDistributor.Error;
+import org.apache.solr.update.UpdateCommand;
+import org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** 
+ * <p> 
+ * Suppresses errors for individual add/delete commands within a batch.
+ * Instead, all errors are logged and the batch continues. The client
+ * will receive a 200 response, but gets a list of errors (keyed by
+ * unique key) unless <code>maxErrors</code> is reached. 
+ * If <code>maxErrors</code> occur, the first exception caught will be re-thrown, 
+ * Solr will respond with 5XX or 4XX (depending on the exception) and
+ * it won't finish processing the batch. This means that the last docs
+ * in the batch may not be added in this case even if they are valid. 
+ * </p>
+ * 
+ * <p>
+ * NOTE: In cloud based collections, this processor expects to <b>NOT</b> be used on {@link DistribPhase#FROMLEADER} 
+ * requests (because any successes that occur locally on the leader are considered successes even if there is some 
+ * subsequent error on a replica).  {@link TolerantUpdateProcessorFactory} will short circut it away in those 
+ * requests.
+ * </p>
+ * 
+ * @see TolerantUpdateProcessorFactory
+ */
+public class TolerantUpdateProcessor extends UpdateRequestProcessor {
+  private static final Logger log = LoggerFactory.getLogger(TolerantUpdateProcessor.class);
+  /**
+   * String to be used as document key in the response if a real ID can't be determined
+   */
+  private static final String UNKNOWN_ID = "(unknown)"; // nocommit: fail hard and fast if no uniqueKey
+
+
+  private final static String ERR_META_PREFIX = java.lang.invoke.MethodHandles.lookup().lookupClass().getName() + "--";
+  /**
+   * Response Header
+   */
+  private final NamedList<Object> header;
+  
+  /**
+   * Number of errors this UpdateRequestProcessor will tolerate. If more then this occur, 
+   * the original exception will be thrown, interrupting the processing of the document
+   * batch
+   */
+  private final int maxErrors;
+  
+  private final SolrQueryRequest req;
+  private final SolrQueryResponse rsp; // nocommit: needed?
+  private ZkController zkController;
+
+  /**
+   * Number of errors that occurred
+   */
+  private int numErrors = 0; // nocmmit: why isn't thisjust errors.size() ? ? ? ? 
+  
+  /**
+   * Number of documents successfully added. 
+   */
+  private int numAdds = 0;
+  
+  /**
+   * 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.
+   */
+  private SimpleOrderedMap<SimpleOrderedMap<String>> errors; // nocommit: why not just SimpleOrderedMap<String> ?
+
+  private final FirstErrTracker firstErrTracker = new FirstErrTracker();
+  private final DistribPhase distribPhase;
+
+  public TolerantUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next, int maxErrors, DistribPhase distribPhase) {
+    super(next);
+    assert maxErrors >= 0;
+      
+    this.rsp = rsp; // nocommit: needed?
+    header = rsp.getResponseHeader();
+    this.maxErrors = maxErrors;
+    this.req = req;
+    this.distribPhase = distribPhase;
+    assert ! DistribPhase.FROMLEADER.equals(distribPhase);
+    
+    this.zkController = this.req.getCore().getCoreDescriptor().getCoreContainer().getZkController();
+
+    // nocommit: assert existence of uniqueKey & record for future processAdd+processError calls
+  }
+  
+  @Override
+  public void processAdd(AddUpdateCommand cmd) throws IOException {
+    boolean isLeader = isLeader(cmd);
+    BytesRef id = null;
+    
+    try {
+      // force AddUpdateCommand to validate+cache the id before proceeding
+      id = cmd.getIndexedId();
+      
+      super.processAdd(cmd);
+      numAdds++;
+
+    } catch (Throwable t) { // nocommit: OOM trap
+      firstErrTracker.caught(t);
+      
+      if (isLeader || distribPhase.equals(DistribPhase.NONE)) {
+        processError(getPrintableId(id, cmd.getReq().getSchema().getUniqueKeyField()), t);
+        if (++numErrors > maxErrors) {
+          firstErrTracker.throwFirst();
+        }
+      } else {
+        firstErrTracker.throwFirst();
+      }
+    }
+  }
+  
+  // nocommit: need to subclass & handle & count errors during processDelete
+  
+  // nocommit: what about processCommit and other 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());
+  }
+  
+  /** 
+   * 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);
+    if (errors == null) {
+      errors = new SimpleOrderedMap<>();
+    }
+    errors.add(id, map);
+  }
+
+  @Override
+  public void finish() throws IOException {
+
+    // even if processAdd threw an error, this.finish() is still called and we might have additional
+    // errors from other remote leaders that we need to check for from the finish method of downstream processors
+    // (like DUP)
+    
+    try {
+      super.finish();
+    } catch (DistributedUpdateProcessor.DistributedUpdatesAsyncException duae) {
+      firstErrTracker.caught(duae);
+
+      // adjust out stats based on the distributed errors
+      for (Error error : duae.errors) {
+        // we can't trust the req info from the Error, because multiple original requests might have been
+        // lumped together
+        //
+        // instead we trust the metadata that the TolerantUpdateProcessor running on the remote node added
+        // to the exception when it failed.
+        if ( ! (error.e instanceof SolrException) ) {
+          log.error("async update exception is not SolrException, no metadata to process", error.e);
+          continue;
+        }
+        SolrException remoteErr = (SolrException) error.e;
+        NamedList<String> remoteErrMetadata = remoteErr.getMetadata();
+
+        if (null == remoteErrMetadata) {
+          log.warn("remote error has no metadata to aggregate: " + remoteErr.getMessage(), remoteErr);
+          continue;
+        }
+        
+        int remoteRealNumErrors = 0;
+        int remoteReportedNumErrors = -1;
+
+        for (int i = 0; i < remoteErrMetadata.size(); i++) {
+          String key = remoteErrMetadata.getName(i);
+          if (! key.startsWith(ERR_META_PREFIX) ) {
+            continue;
+          }
+          String val = remoteErrMetadata.getVal(i);
+          if (key.startsWith("id-", ERR_META_PREFIX.length())) {
+            remoteRealNumErrors++;
+            CharSequence id = key.subSequence(ERR_META_PREFIX.length() + 3, key.length());
+            processError(id, val);
+          } else if (key.equals(ERR_META_PREFIX + "numErrors")) {
+            // nocommit: do we really need this?
+            try {
+              remoteReportedNumErrors = Integer.valueOf(val);
+            } catch (NumberFormatException nfe) {
+              log.error("Ignoring invalid numErrors reported from remote shard: " + val, nfe);
+            }
+          } else if (key.equals(ERR_META_PREFIX + "numAdds")) {
+            // nocommit: nothing very useful we can do with (remote) numAdds?
+            // nocommit: perhaps do some kind of "numAddsConfirmed" ?
+          } else {
+            log.error("found remote error metadata using our prefix but not a key we expect: " + key, remoteErr);
+            assert false;
+          }
+        }
+        assert remoteReportedNumErrors == remoteRealNumErrors : "errors reported doesn't match errors returned";
+        numErrors += remoteRealNumErrors;
+      }
+    }
+
+    // good or bad populate the response header
+    if (errors != null) { // nocommit: we should just always set errors, even if empty?
+      assert errors.size() == numErrors;
+      header.add("numErrors", errors.size());
+      header.add("errors", errors);
+    } else {
+      assert 0 == numErrors;
+      header.add("numErrors", 0);
+    }
+    header.add("numAdds", numAdds);
+
+    // annotate any error that might be thrown (or was already thrown)
+    firstErrTracker.annotate(numAdds, numErrors, errors);
+
+    // decide if we have hit a situation where we know an error needs to be thrown.
+    
+    if ((DistribPhase.TOLEADER.equals(distribPhase) ? 0 : maxErrors) < numErrors) {
+      // 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
+
+      firstErrTracker.throwFirst();
+    }
+  }
+
+  /**
+   * Returns the output of {@link org.apache.solr.schema.FieldType#
+   * indexedToReadable(BytesRef, CharsRef)} of the field
+   * type of the uniqueKey on the {@link BytesRef} passed as parameter.
+   * <code>ref</code> should be the indexed representation of the id and
+   * <code>field</code> should be the uniqueKey schema field. If any of
+   * the two parameters is null this method will return {@link #UNKNOWN_ID}
+   */
+  private String getPrintableId(BytesRef ref, SchemaField field) {
+    if(ref == null || field == null) {
+      return UNKNOWN_ID; // nocommit: fail hard and fast
+    }
+    return field.getType().indexedToReadable(ref, new CharsRefBuilder()).toString();
+  }
+
+  // nocommit: javadocs ... also: sanity check this method is even accurate
+  private boolean isLeader(AddUpdateCommand cmd) {
+    if(!cmd.getReq().getCore().getCoreDescriptor().getCoreContainer().isZooKeeperAware())
+      return true;
+    String collection = cmd.getReq().getCore().getCoreDescriptor().getCollectionName();
+    DocCollection coll = zkController.getClusterState().getCollection(collection);
+
+    SolrParams params = req.getParams();
+    String route = req.getParams().get(ShardParams._ROUTE_);
+    Slice slice = coll.getRouter().getTargetSlice(cmd.getHashableId(), cmd.getSolrInputDocument(), route, params, coll);
+    return slice.getLeader().getName().equals(req.getCore().getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
+
+  }
+
+  /**
+   * Simple helper class for "tracking" any exceptions encountered.
+   * 
+   * Only remembers the "first" exception encountered, and wraps it in a SolrException if needed, so that 
+   * it can later be annotated with the metadata our users expect and re-thrown.
+   *
+   * NOTE: NOT THREAD SAFE
+   */
+  private static final class FirstErrTracker {
+
+    
+    SolrException first = null;
+    boolean thrown = false;
+    
+    public FirstErrTracker() {
+      /* NOOP */
+    }
+    
+    /** 
+     * Call this method immediately anytime an exception is caught from a down stream method -- 
+     * even if you are going to ignore it (for now).  If you plan to rethrow the Exception, use 
+     * {@link #throwFirst} instead.
+     */
+    public void caught(Throwable t) {    // nocommit: switch to just Exception?
+      assert null != t;
+      if (null == first) {
+        if (t instanceof SolrException) {
+          first = (SolrException)t;
+        } else {
+          first = new SolrException(ErrorCode.SERVER_ERROR, "Tolerantly Caught Exception: " + t.getMessage(), t);
+        }
+      }
+    }
+    
+    /** 
+     * Call this method in place of any situation where you would normally (re)throw an exception 
+     * (already passed to the {@link #caught} method because maxErrors was exceeded
+     * is exceed.
+     *
+     * This method will keep a record that this update processor has already thrown the exception, and do 
+     * nothing on future calls, so subsequent update processor methods can update the metadata but won't 
+     * inadvertantly re-throw this (or any other) cascading exception by mistake.
+     */
+    public void throwFirst() throws SolrException {
+      assert null != first : "caught was never called?";
+      if (! thrown) {
+        thrown = true;
+        throw first;
+      }
+    }
+    
+    /** 
+     * 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(int numAdds, int numErrors, SimpleOrderedMap<SimpleOrderedMap<String>> errors) {
+      // nocommit: eliminate numAdds?
+      // nocommit: eliminate numErrors?
+
+      if (null == first) {
+        return; // no exception to annotate
+      }
+      
+      assert null != errors : "how do we have an exception to annotate w/o any errors?";
+      assert errors.size() == numErrors;
+      
+      NamedList<String> errMetadata = first.getMetadata();
+      if (null == errMetadata) { // obnoxious
+        errMetadata = new NamedList<String>();
+        first.setMetadata(errMetadata);
+      }
+      
+      // nocommit: do we really an explicit numErrors here? can't caller just count the errors returned?
+      errMetadata.add(ERR_META_PREFIX + "numErrors", ""+errors.size());
+      for (int i = 0; i < errors.size(); i++) {
+        errMetadata.add(ERR_META_PREFIX + "id-" + errors.getName(i), errors.getVal(i).get("message"));
+      }
+      // nocommit: useless? remove?
+      errMetadata.add(ERR_META_PREFIX + "numAdds", ""+numAdds);
+    }
+    
+    
+    /** The first exception that was thrown (or may be thrown) whose metadata can be annotated. */
+    public SolrException getFirst() {
+      return first;
+    }
+    
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a9da7ae/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
new file mode 100644
index 0000000..983f915
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessorFactory.java
@@ -0,0 +1,121 @@
+package org.apache.solr.update.processor;
+
+/*
+ * 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.
+ */
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
+
+import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
+
+/**
+ * <p> 
+ * Suppresses errors for individual add/delete commands within a batch.
+ * Instead, all errors are logged and the batch continues. The client
+ * will receive a 200 response, but gets a list of errors (keyed by
+ * unique key) unless <code>maxErrors</code> is reached. 
+ * If <code>maxErrors</code> occur, the last exception caught will be re-thrown, 
+ * Solr will respond with 5XX or 4XX (depending on the exception) and
+ * it won't finish processing the batch. This means that the last docs
+ * in the batch may not be added in this case even if they are valid. 
+ * Note that this UpdateRequestProcessor will only catch exceptions that occur 
+ * on later elements in the chain.  
+ * 
+ * </p>
+ * 
+ * <p>
+ * <code>maxErrors</code> is an int value that can be specified in the 
+ * configuration and can also be overridden per request. If unset, it will 
+ * default to <code>Integer.MAX_VALUE</code>
+ * </p>
+ * 
+ * An example configuration would be:
+ * <pre class="prettyprint">
+ * &lt;updateRequestProcessorChain name="tolerant-chain"&gt;
+ *   &lt;processor class="solr.TolerantUpdateProcessorFactory"&gt;
+ *     &lt;int name="maxErrors"&gt;10&lt;/int&gt;
+ *   &lt;/processor&gt;
+ *   &lt;processor class="solr.RunUpdateProcessorFactory" /&gt;
+ * &lt;/updateRequestProcessorChain&gt;
+ * 
+ * </pre>
+ * 
+ * <p>
+ * The maxErrors parameter can be overwritten per request, for example:
+ * </p>
+ * <pre class="prettyprint">
+ * curl http://localhost:8983/update?maxErrors=100 -H "Content-Type: text/xml" -d @myfile.xml
+ * </pre>
+ * 
+ * 
+ * 
+ */
+public class TolerantUpdateProcessorFactory extends UpdateRequestProcessorFactory
+    implements UpdateRequestProcessorFactory.RunAlways {
+
+  // nocommit: make SolrCoreAware and fail fast if no uniqueKey configured
+  
+  /**
+   * Parameter that defines how many errors the UpdateRequestProcessor will tolerate
+   */
+  private final static String MAX_ERRORS_PARAM = "maxErrors";
+  
+  /**
+   * Default maxErrors value that will be use if the value is not set in configuration
+   * or in the request
+   */
+  private Integer defaultMaxErrors = Integer.MAX_VALUE;
+  
+  @SuppressWarnings("rawtypes")
+  @Override
+  public void init( NamedList args ) {
+
+    // nocommit: clean error on invalid type for param ... don't fail stupidly on <str ...>42</str>
+    Object maxErrorsObj = args.get(MAX_ERRORS_PARAM); 
+    if (maxErrorsObj != null) {
+      try {
+        defaultMaxErrors = (Integer)maxErrorsObj;
+      } catch (Exception e) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Unnable to parse maxErrors parameter: " + maxErrorsObj, e);
+      }
+    }
+  }
+  
+  @Override
+  public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next) {
+
+    // short circut if we're a replica processing commands from our leader
+    DistribPhase distribPhase = DistribPhase.parseParam(req.getParams().get(DISTRIB_UPDATE_PARAM));
+    if (DistribPhase.FROMLEADER.equals(distribPhase)) {
+      return next;
+    }
+    
+    Integer maxErrors = req.getParams().getInt(MAX_ERRORS_PARAM);
+    if(maxErrors == null) {
+      maxErrors = this.defaultMaxErrors;
+    }
+
+    // nocommit: support maxErrors < 0 to mean the same as Integer.MAX_VALUE
+    
+    // nocommit: if (effective) maxErrors==0, then bypass this processor completley?
+    return new TolerantUpdateProcessor(req, rsp, next, maxErrors, distribPhase);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a9da7ae/solr/core/src/test-files/solr/collection1/conf/solrconfig-distrib-update-processor-chains.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-distrib-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-distrib-update-processor-chains.xml
new file mode 100644
index 0000000..245f856
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-distrib-update-processor-chains.xml
@@ -0,0 +1,84 @@
+<?xml version="1.0" ?>
+
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<config>
+  <jmx />
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+  
+  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}">
+    <!-- used to keep RAM reqs down for HdfsDirectoryFactory -->
+    <bool name="solr.hdfs.blockcache.enabled">${solr.hdfs.blockcache.enabled:true}</bool>
+    <int name="solr.hdfs.blockcache.blocksperbank">${solr.hdfs.blockcache.blocksperbank:1024}</int>
+    <str name="solr.hdfs.home">${solr.hdfs.home:}</str>
+    <str name="solr.hdfs.confdir">${solr.hdfs.confdir:}</str>
+    <str name="solr.hdfs.blockcache.global">${solr.hdfs.blockcache.global:false}</str>
+  </directoryFactory>
+  
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+
+  <!-- an update processor the explicitly excludes distrib to test
+       clean errors when people attempt atomic updates w/o it
+  -->
+  <updateRequestProcessorChain name="nodistrib" >
+   <processor class="solr.NoOpDistributingUpdateProcessorFactory" />
+   <processor class="solr.RunUpdateProcessorFactory" />
+ </updateRequestProcessorChain>
+
+  <requestHandler name="standard" class="solr.StandardRequestHandler">
+  </requestHandler>
+
+  <requestHandler name="/get" class="solr.RealTimeGetHandler">
+    <lst name="defaults">
+      <str name="omitHeader">true</str>
+    </lst>
+  </requestHandler>
+  
+  <requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy" /> 
+
+  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+    <updateLog>
+      <str name="dir">${solr.ulog.dir:}</str>
+    </updateLog>
+  </updateHandler>
+  
+  <updateRequestProcessorChain name="tolerant-chain-max-errors-10">
+    <processor class="solr.TolerantUpdateProcessorFactory">
+      <int name="maxErrors">10</int>
+    </processor>
+    <processor class="solr.DistributedUpdateProcessorFactory" />
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="tolerant-chain-max-errors-not-set">
+    <processor class="solr.TolerantUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory" />
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+  
+  <updateRequestProcessorChain name="not-tolerant">
+  <processor class="solr.DistributedUpdateProcessorFactory" />
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a9da7ae/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
index d0b5472..bb1cbcf 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
@@ -26,6 +26,7 @@
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
+  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <schemaFactory class="ClassicIndexSchemaFactory"/>
 
@@ -628,4 +629,20 @@
     <processor class="solr.RunUpdateProcessorFactory" />
   </updateRequestProcessorChain>
 
+  <updateRequestProcessorChain name="tolerant-chain-max-errors-10">
+    <processor class="solr.TolerantUpdateProcessorFactory">
+      <int name="maxErrors">10</int>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="tolerant-chain-max-errors-not-set">
+    <processor class="solr.TolerantUpdateProcessorFactory"/>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+  
+  <updateRequestProcessorChain name="not-tolerant">
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
 </config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a9da7ae/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
new file mode 100644
index 0000000..d49adc2
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java
@@ -0,0 +1,146 @@
+package org.apache.solr.cloud;
+
+import java.io.IOException;
+
+import jdk.nashorn.internal.ir.annotations.Ignore;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.junit.Test;
+
+/*
+ * 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.
+ */
+
+public class DistribTolerantUpdateProcessorTest extends AbstractFullDistribZkTestBase {
+  
+
+  @Override
+  protected String getCloudSolrConfig() {
+    return "solrconfig-distrib-update-processor-chains.xml";
+  }
+
+  @ShardsFixed(num = 2)
+  public void testValidAdds() throws Exception {
+    del("*:*");
+    handle.clear();
+    handle.put("maxScore", SKIPVAL);
+    handle.put("QTime", SKIPVAL);
+    handle.put("timestamp", SKIPVAL);
+    SolrInputDocument validDoc = sdoc("id", "1", "text", "the quick brown fox");
+    indexDoc("tolerant-chain-max-errors-10", validDoc);
+    validDoc = sdoc("id", "2", "text", "the quick brown fox");
+    indexDoc("tolerant-chain-max-errors-not-set", validDoc);
+
+    commit();
+    ModifiableSolrParams query = new ModifiableSolrParams();
+    query.add("q", "*:*");
+    QueryResponse queryResponse = queryServer(query);
+    SolrDocumentList solrDocuments = queryResponse.getResults();
+    assertEquals("Expected to match 2 docs", solrDocuments.size(), 2);
+
+    query = new ModifiableSolrParams();
+    query.add("q", "id:1");
+    queryResponse = queryServer(query);
+    solrDocuments = queryResponse.getResults();
+    assertEquals("Expected to match 1 doc", solrDocuments.size(), 1);
+
+    query = new ModifiableSolrParams();
+    query.add("q", "id:2");
+    queryResponse = queryServer(query);
+    solrDocuments = queryResponse.getResults();
+    assertEquals("Expected to match 1 doc", solrDocuments.size(), 1);
+  }
+
+  @Test
+  @ShardsFixed(num = 2)
+  public void testInvalidAdds() throws Exception {
+    //del("*:*");
+    handle.clear();
+    handle.put("maxScore", SKIPVAL);
+    handle.put("QTime", SKIPVAL);
+    handle.put("timestamp", SKIPVAL);
+    SolrInputDocument invalidDoc = sdoc("id", 1, "iind", "b");
+    try {
+      // This doc should fail without being tolerant
+      indexDoc("not-tolerant", invalidDoc);
+      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?
+      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");
+    commit();
+
+    ModifiableSolrParams query = new ModifiableSolrParams();
+    query.add("q", "id:4");
+    QueryResponse queryResponse = queryServer(query);
+    SolrDocumentList solrDocuments = queryResponse.getResults();
+    assertEquals("Expected to match 1 doc", 1, solrDocuments.size());
+
+  }
+
+  private void assertUSucceedsWithErrors(String chain, SolrInputDocument[] docs,
+                                         SolrParams requestParams,
+                                         int numErrors,
+                                         String... ids) throws Exception {
+    ModifiableSolrParams newParams = new ModifiableSolrParams(requestParams);
+    newParams.set("update.chain", chain);
+    UpdateResponse response = indexDoc(newParams, docs);
+    @SuppressWarnings("unchecked")
+    SimpleOrderedMap<Object> errors = (SimpleOrderedMap<Object>) response.getResponseHeader().get("errors");
+    assertNotNull("Null errors in response: " + response.toString(), errors);
+    assertEquals("Wrong numErrors in response: " + response.toString(),
+                 numErrors, response.getResponseHeader().get("numErrors"));
+    assertEquals("numAdds doesn't make sense given input vs numErrors: " + response.toString(),
+                 docs.length - numErrors, response.getResponseHeader().get("numAdds"));
+    for (String id : ids) {
+      assertNotNull("Id " + id + " not found in errors list: " + response.toString(), errors.get(id));
+    }
+    
+  }
+
+  @Override
+  protected void setDistributedParams(ModifiableSolrParams params) {
+    //don't add anything, let SolrCloud handle this
+  }
+
+  private UpdateResponse indexDoc(String updateChain, SolrInputDocument doc) throws IOException, SolrServerException {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("update.chain", updateChain);
+    return indexDoc(params, doc);
+  }
+
+  private UpdateResponse indexDoc(SolrParams params, SolrInputDocument... docs)
+      throws IOException, SolrServerException {
+    int which = random().nextInt(clients.size());
+    SolrClient client = clients.get(which);
+    return add(client, params, docs);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a9da7ae/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..84247a1
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
@@ -0,0 +1,566 @@
+package org.apache.solr.cloud;
+
+/*
+ * 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.
+ */
+
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.solr.SolrTestCaseJ4;
+import static org.apache.solr.SolrTestCaseJ4.params;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.embedded.JettyConfig;
+import org.apache.solr.client.solrj.embedded.JettyConfig.Builder;
+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.client.solrj.response.QueryResponse;
+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.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.SimpleOrderedMap;
+import org.apache.solr.util.RevertDefaultThreadHandlerRule;
+
+import org.junit.ClassRule;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.RuleChain;
+import org.junit.rules.TestRule;
+
+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 assumesthat the state of the cluster is healthy.
+ * </p>
+ *
+ * nocommit: test deletions that fail (and are ignored because of maxErrors) as well...
+ *  - nocommit: DBQ with malformed query
+ *  - nocommit: delete by id with incorrect version (optimistic concurrency fail)
+ *
+ * nocommit: what about shard splitting and "sub shard leaders" ? ...
+ * (no idea if/how that affects things, but i notice lots of logic in DistributedUpdateProcessor along 
+ * the lines of "if (isLeader || isSubShardLeader)" and "if (!isLeader) { if (subShardLeader) {..." 
+ * which makes me worry that we may need explict testing of "tolerant" behavior when updates are routed 
+ * to subshards and then fail?
+ *
+ * nocommit: once these tests are passing reliably, we should also have a fully randomized sibling test...
+ * - randomized # nodes, shards, replicas
+ * - random updates contain rand # of docs with rand # failures to a random client
+ */
+@SuppressSysoutChecks(bugUrl = "Solr logs to JUL")
+public class TestTolerantUpdateProcessorCloud extends LuceneTestCase {
+
+  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";
+  
+  private static MiniSolrCloudCluster SOLR_CLUSTER;
+
+  /** 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!";
+  
+  
+  @Rule
+  public TestRule solrTestRules = RuleChain.outerRule(new SystemPropertiesRestoreRule());
+  
+  @ClassRule
+  public static TestRule solrClassRules = RuleChain.outerRule
+    (new SystemPropertiesRestoreRule()).around(new RevertDefaultThreadHandlerRule());
+
+  @BeforeClass
+  private static void createMiniSolrCloudCluster() throws Exception {
+    Builder jettyConfig = JettyConfig.builder();
+    jettyConfig.waitForLoadingCoresToFinish(null);
+    SOLR_CLUSTER = new MiniSolrCloudCluster(NUM_SERVERS, createTempDir(), jettyConfig.build());
+    
+    String configName = "solrCloudCollectionConfig";
+    File configDir = new File(SolrTestCaseJ4.TEST_HOME() + File.separator + "collection1" + File.separator + "conf");
+    SOLR_CLUSTER.uploadConfigDir(configDir, configName);
+
+    SolrTestCaseJ4.newRandomConfig();
+    
+    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(SOLR_CLUSTER.createCollection(COLLECTION_NAME, NUM_SHARDS, REPLICATION_FACTOR,
+                                                configName, null, null, collectionProperties));
+    
+    CLOUD_CLIENT = SOLR_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 : SOLR_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();
+      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));
+    }
+  }
+  
+  @AfterClass
+  private static void shutdownMiniSolrCloudCluster() throws Exception {
+    SOLR_CLUSTER.shutdown();
+  }
+  
+  @Before
+  private void clearIndex() 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
+    
+    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, "42");
+    }
+
+
+    
+  }
+
+  public void testVariousUpdatesViaCloudClient() throws Exception {
+    testVariousUpdates(CLOUD_CLIENT);
+  }
+  public void testVariousUpdatesViaShard1LeaderClient() throws Exception {
+    testVariousUpdates(S_ONE_LEADER_CLIENT);
+  }
+  public void testVariousUpdatesViaShard2LeaderClient() throws Exception {
+    testVariousUpdates(S_TWO_LEADER_CLIENT);
+  }
+  public void testVariousUpdatesViaShard1NonLeaderClient() throws Exception {
+    testVariousUpdates(S_ONE_NON_LEADER_CLIENT);
+  }
+  public void testVariousUpdatesViaShard2NonLeaderClient() throws Exception {
+    testVariousUpdates(S_TWO_NON_LEADER_CLIENT);
+  }
+  public void testVariousUpdatesViaNoCollectionClient() throws Exception {
+    testVariousUpdates(NO_COLLECTION_CLIENT);
+  }
+
+  protected static void testVariousUpdates(SolrClient client) throws Exception {
+    assertNotNull("client not initialized", client);
+    
+    UpdateResponse rsp = null;
+
+    // 2 docs that are both on shard1, the first one 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_ONE_PRE + "666"), f("foo_i", "1976"))).process(client);
+    
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantErrors("single shard, 1st doc should fail", rsp, 1, S_ONE_PRE + "42");
+    assertEquals(0, client.commit().getStatus());
+    assertQueryDocIds(client, false, S_ONE_PRE + "42");
+    assertQueryDocIds(client, true, S_ONE_PRE + "666");
+           
+    // 2 docs that are both on shard1, the second one 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_ONE_PRE + "77"), f("foo_i", "bogus_val"))).process(client);
+    
+    assertEquals(0, rsp.getStatus());
+    assertUpdateTolerantErrors("single shard, 2nd doc should fail", rsp, 1, S_ONE_PRE + "77");
+    assertQueryDocIds(client, false, S_ONE_PRE + "77");
+    assertQueryDocIds(client, true, S_ONE_PRE + "666", S_ONE_PRE + "55");
+
+    // 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());
+    assertUpdateTolerantErrors("two shards, 1st doc should fail", rsp, 1, 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());
+    assertUpdateTolerantErrors("two shards, 2nd doc should fail", rsp, 1, 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());
+    assertUpdateTolerantErrors("many docs, 1 from each shard should fail", rsp, 10,
+                               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, 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);
+
+      // nocommit: should this really be a top level exception?
+      // nocommit: or should it be an HTTP:200 with the details of what faild in the body?
+      
+      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: can we tighten this any more? substring check?
+      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());
+
+      // nocommit: is there a way to inspect the response body anyway?
+      // nocommit: look for the correct "numAdds" and "errors" ?  .... check e's metatata
+    }
+    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);
+      
+      // nocommit: should this really be a top level exception?
+      // nocommit: or should it be an HTTP:200 with the details of what faild in the body?
+      
+      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: can we tighten this any more? substring check?
+      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());
+
+      // nocommit: is there a way to inspect the response body anyway?
+      // nocommit: look for the correct "numAdds" and "errors" ?  .... check e's metatata
+    }
+    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
+                      );
+
+  }
+
+  // nocommit: refactor into multiple methods, some of which can check tolerant deletions as well?
+  public static void assertUpdateTolerantErrors(String assertionMsgPrefix,
+                                                UpdateResponse response,
+                                                int numAddsExpected,
+                                                String... errorIdsExpected) {
+
+    // nocommit: numAdds seems virtually impossible to get right in distrib/async mode...
+    // nocommit: for now make no assertions about it
+    //
+    // nocommit: perhaps it should be numAddsAttempted and we should assert numAddsAttempted < maxNumAddsExpected
+    // nocommit: ...at least that way we can assert that requests failed fast when we expected them too?
+    // nocommit: (even that's hard to be sure when we have the same test logic being used on diff clients  -- fail fast to S1 leader will be slow async failure to S2 leader)
+    //
+    //
+    // assertEquals(assertionMsgPrefix + ": numAdds: " + response.toString(),
+    //              numAddsExpected, response.getResponseHeader().get("numAdds"));
+    
+    @SuppressWarnings("unchecked")
+    SimpleOrderedMap<Object> errors = (SimpleOrderedMap<Object>) 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));
+    }
+    
+    assertEquals(assertionMsgPrefix + ": numErrors: " + response.toString(),
+                 errorIdsExpected.length, response.getResponseHeader().get("numErrors"));
+  }
+
+  /** 
+   * 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;
+  }
+
+}


[3/3] lucene-solr git commit: SOLR-445: some cleanup

Posted by ma...@apache.org.
SOLR-445: some cleanup


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

Branch: refs/heads/jira/SOLR-445
Commit: fd12a5b9f8d6319945d4445ac31e650bd1627dfc
Parents: 3a9da7a
Author: markrmiller <ma...@apache.org>
Authored: Fri Feb 19 13:41:57 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Fri Feb 19 13:41:57 2016 -0500

----------------------------------------------------------------------
 .../apache/solr/response/SolrQueryResponse.java |  4 +++
 .../processor/TolerantUpdateProcessor.java      |  4 +--
 .../TolerantUpdateProcessorFactory.java         |  3 +-
 .../DistribTolerantUpdateProcessorTest.java     | 32 +++++++++-----------
 .../cloud/TestTolerantUpdateProcessorCloud.java |  9 +-----
 5 files changed, 22 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fd12a5b9/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java b/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java
index f1ccd08..378dee8 100644
--- a/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java
+++ b/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java
@@ -161,6 +161,10 @@ public class SolrQueryResponse {
 
   /**
    * Causes an error to be returned instead of the results.
+   * 
+   * In general, new calls to this method should not be added. In most cases
+   * you should simply throw an exception and let it bubble out to 
+   * RequestHandlerBase, which will set the exception thrown.
    */
   public void setException(Exception e) {
     err=e;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fd12a5b9/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 67684e9..8b10ee9 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
@@ -33,9 +33,7 @@ import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.update.AddUpdateCommand;
-import org.apache.solr.update.DeleteUpdateCommand;
 import org.apache.solr.update.SolrCmdDistributor.Error;
-import org.apache.solr.update.UpdateCommand;
 import org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
 
 import org.slf4j.Logger;
@@ -271,7 +269,7 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
 
   /**
    * Returns the output of {@link org.apache.solr.schema.FieldType#
-   * indexedToReadable(BytesRef, CharsRef)} of the field
+   * indexedToReadable(BytesRef, CharsRefBuilder)} of the field
    * type of the uniqueKey on the {@link BytesRef} passed as parameter.
    * <code>ref</code> should be the indexed representation of the id and
    * <code>field</code> should be the uniqueKey schema field. If any of

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fd12a5b9/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 983f915..45f5fcb 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
@@ -1,5 +1,3 @@
-package org.apache.solr.update.processor;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -16,6 +14,7 @@ package org.apache.solr.update.processor;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.solr.update.processor;
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fd12a5b9/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 d49adc2..4dc3043 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java
@@ -1,20 +1,3 @@
-package org.apache.solr.cloud;
-
-import java.io.IOException;
-
-import jdk.nashorn.internal.ir.annotations.Ignore;
-import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.client.solrj.response.UpdateResponse;
-import org.apache.solr.common.SolrDocumentList;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.SimpleOrderedMap;
-import org.junit.Test;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -31,6 +14,21 @@ import org.junit.Test;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.solr.cloud;
+
+import java.io.IOException;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.junit.Test;
 
 public class DistribTolerantUpdateProcessorTest extends AbstractFullDistribZkTestBase {
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fd12a5b9/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 84247a1..dcf720b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
@@ -1,5 +1,3 @@
-package org.apache.solr.cloud;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -16,17 +14,14 @@ package org.apache.solr.cloud;
  * 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.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
@@ -35,7 +30,6 @@ import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 import org.apache.solr.SolrTestCaseJ4;
 import static org.apache.solr.SolrTestCaseJ4.params;
 import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.embedded.JettyConfig;
 import org.apache.solr.client.solrj.embedded.JettyConfig.Builder;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@@ -43,7 +37,6 @@ 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.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrInputDocument;