mirror of https://github.com/apache/lucene.git
SOLR-6763: Shard leader elections should not persist across ZK session expiry
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1641590 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
63867cb7ce
commit
5dbff1d1c1
|
@ -299,6 +299,9 @@ Bug Fixes
|
|||
* SOLR-3774: Solr adds RequestHandler SolrInfoMBeans twice to the JMX server.
|
||||
(Tomás Fernández Löbbe, hossman, Mark Miller)
|
||||
|
||||
* SOLR-6763: Shard leader elections should not persist across session expiry
|
||||
(Alan Woodward, Mark Miller)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -1,9 +1,5 @@
|
|||
package org.apache.solr.cloud;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
|
@ -28,6 +24,7 @@ import org.apache.zookeeper.KeeperException.NodeExistsException;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
@ -136,13 +133,12 @@ class ShardLeaderElectionContextBase extends ElectionContext {
|
|||
try {
|
||||
RetryUtil.retryOnThrowable(NodeExistsException.class, 15000, 1000,
|
||||
new RetryCmd() {
|
||||
|
||||
@Override
|
||||
public void execute() throws Throwable {
|
||||
zkClient.makePath(leaderPath, ZkStateReader.toJSON(leaderProps),
|
||||
CreateMode.EPHEMERAL, true);
|
||||
zkClient.makePath(leaderPath, ZkStateReader.toJSON(leaderProps), CreateMode.EPHEMERAL, true);
|
||||
}
|
||||
});
|
||||
}
|
||||
);
|
||||
} catch (Throwable t) {
|
||||
if (t instanceof OutOfMemoryError) {
|
||||
throw (OutOfMemoryError) t;
|
||||
|
@ -407,13 +403,19 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
|
|||
|
||||
Slice slices = zkController.getClusterState().getSlice(collection, shardId);
|
||||
int cnt = 0;
|
||||
while (true && !isClosed && !cc.isShutDown()) {
|
||||
while (!isClosed && !cc.isShutDown()) {
|
||||
// wait for everyone to be up
|
||||
if (slices != null) {
|
||||
int found = 0;
|
||||
try {
|
||||
found = zkClient.getChildren(shardsElectZkPath, null, true).size();
|
||||
} catch (KeeperException e) {
|
||||
if (e instanceof KeeperException.SessionExpiredException) {
|
||||
// if the session has expired, then another election will be launched, so
|
||||
// quit here
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR,
|
||||
"ZK session expired - cancelling election for " + collection + " " + shardId);
|
||||
}
|
||||
SolrException.log(log,
|
||||
"Error checking for the number of election participants", e);
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue