mirror of https://github.com/apache/lucene.git
SOLR-13339: Prevent recovery, fetching index being kicked off after SolrCores already closed
This commit is contained in:
parent
02c4503f8c
commit
a67a941e19
|
@ -172,6 +172,8 @@ Bug Fixes
|
|||
|
||||
* SOLR-13388: Fix FileExchangeRateProvider to be a public class, as it appears in schema.xml (Uwe Schindler)
|
||||
|
||||
* SOLR-13339: Prevent recovery, fetching index being kicked off after SolrCores already closed (Cao Manh Dat)
|
||||
|
||||
Improvements
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -563,45 +563,59 @@ public class ZkController implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
public void preClose() {
|
||||
this.isClosed = true;
|
||||
|
||||
try {
|
||||
this.removeEphemeralLiveNode();
|
||||
} catch (AlreadyClosedException | SessionExpiredException | KeeperException.ConnectionLossException e) {
|
||||
|
||||
} catch (Exception e) {
|
||||
log.warn("Error removing live node. Continuing to close CoreContainer", e);
|
||||
}
|
||||
|
||||
try {
|
||||
if (getZkClient().getConnectionManager().isConnected()) {
|
||||
log.info("Publish this node as DOWN...");
|
||||
publishNodeAsDown(getNodeName());
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.warn("Error publishing nodes as down. Continuing to close CoreContainer", e);
|
||||
}
|
||||
|
||||
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("preCloseThreadPool"));
|
||||
|
||||
try {
|
||||
synchronized (collectionToTerms) {
|
||||
customThreadPool.submit(() -> collectionToTerms.values().parallelStream().forEach(ZkCollectionTerms::close));
|
||||
}
|
||||
|
||||
customThreadPool.submit(() -> replicateFromLeaders.values().parallelStream().forEach(ReplicateFromLeader::stopReplication));
|
||||
} finally {
|
||||
ExecutorUtil.shutdownAndAwaitTermination(customThreadPool);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the underlying ZooKeeper client.
|
||||
*/
|
||||
public void close() {
|
||||
this.isClosed = true;
|
||||
if (!this.isClosed)
|
||||
preClose();
|
||||
|
||||
ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("closeThreadPool"));
|
||||
|
||||
customThreadPool.submit(() -> Collections.singleton(overseerElector.getContext()).parallelStream().forEach(c -> {
|
||||
IOUtils.closeQuietly(c);
|
||||
}));
|
||||
customThreadPool.submit(() -> Collections.singleton(overseerElector.getContext()).parallelStream().forEach(IOUtils::closeQuietly));
|
||||
|
||||
customThreadPool.submit(() -> Collections.singleton(overseer).parallelStream().forEach(c -> {
|
||||
IOUtils.closeQuietly(c);
|
||||
}));
|
||||
customThreadPool.submit(() -> Collections.singleton(overseer).parallelStream().forEach(IOUtils::closeQuietly));
|
||||
|
||||
synchronized (collectionToTerms) {
|
||||
customThreadPool.submit(() -> collectionToTerms.values().parallelStream().forEach(c -> {
|
||||
c.close();
|
||||
}));
|
||||
}
|
||||
try {
|
||||
|
||||
customThreadPool.submit(() -> replicateFromLeaders.values().parallelStream().forEach(c -> {
|
||||
c.stopReplication();
|
||||
}));
|
||||
|
||||
customThreadPool.submit(() -> electionContexts.values().parallelStream().forEach(c -> {
|
||||
IOUtils.closeQuietly(c);
|
||||
}));
|
||||
customThreadPool.submit(() -> electionContexts.values().parallelStream().forEach(IOUtils::closeQuietly));
|
||||
|
||||
} finally {
|
||||
|
||||
customThreadPool.submit(() -> Collections.singleton(cloudSolrClient).parallelStream().forEach(c -> {
|
||||
IOUtils.closeQuietly(c);
|
||||
}));
|
||||
customThreadPool.submit(() -> Collections.singleton(cloudManager).parallelStream().forEach(c -> {
|
||||
IOUtils.closeQuietly(c);
|
||||
}));
|
||||
customThreadPool.submit(() -> Collections.singleton(cloudSolrClient).parallelStream().forEach(IOUtils::closeQuietly));
|
||||
customThreadPool.submit(() -> Collections.singleton(cloudManager).parallelStream().forEach(IOUtils::closeQuietly));
|
||||
|
||||
try {
|
||||
try {
|
||||
|
|
|
@ -114,8 +114,6 @@ import org.apache.solr.util.OrderedExecutor;
|
|||
import org.apache.solr.util.RefCounted;
|
||||
import org.apache.solr.util.stats.MetricUtils;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.KeeperException.ConnectionLossException;
|
||||
import org.apache.zookeeper.KeeperException.SessionExpiredException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -887,26 +885,7 @@ public class CoreContainer {
|
|||
try {
|
||||
if (isZooKeeperAware()) {
|
||||
cancelCoreRecoveries();
|
||||
|
||||
if (isZooKeeperAware()) {
|
||||
cancelCoreRecoveries();
|
||||
try {
|
||||
zkSys.zkController.removeEphemeralLiveNode();
|
||||
} catch (AlreadyClosedException | SessionExpiredException | ConnectionLossException e) {
|
||||
|
||||
} catch (Exception e) {
|
||||
log.warn("Error removing live node. Continuing to close CoreContainer", e);
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
if (zkSys.zkController.getZkClient().getConnectionManager().isConnected()) {
|
||||
log.info("Publish this node as DOWN...");
|
||||
zkSys.zkController.publishNodeAsDown(zkSys.zkController.getNodeName());
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.warn("Error publishing nodes as down. Continuing to close CoreContainer", e);
|
||||
}
|
||||
zkSys.zkController.preClose();
|
||||
}
|
||||
|
||||
ExecutorUtil.shutdownAndAwaitTermination(coreContainerWorkExecutor);
|
||||
|
|
Loading…
Reference in New Issue