mirror of https://github.com/apache/lucene.git
bring back interruptions on executor shutdown
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1426791 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
47fb1898da
commit
5065cdebd3
|
@ -271,7 +271,11 @@ public class SyncStrategy {
|
|||
|
||||
public void close() {
|
||||
this.isClosed = true;
|
||||
ExecutorUtil.shutdownAndAwaitTermination(recoveryCmdExecutor);
|
||||
try {
|
||||
ExecutorUtil.shutdownNowAndAwaitTermination(recoveryCmdExecutor);
|
||||
} catch (Throwable e) {
|
||||
SolrException.log(log, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void requestRecovery(final ZkNodeProps leaderProps, final String baseUrl, final String coreName) throws SolrServerException, IOException {
|
||||
|
|
|
@ -248,6 +248,7 @@ public final class ZkController {
|
|||
final CurrentCoreDescriptorProvider registerOnReconnect, boolean updateLastPublished) {
|
||||
List<CoreDescriptor> descriptors = registerOnReconnect
|
||||
.getCurrentDescriptors();
|
||||
if (isClosed) return;
|
||||
if (descriptors != null) {
|
||||
// before registering as live, make sure everyone is in a
|
||||
// down state
|
||||
|
|
|
@ -678,7 +678,7 @@ public class CoreContainer
|
|||
}
|
||||
} finally {
|
||||
if (coreLoadExecutor != null) {
|
||||
ExecutorUtil.shutdownAndAwaitTermination(coreLoadExecutor);
|
||||
ExecutorUtil.shutdownNowAndAwaitTermination(coreLoadExecutor);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -66,6 +66,7 @@ import org.apache.solr.common.SolrException;
|
|||
import org.apache.solr.common.params.CommonParams;
|
||||
import org.apache.solr.common.params.CommonParams.EchoParamStyle;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.ExecutorUtil;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
import org.apache.solr.handler.SnapPuller;
|
||||
|
@ -961,7 +962,7 @@ public final class SolrCore implements SolrInfoMBean {
|
|||
|
||||
try {
|
||||
infoRegistry.clear();
|
||||
} catch (Exception e) {
|
||||
} catch (Throwable e) {
|
||||
SolrException.log(log, e);
|
||||
}
|
||||
|
||||
|
@ -984,22 +985,11 @@ public final class SolrCore implements SolrInfoMBean {
|
|||
}
|
||||
|
||||
try {
|
||||
searcherExecutor.shutdown();
|
||||
if (!searcherExecutor.awaitTermination(60, TimeUnit.SECONDS)) {
|
||||
log.error("Timeout waiting for searchExecutor to terminate");
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
searcherExecutor.shutdownNow();
|
||||
try {
|
||||
if (!searcherExecutor.awaitTermination(30, TimeUnit.SECONDS)) {
|
||||
log.error("Timeout waiting for searchExecutor to terminate");
|
||||
}
|
||||
} catch (InterruptedException e2) {
|
||||
SolrException.log(log, e2);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
ExecutorUtil.shutdownAndAwaitTermination(searcherExecutor);
|
||||
} catch (Throwable e) {
|
||||
SolrException.log(log, e);
|
||||
}
|
||||
|
||||
try {
|
||||
// Since we waited for the searcherExecutor to shut down,
|
||||
// there should be no more searchers warming in the background
|
||||
|
|
|
@ -1584,9 +1584,22 @@ public class SnapPuller {
|
|||
}
|
||||
|
||||
public void destroy() {
|
||||
if (executorService != null) executorService.shutdown();
|
||||
abortPull();
|
||||
if (executorService != null) ExecutorUtil.shutdownAndAwaitTermination(executorService);
|
||||
try {
|
||||
if (executorService != null) executorService.shutdown();
|
||||
} catch (Throwable e) {
|
||||
SolrException.log(LOG, e);
|
||||
}
|
||||
try {
|
||||
abortPull();
|
||||
} catch (Throwable e) {
|
||||
SolrException.log(LOG, e);
|
||||
}
|
||||
try {
|
||||
if (executorService != null) ExecutorUtil
|
||||
.shutdownNowAndAwaitTermination(executorService);
|
||||
} catch (Throwable e) {
|
||||
SolrException.log(LOG, e);
|
||||
}
|
||||
}
|
||||
|
||||
String getMasterUrl() {
|
||||
|
|
|
@ -156,7 +156,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements Plug
|
|||
@Override
|
||||
public void close() {
|
||||
try {
|
||||
ExecutorUtil.shutdownAndAwaitTermination(commExecutor);
|
||||
ExecutorUtil.shutdownNowAndAwaitTermination(commExecutor);
|
||||
} catch (Throwable e) {
|
||||
SolrException.log(log, e);
|
||||
}
|
||||
|
|
|
@ -820,8 +820,8 @@ public class UpdateLog implements PluginInfoInitialized {
|
|||
public void close(boolean committed, boolean deleteOnClose) {
|
||||
synchronized (this) {
|
||||
try {
|
||||
ExecutorUtil.shutdownAndAwaitTermination(recoveryExecutor);
|
||||
} catch (Exception e) {
|
||||
ExecutorUtil.shutdownNowAndAwaitTermination(recoveryExecutor);
|
||||
} catch (Throwable e) {
|
||||
SolrException.log(log, e);
|
||||
}
|
||||
|
||||
|
|
|
@ -60,7 +60,7 @@ public class UpdateShardHandler {
|
|||
|
||||
public void close() {
|
||||
try {
|
||||
ExecutorUtil.shutdownAndAwaitTermination(cmdDistribExecutor);
|
||||
ExecutorUtil.shutdownNowAndAwaitTermination(cmdDistribExecutor);
|
||||
} catch (Throwable e) {
|
||||
SolrException.log(log, e);
|
||||
}
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.slf4j.LoggerFactory;
|
|||
public class ExecutorUtil {
|
||||
public static Logger log = LoggerFactory.getLogger(ExecutorUtil.class);
|
||||
|
||||
// really? lucene/solr do not like to be interrupted
|
||||
public static void shutdownNowAndAwaitTermination(ExecutorService pool) {
|
||||
pool.shutdown(); // Disable new tasks from being submitted
|
||||
pool.shutdownNow(); // Cancel currently executing tasks
|
||||
|
@ -52,7 +51,7 @@ public class ExecutorUtil {
|
|||
while (!shutdown) {
|
||||
try {
|
||||
// Wait a while for existing tasks to terminate
|
||||
shutdown = pool.awaitTermination(60, TimeUnit.SECONDS);
|
||||
shutdown = pool.awaitTermination(30, TimeUnit.SECONDS);
|
||||
} catch (InterruptedException ie) {
|
||||
// Preserve interrupt status
|
||||
Thread.currentThread().interrupt();
|
||||
|
|
Loading…
Reference in New Issue