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

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

SOLR-445: new ToleranteUpdateProcessorFactory to support skipping update commands that cause failures when sending multiple updates in a single request.

SOLR-8890: New static method in DistributedUpdateProcessorFactory to allow UpdateProcessorFactories to indicate request params that should be forwarded when DUP distributes updates.

This commit is a squashed merge from the jira/SOLR-445 branch (as of b08c284b26b1779d03693a45e219db89839461d0)


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

Branch: refs/heads/branch_6x
Commit: 5b6eacb80bca5815059cd50a1646fa4ecb146e43
Parents: e26a443
Author: Chris Hostetter <ho...@apache.org>
Authored: Fri Mar 25 11:00:29 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Fri Mar 25 13:13:50 2016 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |    7 +
 .../apache/solr/response/SolrQueryResponse.java |    4 +
 .../apache/solr/update/SolrCmdDistributor.java  |   31 +-
 .../processor/DistributedUpdateProcessor.java   |  135 ++-
 .../DistributedUpdateProcessorFactory.java      |   20 +
 .../processor/TolerantUpdateProcessor.java      |  415 +++++++
 .../TolerantUpdateProcessorFactory.java         |  142 +++
 ...lrconfig-distrib-update-processor-chains.xml |   85 ++
 .../conf/solrconfig-tolerant-update-minimal.xml |   40 +
 .../conf/solrconfig-update-processor-chains.xml |   17 +
 .../cloud/TestTolerantUpdateProcessorCloud.java | 1065 ++++++++++++++++++
 .../TestTolerantUpdateProcessorRandomCloud.java |  389 +++++++
 .../org/apache/solr/core/TestBadConfig.java     |    5 +
 .../processor/TolerantUpdateProcessorTest.java  |  447 ++++++++
 .../solr/client/solrj/impl/CloudSolrClient.java |   67 +-
 .../solr/common/ToleratedUpdateError.java       |  197 ++++
 .../solr/common/TestToleratedUpdateError.java   |  193 ++++
 17 files changed, 3214 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b6eacb8/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4bd24df..fcdfad4 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -49,6 +49,13 @@ New Features
 * SOLR-8859: AbstractSpatialFieldType will now convert Shapes to/from Strings
   using the SpatialContext.  (ryan)
 
+* SOLR-445: new ToleranteUpdateProcessorFactory to support skipping update commands that cause
+  failures when sending multiple updates in a single request.
+  (Erick Erickson, Tomás Fernández Löbbe, Anshum Gupta, hossman)
+
+* SOLR-8890: New static method in DistributedUpdateProcessorFactory to allow UpdateProcessorFactories
+  to indicate request params that should be forwarded when DUP distributes updates. (hossman)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b6eacb8/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/5b6eacb8/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 d9b6478..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;
@@ -138,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);
@@ -187,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);
     }
   }
   
@@ -200,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);
     }
     
   }
@@ -226,7 +226,7 @@ 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);
     }
     
   }
@@ -272,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) {
@@ -314,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();
     }
@@ -382,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/5b6eacb8/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 365de6c..5f4e4f1 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -92,7 +92,8 @@ import org.slf4j.LoggerFactory;
 // NOT mt-safe... create a new processor for each add thread
 // TODO: we really should not wait for distrib after local? unless a certain replication factor is asked for
 public class DistributedUpdateProcessor extends UpdateRequestProcessor {
-  
+
+  final static String PARAM_WHITELIST_CTX_KEY = DistributedUpdateProcessor.class + "PARAM_WHITELIST_CTX_KEY";
   public static final String DISTRIB_FROM_SHARD = "distrib.from.shard";
   public static final String DISTRIB_FROM_COLLECTION = "distrib.from.collection";
   public static final String DISTRIB_FROM_PARENT = "distrib.from.parent";
@@ -292,6 +293,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
 
     this.req = req;
     
+    // this should always be used - see filterParams
+    DistributedUpdateProcessorFactory.addParamToDistributedRequestWhitelist
+      (this.req, UpdateParams.UPDATE_CHAIN, TEST_DISTRIB_SKIP_SERVERS);
+    
     CoreDescriptor coreDesc = req.getCore().getCoreDescriptor();
     
     this.zkEnabled  = coreDesc.getCoreContainer().isZooKeeperAware();
@@ -790,38 +795,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));       
@@ -841,8 +838,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);
+        continue;
       }
 
       String collection = null;
