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 2017/01/16 00:10:48 UTC

[30/50] [abbrv] lucene-solr:jira/solr-5944: SOLR-9918: Add SkipExistingDocumentsProcessor that skips duplicate inserts and ignores updates to missing docs

SOLR-9918: Add SkipExistingDocumentsProcessor that skips duplicate inserts and ignores updates to missing docs


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

Branch: refs/heads/jira/solr-5944
Commit: d66bfba5dc1bd9154bd48898865f51d9715e8d0c
Parents: 118fc42
Author: koji <ko...@apache.org>
Authored: Wed Jan 11 11:48:33 2017 +0900
Committer: koji <ko...@apache.org>
Committed: Wed Jan 11 11:48:33 2017 +0900

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../SkipExistingDocumentsProcessorFactory.java  | 255 ++++++++++++++
 ...ipExistingDocumentsProcessorFactoryTest.java | 336 +++++++++++++++++++
 3 files changed, 594 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d66bfba5/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0ee18ba..204ea26 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -126,6 +126,9 @@ Upgrade Notes
 
 New Features
 ----------------------
+* SOLR-9918: Add SkipExistingDocumentsProcessor that skips duplicate inserts and ignores updates to missing docs
+  (Tim Owen via koji)
+
 * SOLR-9293: Solrj client support for hierarchical clusters and other topics 
   marker. (Dawid Weiss)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d66bfba5/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java
