RTR: Ensure that there is only one cleanup task scheduled for a worker at once.

This is accomplished by making sure that scheduleTasksCleanupForWorker is only called
from the PathChildrenCache event thread, having it cancel existing cleanup tasks when
it adds a new one, and having tasks check on finish that the thing they are removing
from the task list is actually themselves.
This commit is contained in:
Gian Merlino 2015-10-27 21:16:37 -07:00
parent 59872bd0cd
commit 513bc76252

View File

@ -35,6 +35,9 @@ import com.google.common.io.ByteSource;
import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListenableScheduledFuture;
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.SettableFuture; import com.google.common.util.concurrent.SettableFuture;
import com.metamx.common.ISE; import com.metamx.common.ISE;
import com.metamx.common.RE; import com.metamx.common.RE;
@ -140,7 +143,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
private volatile boolean started = false; private volatile boolean started = false;
private final ScheduledExecutorService cleanupExec; private final ListeningScheduledExecutorService cleanupExec;
private final ConcurrentMap<String, ScheduledFuture> removedWorkerCleanups = new ConcurrentHashMap<>(); private final ConcurrentMap<String, ScheduledFuture> removedWorkerCleanups = new ConcurrentHashMap<>();
@ -164,7 +167,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
this.workerPathCache = pathChildrenCacheFactory.make(cf, indexerZkConfig.getAnnouncementsPath()); this.workerPathCache = pathChildrenCacheFactory.make(cf, indexerZkConfig.getAnnouncementsPath());
this.httpClient = httpClient; this.httpClient = httpClient;
this.workerConfigRef = workerConfigRef; this.workerConfigRef = workerConfigRef;
this.cleanupExec = cleanupExec; this.cleanupExec = MoreExecutors.listeningDecorator(cleanupExec);
} }
@LifecycleStart @LifecycleStart
@ -235,6 +238,25 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
removeWorker(worker); removeWorker(worker);
break; break;
case INITIALIZED: case INITIALIZED:
// Schedule cleanup for task status of the workers that might have disconnected while overlord was not running
List<String> workers;
try {
workers = cf.getChildren().forPath(indexerZkConfig.getStatusPath());
} catch (KeeperException.NoNodeException e) {
// statusPath doesn't exist yet; can occur if no middleManagers have started.
workers = ImmutableList.of();
}
for (String workerId : workers) {
if (!zkWorkers.containsKey(workerId)
&& cf.checkExists().forPath(JOINER.join(indexerZkConfig.getAnnouncementsPath(), workerId)) == null) {
scheduleTasksCleanupForWorker(
workerId,
cf.getChildren()
.forPath(JOINER.join(indexerZkConfig.getStatusPath(), workerId))
);
}
}
synchronized (waitingForMonitor) { synchronized (waitingForMonitor) {
waitingFor.decrement(); waitingFor.decrement();
waitingForMonitor.notifyAll(); waitingForMonitor.notifyAll();
@ -251,26 +273,6 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
waitingForMonitor.wait(); waitingForMonitor.wait();
} }
} }
// Schedule cleanup for task status of the workers that might have disconnected while overlord was not running
List<String> workers;
try {
workers = cf.getChildren().forPath(indexerZkConfig.getStatusPath());
}
catch (KeeperException.NoNodeException e) {
// statusPath doesn't exist yet; can occur if no middleManagers have started.
workers = ImmutableList.of();
}
for (String worker : workers) {
if (!zkWorkers.containsKey(worker)
&& cf.checkExists().forPath(JOINER.join(indexerZkConfig.getAnnouncementsPath(), worker)) == null) {
scheduleTasksCleanupForWorker(
worker,
cf.getChildren()
.forPath(JOINER.join(indexerZkConfig.getStatusPath(), worker))
);
}
}
started = true; started = true;
} }
catch (Exception e) { catch (Exception e) {
@ -698,6 +700,16 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
} }
} }
private boolean cancelWorkerCleanup(String workerHost)
{
ScheduledFuture previousCleanup = removedWorkerCleanups.remove(workerHost);
if (previousCleanup != null) {
log.info("Cancelling Worker[%s] scheduled task cleanup", workerHost);
previousCleanup.cancel(false);
}
return previousCleanup != null;
}
/** /**
* When a new worker appears, listeners are registered for status changes associated with tasks assigned to * When a new worker appears, listeners are registered for status changes associated with tasks assigned to
* the worker. Status changes indicate the creation or completion of a task. * the worker. Status changes indicate the creation or completion of a task.
@ -712,11 +724,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
log.info("Worker[%s] reportin' for duty!", worker.getHost()); log.info("Worker[%s] reportin' for duty!", worker.getHost());
try { try {
ScheduledFuture previousCleanup = removedWorkerCleanups.remove(worker.getHost()); cancelWorkerCleanup(worker.getHost());
if (previousCleanup != null) {
log.info("Cancelling Worker[%s] scheduled task cleanup", worker.getHost());
previousCleanup.cancel(false);
}
final String workerStatusPath = JOINER.join(indexerZkConfig.getStatusPath(), worker.getHost()); final String workerStatusPath = JOINER.join(indexerZkConfig.getStatusPath(), worker.getHost());
final PathChildrenCache statusCache = pathChildrenCacheFactory.make(cf, workerStatusPath); final PathChildrenCache statusCache = pathChildrenCacheFactory.make(cf, workerStatusPath);
@ -880,10 +888,17 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
lazyWorkers.remove(worker.getHost()); lazyWorkers.remove(worker.getHost());
} }
/**
* Schedule a task that will, at some point in the future, clean up znodes and issue failures for "tasksToFail"
* if they are being run by "worker".
*/
private void scheduleTasksCleanupForWorker(final String worker, final List<String> tasksToFail) private void scheduleTasksCleanupForWorker(final String worker, final List<String> tasksToFail)
{ {
removedWorkerCleanups.put( // This method is only called from the PathChildrenCache event handler, so this may look like a race,
worker, cleanupExec.schedule( // but is actually not.
cancelWorkerCleanup(worker);
final ListenableScheduledFuture<?> cleanupTask = cleanupExec.schedule(
new Runnable() new Runnable()
{ {
@Override @Override
@ -921,14 +936,31 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
log.makeAlert("Exception while cleaning up worker[%s]", worker).emit(); log.makeAlert("Exception while cleaning up worker[%s]", worker).emit();
throw Throwables.propagate(e); throw Throwables.propagate(e);
} }
finally {
removedWorkerCleanups.remove(worker);
}
} }
}, },
config.getTaskCleanupTimeout().toStandardDuration().getMillis(), config.getTaskCleanupTimeout().toStandardDuration().getMillis(),
TimeUnit.MILLISECONDS TimeUnit.MILLISECONDS
) );
removedWorkerCleanups.put(worker, cleanupTask);
// Remove this entry from removedWorkerCleanups when done, if it's actually the one in there.
Futures.addCallback(
cleanupTask,
new FutureCallback<Object>()
{
@Override
public void onSuccess(Object result)
{
removedWorkerCleanups.remove(worker, cleanupTask);
}
@Override
public void onFailure(Throwable t)
{
removedWorkerCleanups.remove(worker, cleanupTask);
}
}
); );
} }
@ -1009,7 +1041,6 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
return assignedTasks; return assignedTasks;
} }
// Used for tests
public List<ZkWorker> getLazyWorkers() public List<ZkWorker> getLazyWorkers()
{ {
return ImmutableList.copyOf(lazyWorkers.values()); return ImmutableList.copyOf(lazyWorkers.values());