@@ -927,7 +924,12 @@ 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()) {
+      throw new DistributedUpdatesAsyncException(errorsForClient);
+    }
   }
 
  
@@ -1210,10 +1212,16 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     }
   }
 
+  /** @see DistributedUpdateProcessorFactory#addParamToDistributedRequestWhitelist */
   protected ModifiableSolrParams filterParams(SolrParams params) {
     ModifiableSolrParams fparams = new ModifiableSolrParams();
-    passParam(params, fparams, UpdateParams.UPDATE_CHAIN);
-    passParam(params, fparams, TEST_DISTRIB_SKIP_SERVERS);
+    
+    Set<String> whitelist = (Set<String>) this.req.getContext().get(PARAM_WHITELIST_CTX_KEY);
+    assert null != whitelist : "whitelist can't be null, constructor adds to it";
+
+    for (String p : whitelist) {
+      passParam(params, fparams, p);
+    }
     return fparams;
   }
 
@@ -1698,4 +1706,67 @@ 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;
+
+      // create a merged copy of the metadata from all wrapped exceptions
+      NamedList<String> metadata = new NamedList<String>();
+      for (Error error : errors) {
+        if (error.e instanceof SolrException) {
+          SolrException e = (SolrException) error.e;
+          NamedList<String> eMeta = e.getMetadata();
+          if (null != eMeta) {
+            metadata.addAll(eMeta);
+          }
+        }
+      }
+      if (0 < metadata.size()) {
+        this.setMetadata(metadata);
+      }
+    }
+
+    /** Helper method for constructor */
+    private static final int buildCode(List<Error> errors) {
+      assert null != errors;
+      assert 0 < errors.size();
+
+      int minCode = Integer.MAX_VALUE;
+      int maxCode = Integer.MIN_VALUE;
+      for (Error error : errors) {
+        log.trace("REMOTE ERROR: {}", error);
+        minCode = Math.min(error.statusCode, minCode);
+        maxCode = Math.max(error.statusCode, maxCode);
+      }
+      if (minCode == maxCode) {
+        // all codes are consistent, use that...
+        return minCode;
+      } else if (400 <= minCode && maxCode < 500) {
+        // all codes are 4xx, use 400
+        return ErrorCode.BAD_REQUEST.code;
+      } 
+      // ...otherwise use sensible default
+      return ErrorCode.SERVER_ERROR.code;
+    }
+    
+    /** 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/5b6eacb8/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java
index 4b64dec..6446b1a 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java
@@ -16,6 +16,9 @@
  */
 package org.apache.solr.update.processor;
 
