SOLR-4260: ConcurrentUpdateSolrServer#blockUntilFinished can return before all previously added updates have finished. This could cause distributed updates meant for replicas to be lost.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1558978 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mark Robert Miller 2014-01-17 01:48:41 +00:00
parent a6b1c654d1
commit 5cacfe37f7
5 changed files with 12 additions and 13 deletions

View File

@ -378,6 +378,11 @@ Bug Fixes
* SOLR-5618: Fix false cache hits in queryResultCache when hashCodes are equal
and duplicate filter queries exist in one of the requests (hossman)
* SOLR-4260: ConcurrentUpdateSolrServer#blockUntilFinished can return before
all previously added updates have finished. This could cause distributed
updates meant for replicas to be lost. (Markus Jelsma, Timothy Potter,
Joel Bernstein)
Optimizations
----------------------

View File

@ -25,7 +25,6 @@ import java.util.List;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.HttpSolrServer;
import org.apache.solr.client.solrj.impl.HttpSolrServer.RemoteSolrException;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.common.SolrException;

View File

@ -31,9 +31,7 @@ import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.impl.BinaryRequestWriter;
import org.apache.solr.client.solrj.impl.BinaryResponseParser;
import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrServer;
import org.apache.solr.client.solrj.impl.HttpClientUtil;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.update.SolrCmdDistributor.Error;
import org.apache.solr.update.processor.DistributedUpdateProcessor;
import org.apache.solr.update.processor.DistributingUpdateProcessorFactory;
@ -53,11 +51,6 @@ public class StreamingSolrServers {
public StreamingSolrServers(UpdateShardHandler updateShardHandler) {
this.updateExecutor = updateShardHandler.getUpdateExecutor();
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(HttpClientUtil.PROP_FOLLOW_REDIRECTS, false);
params.set(HttpClientUtil.PROP_CONNECTION_TIMEOUT, 30000);
params.set(HttpClientUtil.PROP_USE_RETRY, false);
httpClient = updateShardHandler.getHttpClient();
}

View File

@ -146,11 +146,11 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
private SolrCmdDistributor cmdDistrib;
private boolean zkEnabled = false;
private final boolean zkEnabled;
private CloudDescriptor cloudDesc;
private String collection;
private ZkController zkController;
private final String collection;
private final ZkController zkController;
// these are setup at the start of each request processing
// method in this update processor
@ -193,6 +193,8 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
if (cloudDesc != null) {
collection = cloudDesc.getCollectionName();
} else {
collection = null;
}
}

View File

@ -344,7 +344,6 @@ public class ConcurrentUpdateSolrServer extends SolrServer {
// successfully, *and*
// while we are still holding the runners lock to prevent race
// conditions.
// race conditions.
if (success)
break;
}
@ -387,7 +386,8 @@ public class ConcurrentUpdateSolrServer extends SolrServer {
synchronized (runners) {
runner = runners.peek();
}
if (runner == null || scheduler.isTerminated())
if ((runner == null && queue.isEmpty()) || scheduler.isTerminated())
break;
runner.runnerLock.lock();
runner.runnerLock.unlock();