SOLR-11171 Catch AlreadyClosedException when trying to add triggers while closing.

This commit is contained in:
Andrzej Bialecki 2017-07-31 23:01:27 +02:00
parent 4e7af72464
commit 19b06e8435
1 changed files with 18 additions and 8 deletions

View File

@ -28,6 +28,7 @@ import java.util.Set;
import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig; import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.cloud.ZkController; import org.apache.solr.cloud.ZkController;
@ -72,7 +73,7 @@ public class OverseerTriggerThread implements Runnable, Closeable {
private Map<String, AutoScaling.Trigger> activeTriggers = new HashMap<>(); private Map<String, AutoScaling.Trigger> activeTriggers = new HashMap<>();
private boolean isClosed = false; private volatile boolean isClosed = false;
private AutoScalingConfig autoScalingConfig; private AutoScalingConfig autoScalingConfig;
@ -178,15 +179,24 @@ public class OverseerTriggerThread implements Runnable, Closeable {
// absent then clean up old nodeLost / nodeAdded markers // absent then clean up old nodeLost / nodeAdded markers
boolean cleanOldNodeLostMarkers = true; boolean cleanOldNodeLostMarkers = true;
boolean cleanOldNodeAddedMarkers = true; boolean cleanOldNodeAddedMarkers = true;
// add new triggers and/or replace and close the replaced triggers try {
for (Map.Entry<String, AutoScaling.Trigger> entry : copy.entrySet()) { // add new triggers and/or replace and close the replaced triggers
if (entry.getValue().getEventType().equals(TriggerEventType.NODELOST)) { for (Map.Entry<String, AutoScaling.Trigger> entry : copy.entrySet()) {
cleanOldNodeLostMarkers = false; if (entry.getValue().getEventType().equals(TriggerEventType.NODELOST)) {
cleanOldNodeLostMarkers = false;
}
if (entry.getValue().getEventType().equals(TriggerEventType.NODEADDED)) {
cleanOldNodeAddedMarkers = false;
}
scheduledTriggers.add(entry.getValue());
} }
if (entry.getValue().getEventType().equals(TriggerEventType.NODEADDED)) { } catch (AlreadyClosedException e) {
cleanOldNodeAddedMarkers = false; // this _should_ mean that we're closing, complain loudly if that's not the case
if (isClosed) {
return;
} else {
throw new IllegalStateException("Caught AlreadyClosedException from ScheduledTriggers, but we're not closed yet!", e);
} }
scheduledTriggers.add(entry.getValue());
} }
if (cleanOldNodeLostMarkers) { if (cleanOldNodeLostMarkers) {
log.debug("-- clean old nodeLost markers"); log.debug("-- clean old nodeLost markers");