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

This commit is contained in:
koji 2017-01-11 11:48:33 +09:00
parent 118fc422d0
commit d66bfba5dc
3 changed files with 594 additions and 0 deletions

View File

@ -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)

View File

@ -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);
}
}
}

View File

@ -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;
}
}