improve timeout cluster service listener

This commit is contained in:
kimchy 2010-07-20 13:50:19 +03:00
parent 4ac1409afa
commit 3c8cf68a17
9 changed files with 73 additions and 55 deletions

View File

@ -20,6 +20,7 @@
package org.elasticsearch.action.support.replication; package org.elasticsearch.action.support.replication;
import org.elasticsearch.ElasticSearchException; import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionResponse;
@ -332,6 +333,18 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
// make it threaded operation so we fork on the discovery listener thread // make it threaded operation so we fork on the discovery listener thread
request.operationThreaded(true); request.operationThreaded(true);
clusterService.add(request.timeout(), new TimeoutClusterStateListener() { clusterService.add(request.timeout(), new TimeoutClusterStateListener() {
@Override public void postAdded() {
if (start(true)) {
// if we managed to start and perform the operation on the primary, we can remove this listener
clusterService.remove(this);
}
}
@Override public void onClose() {
clusterService.remove(this);
listener.onFailure(new ElasticSearchIllegalStateException("node is shutting down"));
}
@Override public void clusterChanged(ClusterChangedEvent event) { @Override public void clusterChanged(ClusterChangedEvent event) {
if (start(true)) { if (start(true)) {
// if we managed to start and perform the operation on the primary, we can remove this listener // if we managed to start and perform the operation on the primary, we can remove this listener
@ -345,6 +358,7 @@ public abstract class TransportShardReplicationOperationAction<Request extends S
clusterService.remove(this); clusterService.remove(this);
return; return;
} }
clusterService.remove(this);
final PrimaryNotStartedActionException failure = new PrimaryNotStartedActionException(shardId, "Timeout waiting for [" + timeValue + "]"); final PrimaryNotStartedActionException failure = new PrimaryNotStartedActionException(shardId, "Timeout waiting for [" + timeValue + "]");
if (request.listenerThreaded()) { if (request.listenerThreaded()) {
threadPool.execute(new Runnable() { threadPool.execute(new Runnable() {

View File

@ -35,7 +35,5 @@ public interface ClusterService extends LifecycleComponent<ClusterService> {
void add(TimeValue timeout, TimeoutClusterStateListener listener); void add(TimeValue timeout, TimeoutClusterStateListener listener);
void remove(TimeoutClusterStateListener listener);
void submitStateUpdateTask(final String source, final ClusterStateUpdateTask updateTask); void submitStateUpdateTask(final String source, final ClusterStateUpdateTask updateTask);
} }

View File

@ -22,9 +22,15 @@ package org.elasticsearch.cluster;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
/** /**
* @author kimchy (Shay Banon) * An exception to cluster state listener that allows for timeouts and for post added notifications.
*
* @author kimchy (shay.banon)
*/ */
public interface TimeoutClusterStateListener extends ClusterStateListener { public interface TimeoutClusterStateListener extends ClusterStateListener {
void postAdded();
void onClose();
void onTimeout(TimeValue timeout); void onTimeout(TimeValue timeout);
} }

View File

@ -23,23 +23,27 @@ import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.cluster.*; import org.elasticsearch.cluster.*;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.timer.Timeout;
import org.elasticsearch.common.timer.TimerTask;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.jsr166y.LinkedTransferQueue;
import org.elasticsearch.discovery.DiscoveryService; import org.elasticsearch.discovery.DiscoveryService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.timer.TimerService;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.util.List; import java.util.List;
import java.util.Queue;
import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import static java.util.concurrent.Executors.*; import static java.util.concurrent.Executors.*;
import static org.elasticsearch.cluster.ClusterState.*; import static org.elasticsearch.cluster.ClusterState.*;
import static org.elasticsearch.common.unit.TimeValue.*;
import static org.elasticsearch.common.util.concurrent.EsExecutors.*; import static org.elasticsearch.common.util.concurrent.EsExecutors.*;
/** /**
@ -47,10 +51,10 @@ import static org.elasticsearch.common.util.concurrent.EsExecutors.*;
*/ */
public class InternalClusterService extends AbstractLifecycleComponent<ClusterService> implements ClusterService { public class InternalClusterService extends AbstractLifecycleComponent<ClusterService> implements ClusterService {
private final TimeValue timeoutInterval;
private final ThreadPool threadPool; private final ThreadPool threadPool;
private final TimerService timerService;
private final DiscoveryService discoveryService; private final DiscoveryService discoveryService;
private final TransportService transportService; private final TransportService transportService;
@ -59,45 +63,28 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
private final List<ClusterStateListener> clusterStateListeners = new CopyOnWriteArrayList<ClusterStateListener>(); private final List<ClusterStateListener> clusterStateListeners = new CopyOnWriteArrayList<ClusterStateListener>();
private final List<TimeoutHolder> clusterStateTimeoutListeners = new CopyOnWriteArrayList<TimeoutHolder>(); private final Queue<Tuple<Timeout, TimeoutClusterStateListener>> onGoingTimeouts = new LinkedTransferQueue<Tuple<Timeout, TimeoutClusterStateListener>>();
private volatile ScheduledFuture scheduledFuture;
private volatile ClusterState clusterState = newClusterStateBuilder().build(); private volatile ClusterState clusterState = newClusterStateBuilder().build();
@Inject public InternalClusterService(Settings settings, DiscoveryService discoveryService, TransportService transportService, ThreadPool threadPool) { @Inject public InternalClusterService(Settings settings, DiscoveryService discoveryService, TransportService transportService, ThreadPool threadPool,
TimerService timerService) {
super(settings); super(settings);
this.transportService = transportService; this.transportService = transportService;
this.discoveryService = discoveryService; this.discoveryService = discoveryService;
this.threadPool = threadPool; this.threadPool = threadPool;
this.timerService = timerService;
this.timeoutInterval = componentSettings.getAsTime("timeout_interval", timeValueMillis(500));
} }
@Override protected void doStart() throws ElasticSearchException { @Override protected void doStart() throws ElasticSearchException {
this.clusterState = newClusterStateBuilder().build(); this.clusterState = newClusterStateBuilder().build();
this.updateTasksExecutor = newSingleThreadExecutor(daemonThreadFactory(settings, "clusterService#updateTask")); this.updateTasksExecutor = newSingleThreadExecutor(daemonThreadFactory(settings, "clusterService#updateTask"));
scheduledFuture = threadPool.scheduleWithFixedDelay(new Runnable() {
@Override public void run() {
long timestamp = System.currentTimeMillis();
for (final TimeoutHolder holder : clusterStateTimeoutListeners) {
if ((timestamp - holder.timestamp) > holder.timeout.millis()) {
clusterStateTimeoutListeners.remove(holder);
InternalClusterService.this.threadPool.execute(new Runnable() {
@Override public void run() {
holder.listener.onTimeout(holder.timeout);
}
});
}
}
}
}, timeoutInterval);
} }
@Override protected void doStop() throws ElasticSearchException { @Override protected void doStop() throws ElasticSearchException {
scheduledFuture.cancel(false); for (Tuple<Timeout, TimeoutClusterStateListener> onGoingTimeout : onGoingTimeouts) {
for (TimeoutHolder holder : clusterStateTimeoutListeners) { onGoingTimeout.v1().cancel();
holder.listener.onTimeout(holder.timeout); onGoingTimeout.v2().onClose();
} }
updateTasksExecutor.shutdown(); updateTasksExecutor.shutdown();
try { try {
@ -122,12 +109,16 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
clusterStateListeners.remove(listener); clusterStateListeners.remove(listener);
} }
public void add(TimeValue timeout, TimeoutClusterStateListener listener) { public void add(TimeValue timeout, final TimeoutClusterStateListener listener) {
clusterStateTimeoutListeners.add(new TimeoutHolder(listener, System.currentTimeMillis(), timeout)); Timeout timerTimeout = timerService.newTimeout(new NotifyTimeout(listener, timeout), timeout, TimerService.ExecutionType.THREADED);
} onGoingTimeouts.add(new Tuple<Timeout, TimeoutClusterStateListener>(timerTimeout, listener));
clusterStateListeners.add(listener);
public void remove(TimeoutClusterStateListener listener) { // call the post added notification on the same event thread
clusterStateTimeoutListeners.remove(new TimeoutHolder(listener, -1, null)); updateTasksExecutor.execute(new Runnable() {
@Override public void run() {
listener.postAdded();
}
});
} }
public void submitStateUpdateTask(final String source, final ClusterStateUpdateTask updateTask) { public void submitStateUpdateTask(final String source, final ClusterStateUpdateTask updateTask) {
@ -194,9 +185,6 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
} }
} }
for (TimeoutHolder timeoutHolder : clusterStateTimeoutListeners) {
timeoutHolder.listener.clusterChanged(clusterChangedEvent);
}
for (ClusterStateListener listener : clusterStateListeners) { for (ClusterStateListener listener : clusterStateListeners) {
listener.clusterChanged(clusterChangedEvent); listener.clusterChanged(clusterChangedEvent);
} }
@ -226,23 +214,21 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
}); });
} }
private static class TimeoutHolder { private class NotifyTimeout implements TimerTask {
final TimeoutClusterStateListener listener; final TimeoutClusterStateListener listener;
final long timestamp;
final TimeValue timeout; final TimeValue timeout;
private TimeoutHolder(TimeoutClusterStateListener listener, long timestamp, TimeValue timeout) { private NotifyTimeout(TimeoutClusterStateListener listener, TimeValue timeout) {
this.listener = listener; this.listener = listener;
this.timestamp = timestamp;
this.timeout = timeout; this.timeout = timeout;
} }
@Override public int hashCode() { @Override public void run(Timeout timeout) throws Exception {
return listener.hashCode(); if (timeout.isCancelled()) {
} return;
}
@Override public boolean equals(Object obj) { listener.onTimeout(this.timeout);
return ((TimeoutHolder) obj).listener == listener; // note, we rely on the listener to remove itself in case of timeout if needed
} }
} }
} }

View File

@ -98,4 +98,10 @@ class RecoveryFileChunkRequest implements Streamable {
out.writeVInt(contentLength); out.writeVInt(contentLength);
out.writeBytes(content, 0, contentLength); out.writeBytes(content, 0, contentLength);
} }
@Override public String toString() {
return shardId + ": name='" + name + '\'' +
", position=" + position +
", length=" + length;
}
} }

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.shard.recovery; package org.elasticsearch.index.shard.recovery;
import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.IndexOutput;
import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.StopWatch; import org.elasticsearch.common.StopWatch;
import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.component.AbstractComponent;
@ -338,6 +339,9 @@ public class RecoveryTarget extends AbstractComponent {
} else { } else {
indexOutput = onGoingRecovery.openIndexOutputs.get(request.name()); indexOutput = onGoingRecovery.openIndexOutputs.get(request.name());
} }
if (indexOutput == null) {
throw new ElasticSearchIllegalStateException("No ongoing output file to write to, request: " + request);
}
synchronized (indexOutput) { synchronized (indexOutput) {
try { try {
indexOutput.writeBytes(request.content(), request.contentLength()); indexOutput.writeBytes(request.content(), request.contentLength());

View File

@ -107,7 +107,7 @@ public class TimerService extends AbstractComponent {
} }
@Override public void run(final Timeout timeout) throws Exception { @Override public void run(final Timeout timeout) throws Exception {
threadPool.execute(new Runnable() { threadPool.cached().execute(new Runnable() {
@Override public void run() { @Override public void run() {
try { try {
task.run(timeout); task.run(timeout);

View File

@ -87,10 +87,14 @@ public class LocalTransport extends AbstractLifecycleComponent<Transport> implem
@Override protected void doStop() throws ElasticSearchException { @Override protected void doStop() throws ElasticSearchException {
transports.remove(localAddress); transports.remove(localAddress);
// now, go over all the transports connected to me, and raise disconnected event // now, go over all the transports connected to me, and raise disconnected event
for (LocalTransport targetTransport : transports.values()) { for (final LocalTransport targetTransport : transports.values()) {
for (Map.Entry<DiscoveryNode, LocalTransport> entry : targetTransport.connectedNodes.entrySet()) { for (final Map.Entry<DiscoveryNode, LocalTransport> entry : targetTransport.connectedNodes.entrySet()) {
if (entry.getValue() == this) { if (entry.getValue() == this) {
targetTransport.disconnectFromNode(entry.getKey()); targetTransport.threadPool().cached().execute(new Runnable() {
@Override public void run() {
targetTransport.disconnectFromNode(entry.getKey());
}
});
} }
} }
} }

View File

@ -33,7 +33,7 @@ import static org.hamcrest.MatcherAssert.*;
import static org.hamcrest.Matchers.*; import static org.hamcrest.Matchers.*;
/** /**
* @author kimchy (Shay Banon) * @author kimchy (shay.banon)
*/ */
public class SimpleDataNodesTests extends AbstractNodesTests { public class SimpleDataNodesTests extends AbstractNodesTests {