目录
前言
接着上两篇博文:Druid.io index_realtime实时任务源码分析, druid.io集群与tranquility对zookeeper的使用(1),本文主要说明下zookeeper在实时任务运行过程中的作用,并直接结合zknode 和 源码来说明
主要是overlord
,middleManager
,peon
这三个进程借助zookeeper进行进程间的通信,当然是利用zookeeper的watch机制,本文力求用log,图,源码来说明的更清楚,如果任何问题或不足之处,欢迎评论之处和交流
如下图,左侧是zookeeper上druid任务相关的节点,右侧是三个进程及内部重要的类
middleManager启动 : znode创建,workers维护
在middleManager正常启动后(middleManager也称为worker),zookeeper上会创建相关节点,具体是
/druid/indexer/announcements/workerHost:ip
临时节点/druid/indexer/tasks/workerHost:ip
持久节点/druid/indexer/status/workerHost:ip
持久节点
overlord 的 io.druid.indexing.overlord.RemoteTaskRunner
对znode 显然是有监听的, 见io.druid.indexing.overlord.RemoteTaskRunner#start
通过workerPathCache
对/druid/indexer/announcements
进行了监听,这样overlord就能维持内部对一个可用worker
列表,用来后续的任务分发
//this.workerPathCache = pathChildrenCacheFactory.build().make(cf, indexerZkConfig.getAnnouncementsPath());
// Add listener for creation/deletion of workers
workerPathCache.getListenable().addListener(
new PathChildrenCacheListener()
{
@Override
public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception
{
final Worker worker;
switch (event.getType()) {
case CHILD_ADDED:
worker = jsonMapper.readValue(
event.getData().getData(),
Worker.class
);
synchronized (waitingForMonitor) {
waitingFor.increment();
}
Futures.addCallback(
addWorker(worker),
new FutureCallback<ZkWorker>()
{
@Override
public void onSuccess(ZkWorker zkWorker)
{
synchronized (waitingForMonitor) {
waitingFor.decrement();
waitingForMonitor.notifyAll();
}
}
@Override
public void onFailure(Throwable throwable)
{
synchronized (waitingForMonitor) {
waitingFor.decrement();
waitingForMonitor.notifyAll();
}
}
}
);
break;
case CHILD_UPDATED:
worker = jsonMapper.readValue(
event.getData().getData(),
Worker.class
);
updateWorker(worker);
break;
case CHILD_REMOVED:
worker = jsonMapper.readValue(
event.getData().getData(),
Worker.class
);
removeWorker(worker);
break;
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) {
final String workerAnnouncePath = JOINER.join(indexerZkConfig.getAnnouncementsPath(), workerId);
final String workerStatusPath = JOINER.join(indexerZkConfig.getStatusPath(), workerId);
if (!zkWorkers.containsKey(workerId) && cf.checkExists().forPath(workerAnnouncePath) == null) {
try {
scheduleTasksCleanupForWorker(workerId, cf.getChildren().forPath(workerStatusPath));
}
catch (Exception e) {
log.warn(
e,
"Could not schedule cleanup for worker[%s] during startup (maybe someone removed the status znode[%s]?). Skipping.",
workerId,
workerStatusPath
);
}
}
}
synchronized (waitingForMonitor) {
waitingFor.decrement();
waitingForMonitor.notifyAll();
}
break;
case CONNECTION_SUSPENDED:
case CONNECTION_RECONNECTED:
case CONNECTION_LOST:
// do nothing
}
}
}
);
workerPathCache.start(PathChildrenCache.StartMode.POST_INITIALIZED_EVENT);
源码调试log 和 zk 如下
overlord接收任务并分配一个worker
在overlord的RemoteTaskRunner
进行了任务worker的分配选择,见io.druid.indexing.overlord.RemoteTaskRunner#tryAssignTask
如果分配成功,则是发布该任务,见io.druid.indexing.overlord.RemoteTaskRunner#announceTask
,这里创建了一个临时节点,如下
overlord给任务分配好了worker,显然middlerManager得去起peon进程去处理了
middleManager监听新增任务去启动peon进程
回到middleManager的WorkerTaskMonitor: 上面overlord创建·了/druid/indexer/tasks/workerHost:ip/<taskId>
表示成功给任务分配了worker, 那么worker要去处理了
见:io.druid.indexing.worker.WorkerTaskMonitor#registerRunListener
方法,同样的WorkerTaskMonitor
内部的pathChildrenCache
对znode进行了监听
this.pathChildrenCache = new PathChildrenCache(
cf, workerCuratorCoordinator.getTaskPathForWorker(), false, true, Execs.makeThreadFactory("TaskMonitorCache-%s")
);
private void registerRunListener()
{
pathChildrenCache.getListenable().addListener(
new PathChildrenCacheListener()
{
@Override
public void childEvent(CuratorFramework curatorFramework, PathChildrenCacheEvent pathChildrenCacheEvent)
throws Exception
{
if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) {
// 解析出 Task 信息来,然后加入 notices 任务队列
final Task task = jsonMapper.readValue(
cf.getData().forPath(pathChildrenCacheEvent.getData().getPath()),
Task.class
);
notices.add(new RunNotice(task));
}
}
}
);
}
overlord新增了任务,这里就能监听到,并加入到notices(notices.add(new RunNotice(task));
)
另一方面起主循环方法中,不断的去去notices中任务,开始启动peon进程去执行
private class RunNotice implements Notice
{
private final Task task;
public RunNotice(Task task)
{
this.task = task;
}
@Override
public String getTaskId()
{
return task.getId();
}
@Override
public void handle() throws Exception
{
if (running.containsKey(task.getId())) {
log.warn(
"Got run notice for task [%s] that I am already running...",
task.getId()
);
workerCuratorCoordinator.removeTaskRunZnode(task.getId());
return;
}
log.info("Submitting runnable for task[%s]", task.getId());
workerCuratorCoordinator.updateTaskStatusAnnouncement(
TaskAnnouncement.create(
task,
TaskStatus.running(task.getId()),
TaskLocation.unknown()
)
);
log.info("Affirmative. Running task [%s]", task.getId());
workerCuratorCoordinator.removeTaskRunZnode(task.getId());
final ListenableFuture<TaskStatus> future = taskRunner.run(task);
addRunningTask(task, future);
}
}
这里的znode两处变化
- 新增
/druid/indexer/status/workerHost:ip/<taskId>
表示要启动peon进程去运行这个任务了 - 先前overlord创建的
/druid/indexer/tasks/workerHost:ip/<taskId>
要删除了
如下图所示
peon进程启动:任务running起来
peon启动后,任务状态信息要更新;任务结束后,任务状态也要更新,都是改变的/druid/indexer/status/workerHost:ip/<taskId>
如下图,任务zk节点存储任务运行的状态,location等信息
middleManager对任务完成/失败的处理: zk任务状态的更新
peon进程的ForkingTaskRunner
中对任务有future通知处理
TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), status);
不过io.druid.indexing.worker.WorkerTaskMonitor#registerLocationListener
可以看到没做任何处理
private void registerLocationListener()
{
taskRunner.registerListener(
new TaskRunnerListener()
{
@Override
public String getListenerId()
{
return "WorkerTaskMonitor";
}
@Override
public void locationChanged(final String taskId, final TaskLocation newLocation)
{
notices.add(new LocationNotice(taskId, newLocation));
}
@Override
public void statusChanged(final String taskId, final TaskStatus status)
{
// do nothing
}
},
MoreExecutors.sameThreadExecutor()
);
但是对于middleManger ,其在WorkerTaskMonitor
中交给ForkingTaskRunner
处理的时候,也是返回的future, 处理如下
private void addRunningTask(final Task task, final ListenableFuture<TaskStatus> future)
{
running.put(task.getId(), new TaskDetails(task));
Futures.addCallback(
future,
new FutureCallback<TaskStatus>()
{
@Override
public void onSuccess(TaskStatus result)
{
notices.add(new StatusNotice(task, result));
}
@Override
public void onFailure(Throwable t)
{
notices.add(new StatusNotice(task, TaskStatus.failure(task.getId())));
}
}
);
}
可以看到middleManager具体处理如下: 主要是任务队列信息更新和zookeeper上任务信息的更新
private class StatusNotice implements Notice
{
private final Task task;
private final TaskStatus status;
public StatusNotice(Task task, TaskStatus status)
{
this.task = task;
this.status = status;
}
@Override
public String getTaskId()
{
return task.getId();
}
@Override
public void handle() throws Exception
{
final TaskDetails details = running.get(task.getId());
if (details == null) {
log.warn("Got status notice for task [%s] that isn't running...", task.getId());
return;
}
if (!status.isComplete()) {
log.warn(
"WTF?! Got status notice for task [%s] that isn't complete (status = [%s])...",
task.getId(),
status.getStatusCode()
);
return;
}
details.status = status.withDuration(System.currentTimeMillis() - details.startTime);
try {
workerCuratorCoordinator.updateTaskStatusAnnouncement(
TaskAnnouncement.create(
details.task,
details.status,
details.location
)
);
log.info(
"Job's finished. Completed [%s] with status [%s]",
task.getId(),
status.getStatusCode()
);
}
catch (InterruptedException e) {
throw e;
}
catch (Exception e) {
log.makeAlert(e, "Failed to update task announcement")
.addData("task", task.getId())
.emit();
}
finally {
running.remove(task.getId());
}
}
}
overlord 对任务完成/失败的处理:删除status的任务znode
在overlord监听/druid/indexer/announcement
添加新worker的时候,同时对/druid/indexer/status/workerHost:ip/<taskId>
进行了监听,见io.druid.indexing.overlord.RemoteTaskRunner#addWorker
/**
* 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 RemoteTaskRunner updates state according to these changes.
*
* @param worker contains metadata for a worker that has appeared in ZK
*
* @return future that will contain a fully initialized worker
*/
private ListenableFuture<ZkWorker> addWorker(final Worker worker)
{
log.info("Worker[%s] reportin' for duty!", worker.getHost());
try {
cancelWorkerCleanup(worker.getHost());
final String workerStatusPath = JOINER.join(indexerZkConfig.getStatusPath(), worker.getHost());
final PathChildrenCache statusCache = workerStatusPathChildrenCacheFactory.make(cf, workerStatusPath);
final SettableFuture<ZkWorker> retVal = SettableFuture.create();
final ZkWorker zkWorker = new ZkWorker(
worker,
statusCache,
jsonMapper
);
// Add status listener to the watcher for status changes
zkWorker.addListener(
new PathChildrenCacheListener()
{
@Override
public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
{
final String taskId;
final RemoteTaskRunnerWorkItem taskRunnerWorkItem;
synchronized (statusLock) {
try {
switch (event.getType()) {
case CHILD_ADDED:
case CHILD_UPDATED:
taskId = ZKPaths.getNodeFromPath(event.getData().getPath());
final TaskAnnouncement announcement = jsonMapper.readValue(
event.getData().getData(), TaskAnnouncement.class
);
log.info(
"Worker[%s] wrote %s status for task [%s] on [%s]",
zkWorker.getWorker().getHost(),
announcement.getTaskStatus().getStatusCode(),
taskId,
announcement.getTaskLocation()
);
// Synchronizing state with ZK
statusLock.notifyAll();
final RemoteTaskRunnerWorkItem tmp;
if ((tmp = runningTasks.get(taskId)) != null) {
taskRunnerWorkItem = tmp;
} else {
final RemoteTaskRunnerWorkItem newTaskRunnerWorkItem = new RemoteTaskRunnerWorkItem(
taskId,
zkWorker.getWorker(),
TaskLocation.unknown()
);
final RemoteTaskRunnerWorkItem existingItem = runningTasks.putIfAbsent(
taskId,
newTaskRunnerWorkItem
);
if (existingItem == null) {
log.warn(
"Worker[%s] announced a status for a task I didn't know about, adding to runningTasks: %s",
zkWorker.getWorker().getHost(),
taskId
);
taskRunnerWorkItem = newTaskRunnerWorkItem;
} else {
taskRunnerWorkItem = existingItem;
}
}
if (!announcement.getTaskLocation().equals(taskRunnerWorkItem.getLocation())) {
taskRunnerWorkItem.setLocation(announcement.getTaskLocation());
TaskRunnerUtils.notifyLocationChanged(listeners, taskId, announcement.getTaskLocation());
}
if (announcement.getTaskStatus().isComplete()) {
taskComplete(taskRunnerWorkItem, zkWorker, announcement.getTaskStatus());
runPendingTasks();
}
break;
case CHILD_REMOVED:
可以在overlord日志中看到如下
- 任务锁要释放
- 标记任务完成,worker的信息更新
- 任务加入到完成队列中,且从运行队列中移除(RemoteTaskRunner)
- etc.
TaskQueue在io.druid.indexing.overlord.TaskQueue#manage
分配任务运行也是用的回调处理任务状态返回ListenableFuture<TaskStatus>
,具体见io.druid.indexing.overlord.TaskQueue#attachCallbacks
/**
* Attach success and failure handlers to a task status future, such that when it completes, we perform the
* appropriate updates.
*
* @param statusFuture a task status future
*
* @return the same future, for convenience
*/
private ListenableFuture<TaskStatus> attachCallbacks(final Task task, final ListenableFuture<TaskStatus> statusFuture)
{
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
.setDimension("dataSource", task.getDataSource())
.setDimension("taskType", task.getType());
Futures.addCallback(
statusFuture,
new FutureCallback<TaskStatus>()
{
@Override
public void onSuccess(final TaskStatus status)
{
log.info("Received %s status for task: %s", status.getStatusCode(), status.getId());
handleStatus(status);
}
@Override
public void onFailure(final Throwable t)
{
log.makeAlert(t, "Failed to run task")
.addData("task", task.getId())
.addData("type", task.getType())
.addData("dataSource", task.getDataSource())
.emit();
handleStatus(TaskStatus.failure(task.getId()));
}
private void handleStatus(final TaskStatus status)
{
try {
// If we're not supposed to be running anymore, don't do anything. Somewhat racey if the flag gets set
// after we check and before we commit the database transaction, but better than nothing.
if (!active) {
log.info("Abandoning task due to shutdown: %s", task.getId());
return;
}
notifyStatus(task, status);
// Emit event and log, if the task is done
if (status.isComplete()) {
metricBuilder.setDimension(DruidMetrics.TASK_STATUS, status.getStatusCode().toString());
emitter.emit(metricBuilder.build("task/run/time", status.getDuration()));
log.info(
"Task %s: %s (%d run duration)",
status.getStatusCode(),
task,
status.getDuration()
);
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to handle task status")
.addData("task", task.getId())
.addData("statusCode", status.getStatusCode())
.emit();
}
}
}
);
return statusFuture;
}
处理任务状态改变,则是io.druid.indexing.overlord.TaskQueue#notifyStatus
方法,主要的几行行代码如下,这里当然是RemoteTaskRunner
,做的事情主要如下两件
// 1. 任务关闭清理
taskRunner.shutdown(task.getId());
// 2. 任务清理成功的后续处理
// Remove from futures list
taskFutures.remove(task.getId());
if (removed > 0) {
// If we thought this task should be running, save status to DB
try {
final Optional<TaskStatus> previousStatus = taskStorage.getStatus(task.getId());
if (!previousStatus.isPresent() || !previousStatus.get().isRunnable()) {
log.makeAlert("Ignoring notification for already-complete task").addData("task", task.getId()).emit();
} else {
taskStorage.setStatus(taskStatus);
log.info("Task done: %s", task);
managementMayBeNecessary.signalAll();
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to persist status for task")
.addData("task", task.getId())
.addData("statusCode", taskStatus.getStatusCode())
.emit();
}
}
shutdown 处理如下
/**
* Finds the worker running the task and forwards the shutdown signal to the worker.
*
* @param taskId - task id to shutdown
*/
@Override
public void shutdown(final String taskId)
{
if (!lifecycleLock.awaitStarted(1, TimeUnit.SECONDS)) {
log.info("This TaskRunner is stopped or not yet started. Ignoring shutdown command for task: %s", taskId);
} else if (pendingTasks.remove(taskId) != null) {
pendingTaskPayloads.remove(taskId);
log.info("Removed task from pending queue: %s", taskId);
} else if (completeTasks.containsKey(taskId)) {
cleanup(taskId);
} else {
final ZkWorker zkWorker = findWorkerRunningTask(taskId);
if (zkWorker == null) {
log.info("Can't shutdown! No worker running task %s", taskId);
return;
}
URL url = null;
try {
url = makeWorkerURL(zkWorker.getWorker(), StringUtils.format("/task/%s/shutdown", taskId));
final StatusResponseHolder response = httpClient.go(
new Request(HttpMethod.POST, url),
RESPONSE_HANDLER,
shutdownTimeout
).get();
log.info(
"Sent shutdown message to worker: %s, status %s, response: %s",
zkWorker.getWorker().getHost(),
response.getStatus(),
response.getContent()
);
if (!HttpResponseStatus.OK.equals(response.getStatus())) {
log.error("Shutdown failed for %s! Are you sure the task was running?", taskId);
}
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RE(e, "Interrupted posting shutdown to [%s] for task [%s]", url, taskId);
}
catch (Exception e) {
throw new RE(e, "Error in handling post to [%s] for task [%s]", zkWorker.getWorker().getHost(), taskId);
}
}
}
可以看到完成任务的重要处理就是执行io.druid.indexing.overlord.RemoteTaskRunner#cleanup
方法
这里删除了zkPath: /druid/indexer/status/workerHost:port/
shutdown成功则更新了任务的元数据状态,及taskStorage.setStatus(taskStatus);
这一行代码
至此overlord才完成了任务结束最后的状态管理
总结
任务从开始到结束这个过程中,交互流程还是比较多的,但无非基于如下几点
- Future回调处理机制
- zookeeper的注册监听watch机制
- 队列的使用:入队/出队
简要的流程如下(不过还是建议看下整个逻辑源码,理解的更透彻一点)
- middle启动,overlord创建 znode 维护workers,并注册监听
- 任务由overlord接收并创建znode去发布任务
- middleManager监听了发布任务的znode,去运行任务,删除overlord创建的tasks 任务znode, 并新建status 任务znode
- peon进程运行任务并更新 status 任务znode的信息
- middleManager回调任务完成,并更新自己维护的任务队列,并对znode进行更新
- overlord回调任务完成,也是更新自己的任务队列相关,并更新任务的元数据信息,完成清理工作,并最终删除 status 任务znode
借助zookeeper,overlord
,middleManager
,peon
三个进程完成各种交互,在任务的生命周期起到了重要作用,当然通过之前的博文知道,index_realtime任务最后是要hand off的,即任务是要与coordinator打交道的,index_realtime还要负责响应查询,与broker也是打交道的,所以本文这里没有进行这方面的深入,作为后面继续深入说明的点
实际工作中:任务状态也确实遇到不少奇怪的现象,不过通过查看zk, log,元数据库都了解并一一解决了,所以了解这些原理对工作确实是极有帮助的