mirror of https://github.com/apache/lucene.git
SOLR-8777: Duplicate Solr process can cripple a running process
(cherry picked from commit 4ea95bf
)
This commit is contained in:
parent
976501f6f3
commit
812fd346f7
|
@ -54,10 +54,12 @@ Bug Fixes
|
||||||
|
|
||||||
* SOLR-8626: 404 error when clicking nodes in cloud graph view in angular UI. (janhoy, Trey Grainger via shalin)
|
* SOLR-8626: 404 error when clicking nodes in cloud graph view in angular UI. (janhoy, Trey Grainger via shalin)
|
||||||
|
|
||||||
|
* SOLR-8777: Duplicate Solr process can cripple a running process. (Jessica Cheng Mallet, Scott Blum, shalin)
|
||||||
|
|
||||||
* SOLR-9254: GraphTermsQueryQParserPlugin throws NPE when field being searched is not present in segment
|
* SOLR-9254: GraphTermsQueryQParserPlugin throws NPE when field being searched is not present in segment
|
||||||
(Joel Bernstein)
|
(Joel Bernstein)
|
||||||
|
|
||||||
* SOLR-8657: Fix SolrRequestInfo error logs if QuerySenderListener is being used (Pascal Chollet,
|
* SOLR-8657: Fix SolrRequestInfo error logs if QuerySenderListener is being used (Pascal Chollet,
|
||||||
Tomás Fernández Löbbe)
|
Tomás Fernández Löbbe)
|
||||||
|
|
||||||
Optimizations
|
Optimizations
|
||||||
|
|
|
@ -25,8 +25,19 @@ import java.net.URLEncoder;
|
||||||
import java.net.UnknownHostException;
|
import java.net.UnknownHostException;
|
||||||
import java.nio.charset.StandardCharsets;
|
import java.nio.charset.StandardCharsets;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
import java.util.*;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Enumeration;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Locale;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Properties;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
@ -41,7 +52,25 @@ import org.apache.solr.cloud.overseer.OverseerAction;
|
||||||
import org.apache.solr.cloud.overseer.SliceMutator;
|
import org.apache.solr.cloud.overseer.SliceMutator;
|
||||||
import org.apache.solr.common.SolrException;
|
import org.apache.solr.common.SolrException;
|
||||||
import org.apache.solr.common.SolrException.ErrorCode;
|
import org.apache.solr.common.SolrException.ErrorCode;
|
||||||
import org.apache.solr.common.cloud.*;
|
import org.apache.solr.common.cloud.BeforeReconnect;
|
||||||
|
import org.apache.solr.common.cloud.ClusterState;
|
||||||
|
import org.apache.solr.common.cloud.ClusterStateUtil;
|
||||||
|
import org.apache.solr.common.cloud.DefaultConnectionStrategy;
|
||||||
|
import org.apache.solr.common.cloud.DefaultZkACLProvider;
|
||||||
|
import org.apache.solr.common.cloud.DefaultZkCredentialsProvider;
|
||||||
|
import org.apache.solr.common.cloud.DocCollection;
|
||||||
|
import org.apache.solr.common.cloud.OnReconnect;
|
||||||
|
import org.apache.solr.common.cloud.Replica;
|
||||||
|
import org.apache.solr.common.cloud.Slice;
|
||||||
|
import org.apache.solr.common.cloud.SolrZkClient;
|
||||||
|
import org.apache.solr.common.cloud.ZkACLProvider;
|
||||||
|
import org.apache.solr.common.cloud.ZkCmdExecutor;
|
||||||
|
import org.apache.solr.common.cloud.ZkConfigManager;
|
||||||
|
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
||||||
|
import org.apache.solr.common.cloud.ZkCredentialsProvider;
|
||||||
|
import org.apache.solr.common.cloud.ZkNodeProps;
|
||||||
|
import org.apache.solr.common.cloud.ZkStateReader;
|
||||||
|
import org.apache.solr.common.cloud.ZooKeeperException;
|
||||||
import org.apache.solr.common.params.CollectionParams;
|
import org.apache.solr.common.params.CollectionParams;
|
||||||
import org.apache.solr.common.params.CommonParams;
|
import org.apache.solr.common.params.CommonParams;
|
||||||
import org.apache.solr.common.params.SolrParams;
|
import org.apache.solr.common.params.SolrParams;
|
||||||
|
@ -642,6 +671,8 @@ public final class ZkController {
|
||||||
zkStateReader.createClusterStateWatchersAndUpdate();
|
zkStateReader.createClusterStateWatchersAndUpdate();
|
||||||
this.baseURL = zkStateReader.getBaseUrlForNodeName(this.nodeName);
|
this.baseURL = zkStateReader.getBaseUrlForNodeName(this.nodeName);
|
||||||
|
|
||||||
|
checkForExistingEphemeralNode();
|
||||||
|
|
||||||
// start the overseer first as following code may need it's processing
|
// start the overseer first as following code may need it's processing
|
||||||
if (!zkRunOnly) {
|
if (!zkRunOnly) {
|
||||||
overseerElector = new LeaderElector(zkClient);
|
overseerElector = new LeaderElector(zkClient);
|
||||||
|
@ -678,6 +709,39 @@ public final class ZkController {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void checkForExistingEphemeralNode() throws KeeperException, InterruptedException {
|
||||||
|
if (zkRunOnly) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
String nodeName = getNodeName();
|
||||||
|
String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
|
||||||
|
|
||||||
|
if (!zkClient.exists(nodePath, true)) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
final CountDownLatch deletedLatch = new CountDownLatch(1);
|
||||||
|
Stat stat = zkClient.exists(nodePath, event -> {
|
||||||
|
if (Watcher.Event.EventType.None.equals(event.getType())) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (Watcher.Event.EventType.NodeDeleted.equals(event.getType())) {
|
||||||
|
deletedLatch.countDown();
|
||||||
|
}
|
||||||
|
}, true);
|
||||||
|
|
||||||
|
if (stat == null) {
|
||||||
|
// znode suddenly disappeared but that's okay
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
boolean deleted = deletedLatch.await(zkClient.getSolrZooKeeper().getSessionTimeout() * 2, TimeUnit.MILLISECONDS);
|
||||||
|
if (!deleted) {
|
||||||
|
throw new SolrException(ErrorCode.SERVER_ERROR, "A previous ephemeral live node still exists. " +
|
||||||
|
"Solr cannot continue. Please ensure that no other Solr process using the same port is running already.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public void publishAndWaitForDownStates() throws KeeperException,
|
public void publishAndWaitForDownStates() throws KeeperException,
|
||||||
InterruptedException {
|
InterruptedException {
|
||||||
|
|
||||||
|
@ -752,33 +816,7 @@ public final class ZkController {
|
||||||
String nodeName = getNodeName();
|
String nodeName = getNodeName();
|
||||||
String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
|
String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
|
||||||
log.info("Register node as live in ZooKeeper:" + nodePath);
|
log.info("Register node as live in ZooKeeper:" + nodePath);
|
||||||
|
zkClient.makePath(nodePath, CreateMode.EPHEMERAL, true);
|
||||||
try {
|
|
||||||
boolean nodeDeleted = true;
|
|
||||||
try {
|
|
||||||
// we attempt a delete in the case of a quick server bounce -
|
|
||||||
// if there was not a graceful close, the node may exist
|
|
||||||
// until expiration timeout - so a node won't be created here because
|
|
||||||
// it exists, but eventually the node will be removed. So delete
|
|
||||||
// in case it exists and create a new node.
|
|
||||||
zkClient.delete(nodePath, -1, true);
|
|
||||||
} catch (KeeperException.NoNodeException e) {
|
|
||||||
// fine if there is nothing to delete
|
|
||||||
// TODO: annoying that ZK logs a warning on us
|
|
||||||
nodeDeleted = false;
|
|
||||||
}
|
|
||||||
if (nodeDeleted) {
|
|
||||||
log
|
|
||||||
.info("Found a previous node that still exists while trying to register a new live node "
|
|
||||||
+ nodePath + " - removing existing node to create another.");
|
|
||||||
}
|
|
||||||
zkClient.makePath(nodePath, CreateMode.EPHEMERAL, true);
|
|
||||||
} catch (KeeperException e) {
|
|
||||||
// it's okay if the node already exists
|
|
||||||
if (e.code() != KeeperException.Code.NODEEXISTS) {
|
|
||||||
throw e;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getNodeName() {
|
public String getNodeName() {
|
||||||
|
|
Loading…
Reference in New Issue