+import java.util.Set;
+import java.util.TreeSet;
+
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
@@ -28,6 +31,23 @@ import org.apache.solr.response.SolrQueryResponse;
 public class DistributedUpdateProcessorFactory 
   extends UpdateRequestProcessorFactory 
   implements DistributingUpdateProcessorFactory {
+
+  /**
+   * By default, the {@link DistributedUpdateProcessor} is extremely conservative in the list of request 
+   * params that will be copied/included when updates are forwarded to other nodes.  This method may be 
+   * used by any {@link UpdateRequestProcessorFactory#getInstance} call to annotate a 
+   * SolrQueryRequest with the names of parameters that should also be forwarded.
+   */
+  public static void addParamToDistributedRequestWhitelist(final SolrQueryRequest req, final String... paramNames) {
+    Set<String> whitelist = (Set<String>) req.getContext().get(DistributedUpdateProcessor.PARAM_WHITELIST_CTX_KEY);
+    if (null == whitelist) {
+      whitelist = new TreeSet<String>();
+      req.getContext().put(DistributedUpdateProcessor.PARAM_WHITELIST_CTX_KEY, whitelist);
+    }
+    for (String p : paramNames) {
+      whitelist.add(p);
+    }
+  }
   
   @Override
   public void init(NamedList args) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b6eacb8/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..f9437f5
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessor.java
@@ -0,0 +1,415 @@
+/*
+ * 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.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Set;
+import java.lang.invoke.MethodHandles;
+
+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.ToleratedUpdateError;
+import org.apache.solr.common.ToleratedUpdateError.CmdType;
+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.CommitUpdateCommand;
+import org.apache.solr.update.DeleteUpdateCommand;
+import org.apache.solr.update.MergeIndexesCommand;
+import org.apache.solr.update.RollbackUpdateCommand;
+import org.apache.solr.update.SolrCmdDistributor.Error;
+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 single request.
+ * Instead of failing on the first error, at most <code>maxErrors</code> errors (or unlimited 
+ * if <code>-1==maxErrors</code>) are logged and recorded the batch continues. 
+ * The client will receive a <code>status==200</code> response, which includes a list of errors 
+ * that were tolerated.
+ * </p>
+ * <p>
+ * If more then <code>maxErrors</code> occur, the first exception recorded will be re-thrown, 
+ * Solr will respond with <code>status==5xx</code> or <code>status==4xx</code> 
+ * (depending on the underlying exceptions) and it won't finish processing any more updates in the request. 
+ * (ie: subsequent update commands in the request will not be processed 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(MethodHandles.lookup().lookupClass());
+  
+  /**
+   * String to be used as document key for errors when a real uniqueKey can't be determined
+   */
+  private static final String UNKNOWN_ID = "(unknown)"; 
+
+  /**
+   * 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;
+
+  /** The uniqueKey field */
+  private SchemaField uniqueKeyField;
+
+  private final SolrQueryRequest req;
+  private ZkController zkController;
+
+  /**
+   * Known errors that occurred in this batch, in order encountered (may not be the same as the 
+   * order the commands were originally executed in due to the async distributed updates).
+   */
+  private final List<ToleratedUpdateError> knownErrors = new ArrayList<ToleratedUpdateError>();
+
+  // Kludge: Because deleteByQuery updates are forwarded to every leader, we can get identical
+  // errors reported by every leader for the same underlying problem.
+  //
+  // It would be nice if we could cleanly handle the unlikely (but possible) situation of an
+  // update stream that includes multiple identical DBQs, with identical failures, and 
+  // to report each one once, for example...
+  //   add: id#1
+  //   dbq: foo:bar
+  //   add: id#2
+  //   add: id#3
+  //   dbq: foo:bar
+  //
+  // ...but i can't figure out a way to accurately identify & return duplicate 
+  // ToleratedUpdateErrors from duplicate identical underlying requests w/o erroneously returning identical 
+  // ToleratedUpdateErrors for the *same* underlying request but from diff shards.
+  //
+  // So as a kludge, we keep track of them for deduping against identical remote failures
+  //
+  private Set<ToleratedUpdateError> knownDBQErrors = new HashSet<>();
+        
+  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 >= -1;
+      
+    header = rsp.getResponseHeader();
+    this.maxErrors = ToleratedUpdateError.getEffectiveMaxErrors(maxErrors);
+    this.req = req;
+    this.distribPhase = distribPhase;
+    assert ! DistribPhase.FROMLEADER.equals(distribPhase);
+    
+    this.zkController = this.req.getCore().getCoreDescriptor().getCoreContainer().getZkController();
+    this.uniqueKeyField = this.req.getCore().getLatestSchema().getUniqueKeyField();
+    assert null != uniqueKeyField : "Factory didn't enforce uniqueKey field?";
+  }
+  
+  @Override
+  public void processAdd(AddUpdateCommand cmd) throws IOException {
+    BytesRef id = null;
+    
+    try {
+      // force AddUpdateCommand to validate+cache the id before proceeding
+      id = cmd.getIndexedId();
+      
+      super.processAdd(cmd);
+
+    } catch (Throwable t) { 
+      firstErrTracker.caught(t);
+      knownErrors.add(new ToleratedUpdateError
+                      (CmdType.ADD,
+                       getPrintableId(id),
+                       t.getMessage()));
+      
+      if (knownErrors.size() > maxErrors) {
+        firstErrTracker.throwFirst();
+      }
+    }
+  }
+
+  @Override
+  public void processDelete(DeleteUpdateCommand cmd) throws IOException {
+    
+    try {
+      
+      super.processDelete(cmd);
+      
+    } catch (Throwable t) {
+      firstErrTracker.caught(t);
+      
+      ToleratedUpdateError err = new ToleratedUpdateError(cmd.isDeleteById() ? CmdType.DELID : CmdType.DELQ,
+                                                          cmd.isDeleteById() ? cmd.id : cmd.query,
+                                                          t.getMessage());
+      knownErrors.add(err);
+
+      // NOTE: we're not using this to dedup before adding to knownErrors.
+      // if we're lucky enough to get an immediate local failure (ie: we're a leader, or some other processor
+      // failed) then recording the multiple failures is a good thing -- helps us with an accurate fail
+      // fast if we exceed maxErrors
+      if (CmdType.DELQ.equals(err.getType())) {
+        knownDBQErrors.add(err);
+      }
+      
+      if (knownErrors.size() > maxErrors) {
+        firstErrTracker.throwFirst();
+      }
+    }
+  }
+
+  @Override
+  public void processMergeIndexes(MergeIndexesCommand cmd) throws IOException {
+    try {
+      super.processMergeIndexes(cmd);
+    } catch (Throwable t) {
+      // we're not tolerante of errors from this type of command, but we
+      // do need to track it so we can annotate it with any other errors we were allready tolerant of
+      firstErrTracker.caught(t);
+      throw t;
+    }
+  }
+
+  @Override
+  public void processCommit(CommitUpdateCommand cmd) throws IOException {
+    try {
+      super.processCommit(cmd);
+    } catch (Throwable t) {
+      // we're not tolerante of errors from this type of command, but we
+      // do need to track it so we can annotate it with any other errors we were allready tolerant of
+      firstErrTracker.caught(t);
+      throw t;
+    }
+  }
+
+  @Override
+  public void processRollback(RollbackUpdateCommand cmd) throws IOException {
+    try {
+      super.processRollback(cmd);
+    } catch (Throwable t) {
+      // we're not tolerante of errors from this type of command, but we
+      // do need to track it so we can annotate it with any other errors we were allready tolerant of
+      firstErrTracker.caught(t);
+      throw t;
+    }
+  }
+
+  @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 our stats based on each of 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;
+        }
+
+        for (int i = 0; i < remoteErrMetadata.size(); i++) {
+          ToleratedUpdateError err =
+            ToleratedUpdateError.parseMetadataIfToleratedUpdateError(remoteErrMetadata.getName(i),
+                                                                     remoteErrMetadata.getVal(i));
+          if (null == err) {
+            // some metadata unrelated to this update processor
+            continue;
+          }
+
+          if (CmdType.DELQ.equals(err.getType())) {
+            if (knownDBQErrors.contains(err)) {
+              // we've already seen this identical error, probably a dup from another shard
+              continue;
+            } else {
+              knownDBQErrors.add(err);
+            }
+          }
+          
+          knownErrors.add(err);
+        }
+      }
+    }
+
+    header.add("errors", ToleratedUpdateError.formatForResponseHeader(knownErrors));
+    // include in response so client knows what effective value was (may have been server side config)
+    header.add("maxErrors", ToleratedUpdateError.getUserFriendlyMaxErrors(maxErrors));
+
+    // annotate any error that might be thrown (or was already thrown)
+    firstErrTracker.annotate(knownErrors);
+
+    // decide if we have hit a situation where we know an error needs to be thrown.
+    
+    if ((DistribPhase.TOLEADER.equals(distribPhase) ? 0 : maxErrors) < knownErrors.size()) {
+      // NOTE: even if maxErrors wasn't exceeeded, we need to throw an error when we have any errors if we're
+      // a leader that was forwarded to by another node so that the forwarding node knows we encountered some
+      // problems and can aggregate the results
+
+      firstErrTracker.throwFirst();
+    }
+  }
+
+  /**
+   * Returns the output of {@link org.apache.solr.schema.FieldType#
+   * 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 -- if null
+   * (possibly because it's missing in the update) this method will return {@link #UNKNOWN_ID}
+   */
+  private String getPrintableId(BytesRef ref) {
+    if (ref == null) {
+      return UNKNOWN_ID;
+    }
+    return uniqueKeyField.getType().indexedToReadable(ref, new CharsRefBuilder()).toString();
+  }
+
+  /**
+   * 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) {
+      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(List<ToleratedUpdateError> errors) {
+
+      if (null == first) {
+        return; // no exception to annotate
+      }
+      
+      assert null != errors : "how do we have an exception to annotate w/o any errors?";
+      
+      NamedList<String> firstErrMetadata = first.getMetadata();
+      if (null == firstErrMetadata) { // obnoxious
+        firstErrMetadata = new NamedList<String>();
+        first.setMetadata(firstErrMetadata);
+      } else {
+        // any existing metadata representing ToleratedUpdateErrors in this single exception needs removed
+        // so we can add *all* of the known ToleratedUpdateErrors (from this and other exceptions)
+        for (int i = 0; i < firstErrMetadata.size(); i++) {
+          if (null != ToleratedUpdateError.parseMetadataIfToleratedUpdateError
+              (firstErrMetadata.getName(i), firstErrMetadata.getVal(i))) {
+               
+            firstErrMetadata.remove(i);
+            // NOTE: post decrementing index so we don't miss anything as we remove items
+            i--;
+          }
+        }
+      }
+
+      for (ToleratedUpdateError te : errors) {
+        firstErrMetadata.add(te.getMetadataKey(), te.getMetadataValue());
+      }
+    }
+    
+    
+    /** 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/5b6eacb8/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..8cd3500
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessorFactory.java
@@ -0,0 +1,142 @@
+/*
+ * 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 org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
+import org.apache.solr.util.plugin.SolrCoreAware;
+
+import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
+
+/**
+ * <p> 
+ * Suppresses errors for individual add/delete commands within a single request.
+ * Instead of failing on the first error, at most <code>maxErrors</code> errors (or unlimited 
+ * if <code>-1==maxErrors</code>) are logged and recorded the batch continues. 
+ * The client will receive a <code>status==200</code> response, which includes a list of errors 
+ * that were tolerated.
+ * </p>
+ * <p>
+ * If more then <code>maxErrors</code> occur, the first exception recorded will be re-thrown, 
+ * Solr will respond with <code>status==5xx</code> or <code>status==4xx</code> 
+ * (depending on the underlying exceptions) and it won't finish processing any more updates in the request. 
+ * (ie: subsequent update commands in the request will not be processed even if they are valid).
+ * </p>
+ * 
+ * <p>
+ * <code>maxErrors</code> is an int value that can be specified in the configuration and/or overridden 
+ * per request. If unset, it will default to {@link Integer#MAX_VALUE}.  Specifying an explicit value 
+ * of <code>-1</code> is supported as shorthand for {@link Integer#MAX_VALUE}, all other negative 
+ * integer values are not supported.
+ * </p>
+ * <p>
+ * An example configuration would be:
+ * </p>
+ * <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 <code>maxErrors</code> parameter in the above chain could be overwritten per request, for example:
+ * </p>
+ * <pre class="prettyprint">
+ * curl http://localhost:8983/update?update.chain=tolerant-chain&amp;maxErrors=100 -H "Content-Type: text/xml" -d @myfile.xml
+ * </pre>
+ * 
+ * <p>
+ * <b>NOTE:</b> The behavior of this UpdateProcessofFactory in conjunction with indexing operations 
+ * while a Shard Split is actively in progress is not well defined (or sufficiently tested).  Users 
+ * of this update processor are encouraged to either disable it, or pause updates, while any shard 
+ * splitting is in progress (see <a href="https://issues.apache.org/jira/browse/SOLR-8881">SOLR-8881</a> 
+ * for more details.)
+ * </p>
+ */
+public class TolerantUpdateProcessorFactory extends UpdateRequestProcessorFactory
+  implements SolrCoreAware, UpdateRequestProcessorFactory.RunAlways {
+  
+  /**
+   * 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 int defaultMaxErrors = Integer.MAX_VALUE;
+
+  private boolean informed = false;
+  
+  @SuppressWarnings("rawtypes")
+  @Override
+  public void init( NamedList args ) {
+
+    Object maxErrorsObj = args.get(MAX_ERRORS_PARAM); 
+    if (maxErrorsObj != null) {
+      try {
+        defaultMaxErrors = Integer.valueOf(maxErrorsObj.toString());
+      } catch (Exception e) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Unnable to parse maxErrors parameter: " + maxErrorsObj, e);
+      }
+      if (defaultMaxErrors < -1) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Config option '"+MAX_ERRORS_PARAM + "' must either be non-negative, or -1 to indicate 'unlimiited': " + maxErrorsObj.toString());
+      }
+    }
+  }
+  
+  @Override
+  public void inform(SolrCore core) {
+    informed = true;
+    if (null == core.getLatestSchema().getUniqueKeyField()) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, this.getClass().getName() +
+                              " requires a schema that includes a uniqueKey field.");
+    }
+  }
+
+  @Override
+  public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next) {
+
+    assert informed : "inform(SolrCore) never called?";
+    
+    // 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;
+    }
+
+    DistributedUpdateProcessorFactory.addParamToDistributedRequestWhitelist(req, MAX_ERRORS_PARAM);
+    int maxErrors = req.getParams().getInt(MAX_ERRORS_PARAM, defaultMaxErrors);
+    if (maxErrors < -1) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "'"+MAX_ERRORS_PARAM + "' must either be non-negative, or -1 to indicate 'unlimiited': " + maxErrors);
+    }
+
+    // NOTE: even if 0==maxErrors, we still inject processor into chain so respones has expected header info
+    return new TolerantUpdateProcessor(req, rsp, next, maxErrors, distribPhase);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b6eacb8/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..97ed18b
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-distrib-update-processor-chains.xml
@@ -0,0 +1,85 @@
+<?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">
+      <!-- explicitly testing that parsing still works if a valid int is specified as a string -->
+      <str name="maxErrors">10</str>
+    </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/5b6eacb8/solr/core/src/test-files/solr/collection1/conf/solrconfig-tolerant-update-minimal.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tolerant-update-minimal.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tolerant-update-minimal.xml
new file mode 100644
index 0000000..d3b90db
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tolerant-update-minimal.xml
@@ -0,0 +1,40 @@
+<?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>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+  
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+      <str name="indent">true</str>
+      <str name="df">text</str>
+    </lst>
+  </requestHandler>
+  
+  <updateRequestProcessorChain name="tolerant-chain">
+    <processor class="solr.TolerantUpdateProcessorFactory" />
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+</config>
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5b6eacb8/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>