new file mode 100644
index 0000000..ec637a4
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactory.java
@@ -0,0 +1,255 @@
+/*
+ * 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.lucene.util.BytesRef;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.component.RealTimeGetComponent;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.UpdateCommand;
+import org.apache.solr.util.RefCounted;
+import org.apache.solr.util.plugin.SolrCoreAware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+
+import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
+import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
+
+/**
+ * <p>
+ *     This Factory generates an UpdateProcessor that will (by default) skip inserting new documents
+ *     if there already exists a document with the same uniqueKey value in the index. It will also
+ *     skip Atomic Updates to a document if that document does not already exist. This behaviour is applied
+ *     to each document in turn, so adding a batch of documents can result in some being added and some
+ *     ignored, depending on what is already in the index. If all of the documents are skipped, no changes
+ *     to the index will occur.
+ * </p>
+ * These two forms of skipping can be switched on or off independently, by using init params:
+ * <ul>
+ *     <li><code>skipInsertIfExists</code> - This boolean parameter defaults to
+ *          <code>true</code>, but if set to <code>false</code> then inserts (i.e. not Atomic Updates)
+ *          will be passed through unchanged even if the document already exists.</li>
+ *     <li><code>skipUpdateIfMissing</code> - This boolean parameter defaults to
+ *         <code>true</code>, but if set to <code>false</code> then Atomic Updates
+ *          will be passed through unchanged regardless of whether the document exists.</li>
+ * </ul>
+ * <p>
+ *     These params can also be specified per-request, to override the configured behaviour
+ *     for specific updates e.g. <code>/update?skipUpdateIfMissing=true</code>
+ * </p>
+ * <p>
+ *     This implementation is a simpler alternative to {@link DocBasedVersionConstraintsProcessorFactory}
+ *     when you are not concerned with versioning, and just want to quietly ignore duplicate documents and/or
+ *     silently skip updates to non-existent documents (in the same way a database <code>UPDATE</code> would).
+ *
+ *     If your documents do have an explicit version field, and you want to ensure older versions are
+ *     skipped instead of replacing the indexed document, you should consider {@link DocBasedVersionConstraintsProcessorFactory}
+ *     instead.
+ * </p>
+ * <p>
+ *     An example chain configuration to use this for skipping duplicate inserts, but not skipping updates to
+ *     missing documents by default, is:
+ * </p>
+ * <pre class="prettyprint">
+ * &lt;updateRequestProcessorChain name="skipexisting"&gt;
+ *   &lt;processor class="solr.LogUpdateProcessorFactory" /&gt;
+ *   &lt;processor class="solr.SkipExistingDocumentsProcessorFactory"&gt;
+ *     &lt;bool name="skipInsertIfExists"&gt;true&lt;/bool&gt;
+ *     &lt;bool name="skipUpdateIfMissing"&gt;false&lt;/bool&gt; &lt;!-- Can override this per-request --&gt;
+ *   &lt;/processor&gt;
+ *   &lt;processor class="solr.DistributedUpdateProcessorFactory" /&gt;
+ *   &lt;processor class="solr.RunUpdateProcessorFactory" /&gt;
+ * &lt;/updateRequestProcessorChain&gt;
+ * </pre>
+ */
+public class SkipExistingDocumentsProcessorFactory extends UpdateRequestProcessorFactory implements SolrCoreAware, UpdateRequestProcessorFactory.RunAlways {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String PARAM_SKIP_INSERT_IF_EXISTS = "skipInsertIfExists";
+  private static final String PARAM_SKIP_UPDATE_IF_MISSING = "skipUpdateIfMissing";
+
+  private boolean skipInsertIfExists = true;
+  private boolean skipUpdateIfMissing = true;
+
+  @Override
+  public void init(NamedList args)  {
+    Object tmp = args.remove(PARAM_SKIP_INSERT_IF_EXISTS);
+    if (null != tmp) {
+      if (! (tmp instanceof Boolean) ) {
+        throw new SolrException(SERVER_ERROR, "'" + PARAM_SKIP_INSERT_IF_EXISTS + "' must be configured as a <bool>");
+      }
+      skipInsertIfExists = (Boolean)tmp;
+    }
+    tmp = args.remove(PARAM_SKIP_UPDATE_IF_MISSING);
+    if (null != tmp) {
+      if (! (tmp instanceof Boolean) ) {
+        throw new SolrException(SERVER_ERROR, "'" + PARAM_SKIP_UPDATE_IF_MISSING + "' must be configured as a <bool>");
+      }
+      skipUpdateIfMissing = (Boolean)tmp;
+    }
+
+    super.init(args);
+  }
+
+  @Override
+  public SkipExistingDocumentsUpdateProcessor getInstance(SolrQueryRequest req,
+                                                          SolrQueryResponse rsp,
+                                                          UpdateRequestProcessor next) {
+    // Ensure the parameters are forwarded to the leader
+    DistributedUpdateProcessorFactory.addParamToDistributedRequestWhitelist(req, PARAM_SKIP_INSERT_IF_EXISTS, PARAM_SKIP_UPDATE_IF_MISSING);
+
+    // Allow the particular request to override the plugin's configured behaviour
+    boolean skipInsertForRequest = req.getOriginalParams().getBool(PARAM_SKIP_INSERT_IF_EXISTS, this.skipInsertIfExists);
+    boolean skipUpdateForRequest = req.getOriginalParams().getBool(PARAM_SKIP_UPDATE_IF_MISSING, this.skipUpdateIfMissing);
+
+    return new SkipExistingDocumentsUpdateProcessor(req, next, skipInsertForRequest, skipUpdateForRequest);
+  }
+
+  @Override
+  public void inform(SolrCore core) {
+
+    if (core.getUpdateHandler().getUpdateLog() == null) {
+      throw new SolrException(SERVER_ERROR, "updateLog must be enabled.");
+    }
+
+    if (core.getLatestSchema().getUniqueKeyField() == null) {
+      throw new SolrException(SERVER_ERROR, "schema must have uniqueKey defined.");
+    }
+  }
+
+  static class SkipExistingDocumentsUpdateProcessor extends UpdateRequestProcessor {
+
+    private final boolean skipInsertIfExists;
+    private final boolean skipUpdateIfMissing;
+    private final SolrCore core;
+
+    private DistributedUpdateProcessor distribProc;  // the distributed update processor following us
+    private DistributedUpdateProcessor.DistribPhase phase;
+
+    SkipExistingDocumentsUpdateProcessor(SolrQueryRequest req,
+                                         UpdateRequestProcessor next,
+                                         boolean skipInsertIfExists,
+                                         boolean skipUpdateIfMissing) {
+      super(next);
+      this.skipInsertIfExists = skipInsertIfExists;
+      this.skipUpdateIfMissing = skipUpdateIfMissing;
+      this.core = req.getCore();
+
+      for (UpdateRequestProcessor proc = next ;proc != null; proc = proc.next) {
+        if (proc instanceof DistributedUpdateProcessor) {
+          distribProc = (DistributedUpdateProcessor)proc;
+          break;
+        }
+      }
+
+      if (distribProc == null) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "DistributedUpdateProcessor must follow SkipExistingDocumentsUpdateProcessor");
+      }
+
+      phase = DistributedUpdateProcessor.DistribPhase.parseParam(req.getParams().get(DISTRIB_UPDATE_PARAM));
+    }
+
+    boolean isSkipInsertIfExists() {
+      return this.skipInsertIfExists;
+    }
+
+    boolean isSkipUpdateIfMissing() {
+      return this.skipUpdateIfMissing;
+    }
+
+    boolean doesDocumentExist(BytesRef indexedDocId) throws IOException {
+      assert null != indexedDocId;
+
+      SolrInputDocument oldDoc = RealTimeGetComponent.getInputDocumentFromTlog(core, indexedDocId);
+      if (oldDoc == RealTimeGetComponent.DELETED) {
+        return false;
+      }
+      if (oldDoc != null) {
+        return true;
+      }
+
+      // need to look up in index now...
+      RefCounted<SolrIndexSearcher> newestSearcher = core.getRealtimeSearcher();
+      try {
+        SolrIndexSearcher searcher = newestSearcher.get();
+        return searcher.lookupId(indexedDocId) >= 0L;
+      } catch (IOException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error reading document from index", e);
+      } finally {
+        if (newestSearcher != null) {
+          newestSearcher.decref();
+        }
+      }
+    }
+
+    boolean isLeader(UpdateCommand cmd) {
+      if ((cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0) {
+        return false;
+      }
+      if (phase == DistributedUpdateProcessor.DistribPhase.FROMLEADER) {
+        return false;
+      }
+      return distribProc.isLeader(cmd);
+    }
+
+    @Override
+    public void processAdd(AddUpdateCommand cmd) throws IOException {
+      BytesRef indexedDocId = cmd.getIndexedId();
+
+      boolean isUpdate = AtomicUpdateDocumentMerger.isAtomicUpdate(cmd);
+
+      // boolean existsByLookup = (RealTimeGetComponent.getInputDocument(core, indexedDocId) != null);
+      // if (docExists != existsByLookup) {
+      //   log.error("Found docExists {} but existsByLookup {} for doc {}", docExists, existsByLookup, indexedDocId.utf8ToString());
+      // }
+
+      if (log.isDebugEnabled()) {
+        log.debug("Document ID {} ... exists already? {} ... isAtomicUpdate? {} ... isLeader? {}",
+                  indexedDocId.utf8ToString(), doesDocumentExist(indexedDocId), isUpdate, isLeader(cmd));
+      }
+
+      if (skipInsertIfExists && !isUpdate && isLeader(cmd) && doesDocumentExist(indexedDocId)) {
+        if (log.isDebugEnabled()) {
+          log.debug("Skipping insert for pre-existing document ID {}", indexedDocId.utf8ToString());
+        }
+        return;
+      }
+
+      if (skipUpdateIfMissing && isUpdate && isLeader(cmd) && !doesDocumentExist(indexedDocId)) {
+        if (log.isDebugEnabled()) {
+          log.debug("Skipping update to non-existent document ID {}", indexedDocId.utf8ToString());
+        }
+        return;
+      }
+
+      if (log.isDebugEnabled()) {
+        log.debug("Passing on document ID {}", indexedDocId.utf8ToString());
+      }
+
+      super.processAdd(cmd);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d66bfba5/solr/core/src/test/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactoryTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactoryTest.java b/solr/core/src/test/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactoryTest.java
new file mode 100644
index 0000000..2afe35c
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/update/processor/SkipExistingDocumentsProcessorFactoryTest.java
@@ -0,0 +1,336 @@
+/*
+ * 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 static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.processor.SkipExistingDocumentsProcessorFactory.SkipExistingDocumentsUpdateProcessor;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class SkipExistingDocumentsProcessorFactoryTest {
+
+  private BytesRef docId = new BytesRef();
+  private SolrQueryRequest defaultRequest = new LocalSolrQueryRequest(null, new NamedList());
+
+  // Tests for logic in the factory
+
+  @Test(expected=SolrException.class)
+  public void testExceptionIfSkipInsertParamNonBoolean() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipInsertIfExists", "false");
+    factory.init(initArgs);
+  }
+
+  @Test(expected=SolrException.class)
+  public void testExceptionIfSkipUpdateParamNonBoolean() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipUpdateIfMissing", 0);
+    factory.init(initArgs);
+  }
+
+  @Test(expected=SolrException.class)
+  public void testExceptionIfNextProcessorIsNull() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    factory.init(initArgs);
+
+    factory.getInstance(defaultRequest, new SolrQueryResponse(), null);
+  }
+
+  @Test(expected=SolrException.class)
+  public void testExceptionIfNextProcessorNotDistributed() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    factory.init(initArgs);
+    UpdateRequestProcessor next = new BufferingRequestProcessor(null);
+
+    factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+  }
+
+  @Test
+  public void testNoExceptionIfNextProcessorIsDistributed() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    factory.init(initArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+  }
+
+  @Test
+  public void testNoExceptionIfNextNextProcessorIsDistributed() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    factory.init(initArgs);
+    UpdateRequestProcessor distProcessor = Mockito.mock(DistributedUpdateProcessor.class);
+    UpdateRequestProcessor next = new BufferingRequestProcessor(distProcessor);
+
+    factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+  }
+
+  @Test
+  public void testSkipInsertsAndUpdatesDefaultToTrueIfNotConfigured() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    factory.init(initArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+    assertTrue("Expected skipInsertIfExists to be true", processor.isSkipInsertIfExists());
+    assertTrue("Expected skipUpdateIfMissing to be true", processor.isSkipUpdateIfMissing());
+  }
+
+  @Test
+  public void testSkipInsertsFalseIfInInitArgs() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipInsertIfExists", false);
+    factory.init(initArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+    assertFalse("Expected skipInsertIfExists to be false", processor.isSkipInsertIfExists());
+    assertTrue("Expected skipUpdateIfMissing to be true", processor.isSkipUpdateIfMissing());
+  }
+
+  @Test
+  public void testSkipUpdatesFalseIfInInitArgs() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipUpdateIfMissing", false);
+    factory.init(initArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+    assertTrue("Expected skipInsertIfExists to be true", processor.isSkipInsertIfExists());
+    assertFalse("Expected skipUpdateIfMissing to be false", processor.isSkipUpdateIfMissing());
+  }
+
+  @Test
+  public void testSkipBothFalseIfInInitArgs() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipInsertIfExists", false);
+    initArgs.add("skipUpdateIfMissing", false);
+    factory.init(initArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(defaultRequest, new SolrQueryResponse(), next);
+    assertFalse("Expected skipInsertIfExists to be false", processor.isSkipInsertIfExists());
+    assertFalse("Expected skipUpdateIfMissing to be false", processor.isSkipUpdateIfMissing());
+  }
+
+  @Test
+  public void testSkipInsertsFalseIfInitArgsTrueButFalseStringInRequest() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipInsertIfExists", true);
+    factory.init(initArgs);
+    NamedList<String> requestArgs = new NamedList<>();
+    requestArgs.add("skipInsertIfExists", "false");
+    SolrQueryRequest req = new LocalSolrQueryRequest(null, requestArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(req, new SolrQueryResponse(), next);
+    assertFalse("Expected skipInsertIfExists to be false", processor.isSkipInsertIfExists());
+    assertTrue("Expected skipUpdateIfMissing to be true", processor.isSkipUpdateIfMissing());
+  }
+
+  @Test
+  public void testSkipUpdatesFalseIfInitArgsTrueButFalseBooleanInRequest() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipUpdateIfMissing", true);
+    factory.init(initArgs);
+    NamedList<Object> requestArgs = new NamedList<>();
+    requestArgs.add("skipUpdateIfMissing", false);
+    SolrQueryRequest req = new LocalSolrQueryRequest(null, requestArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(req, new SolrQueryResponse(), next);
+    assertTrue("Expected skipInsertIfExists to be true", processor.isSkipInsertIfExists());
+    assertFalse("Expected skipUpdateIfMissing to be false", processor.isSkipUpdateIfMissing());
+  }
+
+  @Test
+  public void testSkipUpdatesTrueIfInitArgsFalseButTrueStringInRequest() {
+    SkipExistingDocumentsProcessorFactory factory = new SkipExistingDocumentsProcessorFactory();
+    NamedList<Object> initArgs = new NamedList<>();
+    initArgs.add("skipInsertIfExists", true);
+    initArgs.add("skipUpdateIfMissing", false);
+    factory.init(initArgs);
+    NamedList<Object> requestArgs = new NamedList<>();
+    requestArgs.add("skipUpdateIfMissing", "true");
+    SolrQueryRequest req = new LocalSolrQueryRequest(null, requestArgs);
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+
+    SkipExistingDocumentsUpdateProcessor processor = factory.getInstance(req, new SolrQueryResponse(), next);
+    assertTrue("Expected skipInsertIfExists to be true", processor.isSkipInsertIfExists());
+    assertTrue("Expected skipUpdateIfMissing to be true", processor.isSkipUpdateIfMissing());
+  }
+
+
+  // Tests for logic in the processor
+
+  @Test
+  public void testSkippableInsertIsNotSkippedIfNotLeader() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, true, true));
+
+    AddUpdateCommand cmd = createInsertUpdateCmd(defaultRequest);
+    doReturn(false).when(processor).isLeader(cmd);
+    doReturn(true).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next).processAdd(cmd);
+  }
+
+  @Test
+  public void testSkippableInsertIsNotSkippedIfSkipInsertsFalse() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, false, false));
+
+    AddUpdateCommand cmd = createInsertUpdateCmd(defaultRequest);
+    doReturn(true).when(processor).isLeader(cmd);
+    doReturn(true).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next).processAdd(cmd);
+  }
+
+  @Test
+  public void testSkippableInsertIsSkippedIfSkipInsertsTrue() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, true, false));
+
+    AddUpdateCommand cmd = createInsertUpdateCmd(defaultRequest);
+    doReturn(true).when(processor).isLeader(cmd);
+    doReturn(true).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next, never()).processAdd(cmd);
+  }
+
+  @Test
+  public void testNonSkippableInsertIsNotSkippedIfSkipInsertsTrue() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, true, false));
+
+    AddUpdateCommand cmd = createInsertUpdateCmd(defaultRequest);
+    doReturn(true).when(processor).isLeader(cmd);
+    doReturn(false).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next).processAdd(cmd);
+  }
+
+  @Test
+  public void testSkippableUpdateIsNotSkippedIfNotLeader() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, true, true));
+
+    AddUpdateCommand cmd = createAtomicUpdateCmd(defaultRequest);
+    doReturn(false).when(processor).isLeader(cmd);
+    doReturn(false).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next).processAdd(cmd);
+  }
+
+  @Test
+  public void testSkippableUpdateIsNotSkippedIfSkipUpdatesFalse() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, false, false));
+
+    AddUpdateCommand cmd = createAtomicUpdateCmd(defaultRequest);
+    doReturn(true).when(processor).isLeader(cmd);
+    doReturn(false).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next).processAdd(cmd);
+  }
+
+  @Test
+  public void testSkippableUpdateIsSkippedIfSkipUpdatesTrue() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, false, true));
+
+    AddUpdateCommand cmd = createAtomicUpdateCmd(defaultRequest);
+    doReturn(true).when(processor).isLeader(cmd);
+    doReturn(false).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next, never()).processAdd(cmd);
+  }
+
+  @Test
+  public void testNonSkippableUpdateIsNotSkippedIfSkipUpdatesTrue() throws IOException {
+    UpdateRequestProcessor next = Mockito.mock(DistributedUpdateProcessor.class);
+    SkipExistingDocumentsUpdateProcessor processor
+            = Mockito.spy(new SkipExistingDocumentsUpdateProcessor(defaultRequest, next, false, true));
+
+    AddUpdateCommand cmd = createAtomicUpdateCmd(defaultRequest);
+    doReturn(true).when(processor).isLeader(cmd);
+    doReturn(true).when(processor).doesDocumentExist(docId);
+
+    processor.processAdd(cmd);
+    verify(next).processAdd(cmd);
+  }
+
+  private AddUpdateCommand createInsertUpdateCmd(SolrQueryRequest req) {
+    AddUpdateCommand cmd = new AddUpdateCommand(req);
+    cmd.setIndexedId(docId);
+    cmd.solrDoc = new SolrInputDocument();
+    assertFalse(AtomicUpdateDocumentMerger.isAtomicUpdate(cmd));
+    return cmd;
+  }
+
+  private AddUpdateCommand createAtomicUpdateCmd(SolrQueryRequest req) {
+    AddUpdateCommand cmd = new AddUpdateCommand(req);
+    cmd.setIndexedId(docId);
+    cmd.solrDoc = new SolrInputDocument();
+    cmd.solrDoc.addField("last_name", ImmutableMap.of("set", "Smith"));
+    assertTrue(AtomicUpdateDocumentMerger.isAtomicUpdate(cmd));
+    return cmd;
+  }
+}