mirror of https://github.com/apache/lucene.git
SOLR-13043: cleanup suspicious ExecutorService lifecycles in MiniSolrCloudCluster
This commit is contained in:
parent
f844461357
commit
df07fb33bb
|
@ -122,9 +122,6 @@ public class MiniSolrCloudCluster {
|
|||
private final CloudSolrClient solrClient;
|
||||
private final JettyConfig jettyConfig;
|
||||
|
||||
private final ExecutorService executorLauncher = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("jetty-launcher"));
|
||||
private final ExecutorService executorCloser = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("jetty-closer"));
|
||||
|
||||
private final AtomicInteger nodeIds = new AtomicInteger();
|
||||
|
||||
/**
|
||||
|
@ -272,7 +269,9 @@ public class MiniSolrCloudCluster {
|
|||
startups.add(() -> startJettySolrRunner(newNodeName(), jettyConfig.context, jettyConfig));
|
||||
}
|
||||
|
||||
final ExecutorService executorLauncher = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("jetty-launcher"));
|
||||
Collection<Future<JettySolrRunner>> futures = executorLauncher.invokeAll(startups);
|
||||
ExecutorUtil.shutdownAndAwaitTermination(executorLauncher);
|
||||
Exception startupError = checkForExceptions("Error starting up MiniSolrCloudCluster", futures);
|
||||
if (startupError != null) {
|
||||
try {
|
||||
|
@ -294,10 +293,6 @@ public class MiniSolrCloudCluster {
|
|||
|
||||
private void waitForAllNodes(int numServers, int timeoutSeconds) throws IOException, InterruptedException, TimeoutException {
|
||||
|
||||
executorLauncher.shutdown();
|
||||
|
||||
ExecutorUtil.shutdownAndAwaitTermination(executorLauncher);
|
||||
|
||||
int numRunning = 0;
|
||||
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
|
||||
|
||||
|
@ -327,10 +322,6 @@ public class MiniSolrCloudCluster {
|
|||
public void waitForNode(JettySolrRunner jetty, int timeoutSeconds)
|
||||
throws IOException, InterruptedException, TimeoutException {
|
||||
|
||||
executorLauncher.shutdown();
|
||||
|
||||
ExecutorUtil.shutdownAndAwaitTermination(executorLauncher);
|
||||
|
||||
ZkStateReader reader = getSolrClient().getZkStateReader();
|
||||
|
||||
reader.waitForLiveNodes(30, TimeUnit.SECONDS, (o, n) -> n.contains(jetty.getNodeName()));
|
||||
|
@ -577,21 +568,19 @@ public class MiniSolrCloudCluster {
|
|||
try {
|
||||
|
||||
IOUtils.closeQuietly(solrClient);
|
||||
// accept no new tasks
|
||||
executorLauncher.shutdown();
|
||||
List<Callable<JettySolrRunner>> shutdowns = new ArrayList<>(jettys.size());
|
||||
for (final JettySolrRunner jetty : jettys) {
|
||||
shutdowns.add(() -> stopJettySolrRunner(jetty));
|
||||
}
|
||||
jettys.clear();
|
||||
final ExecutorService executorCloser = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("jetty-closer"));
|
||||
Collection<Future<JettySolrRunner>> futures = executorCloser.invokeAll(shutdowns);
|
||||
ExecutorUtil.shutdownAndAwaitTermination(executorCloser);
|
||||
Exception shutdownError = checkForExceptions("Error shutting down MiniSolrCloudCluster", futures);
|
||||
if (shutdownError != null) {
|
||||
throw shutdownError;
|
||||
}
|
||||
} finally {
|
||||
ExecutorUtil.shutdownAndAwaitTermination(executorLauncher);
|
||||
ExecutorUtil.shutdownAndAwaitTermination(executorCloser);
|
||||
try {
|
||||
if (!externalZkServer) {
|
||||
zkServer.shutdown();
|
||||
|
|
Loading…
Reference in New Issue