Wenjun Ruan
2 years ago
committed by
GitHub
187 changed files with 6803 additions and 5271 deletions
@ -0,0 +1,29 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.dao.repository; |
||||
|
||||
import org.apache.dolphinscheduler.dao.entity.Project; |
||||
|
||||
import java.util.Collection; |
||||
import java.util.List; |
||||
|
||||
public interface ProjectDao { |
||||
|
||||
List<Project> queryByCodes(Collection<Long> projectCodes); |
||||
|
||||
} |
@ -0,0 +1,40 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.dao.repository.impl; |
||||
|
||||
import org.apache.dolphinscheduler.dao.entity.Project; |
||||
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; |
||||
import org.apache.dolphinscheduler.dao.repository.ProjectDao; |
||||
|
||||
import java.util.Collection; |
||||
import java.util.List; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Repository; |
||||
|
||||
@Repository |
||||
public class ProjectDaoImpl implements ProjectDao { |
||||
|
||||
@Autowired |
||||
private ProjectMapper projectMapper; |
||||
|
||||
@Override |
||||
public List<Project> queryByCodes(Collection<Long> projectCodes) { |
||||
return projectMapper.queryByCodes(projectCodes); |
||||
} |
||||
} |
@ -1,353 +0,0 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.consumer; |
||||
|
||||
import org.apache.dolphinscheduler.common.constants.Constants; |
||||
import org.apache.dolphinscheduler.common.enums.Flag; |
||||
import org.apache.dolphinscheduler.common.enums.TaskEventType; |
||||
import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager; |
||||
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; |
||||
import org.apache.dolphinscheduler.common.thread.ThreadUtils; |
||||
import org.apache.dolphinscheduler.common.utils.DateUtils; |
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance; |
||||
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; |
||||
import org.apache.dolphinscheduler.dao.utils.TaskCacheUtils; |
||||
import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; |
||||
import org.apache.dolphinscheduler.remote.command.Message; |
||||
import org.apache.dolphinscheduler.remote.command.task.TaskDispatchMessage; |
||||
import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; |
||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; |
||||
import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; |
||||
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; |
||||
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; |
||||
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; |
||||
import org.apache.dolphinscheduler.service.exceptions.TaskPriorityQueueException; |
||||
import org.apache.dolphinscheduler.service.queue.TaskPriority; |
||||
import org.apache.dolphinscheduler.service.queue.TaskPriorityQueue; |
||||
|
||||
import org.apache.commons.collections4.CollectionUtils; |
||||
|
||||
import java.util.ArrayList; |
||||
import java.util.Collections; |
||||
import java.util.Date; |
||||
import java.util.List; |
||||
import java.util.Objects; |
||||
import java.util.Optional; |
||||
import java.util.concurrent.CountDownLatch; |
||||
import java.util.concurrent.ThreadPoolExecutor; |
||||
import java.util.concurrent.TimeUnit; |
||||
|
||||
import javax.annotation.PostConstruct; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
/** |
||||
* TaskUpdateQueue consumer |
||||
*/ |
||||
@Component |
||||
@Slf4j |
||||
public class TaskPriorityQueueConsumer extends BaseDaemonThread { |
||||
|
||||
/** |
||||
* taskUpdateQueue |
||||
*/ |
||||
@Autowired |
||||
private TaskPriorityQueue<TaskPriority> taskPriorityQueue; |
||||
|
||||
@Autowired |
||||
private TaskInstanceDao taskInstanceDao; |
||||
|
||||
/** |
||||
* executor dispatcher |
||||
*/ |
||||
@Autowired |
||||
private ExecutorDispatcher dispatcher; |
||||
|
||||
/** |
||||
* processInstance cache manager |
||||
*/ |
||||
@Autowired |
||||
private ProcessInstanceExecCacheManager processInstanceExecCacheManager; |
||||
|
||||
/** |
||||
* master config |
||||
*/ |
||||
@Autowired |
||||
private MasterConfig masterConfig; |
||||
|
||||
/** |
||||
* task response service |
||||
*/ |
||||
@Autowired |
||||
private TaskEventService taskEventService; |
||||
|
||||
/** |
||||
* storage operator |
||||
*/ |
||||
@Autowired(required = false) |
||||
private StorageOperate storageOperate; |
||||
|
||||
/** |
||||
* consumer thread pool |
||||
*/ |
||||
private ThreadPoolExecutor consumerThreadPoolExecutor; |
||||
|
||||
protected TaskPriorityQueueConsumer() { |
||||
super("TaskPriorityQueueConsumeThread"); |
||||
} |
||||
|
||||
@PostConstruct |
||||
public void init() { |
||||
this.consumerThreadPoolExecutor = (ThreadPoolExecutor) ThreadUtils |
||||
.newDaemonFixedThreadExecutor("TaskUpdateQueueConsumerThread", masterConfig.getDispatchTaskNumber()); |
||||
log.info("Task priority queue consume thread staring"); |
||||
super.start(); |
||||
log.info("Task priority queue consume thread started"); |
||||
} |
||||
|
||||
@Override |
||||
public void run() { |
||||
int fetchTaskNum = masterConfig.getDispatchTaskNumber(); |
||||
while (!ServerLifeCycleManager.isStopped()) { |
||||
try { |
||||
List<TaskPriority> failedDispatchTasks = this.batchDispatch(fetchTaskNum); |
||||
|
||||
if (CollectionUtils.isNotEmpty(failedDispatchTasks)) { |
||||
log.info("{} tasks dispatch failed, will retry to dispatch", failedDispatchTasks.size()); |
||||
TaskMetrics.incTaskDispatchFailed(failedDispatchTasks.size()); |
||||
for (TaskPriority dispatchFailedTask : failedDispatchTasks) { |
||||
taskPriorityQueue.put(dispatchFailedTask); |
||||
} |
||||
// If the all task dispatch failed, will sleep for 1s to avoid the master cpu higher.
|
||||
if (fetchTaskNum == failedDispatchTasks.size()) { |
||||
log.info("All tasks dispatch failed, will sleep a while to avoid the master cpu higher"); |
||||
TimeUnit.MILLISECONDS.sleep(Constants.SLEEP_TIME_MILLIS); |
||||
} |
||||
} |
||||
} catch (Exception e) { |
||||
TaskMetrics.incTaskDispatchError(); |
||||
log.error("dispatcher task error", e); |
||||
} |
||||
} |
||||
} |
||||
|
||||
/** |
||||
* batch dispatch with thread pool |
||||
*/ |
||||
public List<TaskPriority> batchDispatch(int fetchTaskNum) throws TaskPriorityQueueException, InterruptedException { |
||||
List<TaskPriority> failedDispatchTasks = Collections.synchronizedList(new ArrayList<>()); |
||||
CountDownLatch latch = new CountDownLatch(fetchTaskNum); |
||||
|
||||
for (int i = 0; i < fetchTaskNum; i++) { |
||||
TaskPriority taskPriority = taskPriorityQueue.poll(Constants.SLEEP_TIME_MILLIS, TimeUnit.MILLISECONDS); |
||||
if (Objects.isNull(taskPriority)) { |
||||
latch.countDown(); |
||||
continue; |
||||
} |
||||
|
||||
consumerThreadPoolExecutor.submit(() -> { |
||||
try { |
||||
try { |
||||
this.dispatchTask(taskPriority); |
||||
} catch (WorkerGroupNotFoundException e) { |
||||
// If the worker group not found, will not try to dispatch again.
|
||||
// The task instance will be failed
|
||||
// todo:
|
||||
addDispatchFailedEvent(taskPriority); |
||||
} catch (ExecuteException e) { |
||||
failedDispatchTasks.add(taskPriority); |
||||
} catch (Exception e) { |
||||
log.error("Dispatch task error, meet an unknown exception", e); |
||||
failedDispatchTasks.add(taskPriority); |
||||
} |
||||
} finally { |
||||
// make sure the latch countDown
|
||||
latch.countDown(); |
||||
} |
||||
}); |
||||
} |
||||
|
||||
latch.await(); |
||||
|
||||
return failedDispatchTasks; |
||||
} |
||||
|
||||
/** |
||||
* Dispatch task to worker. |
||||
* |
||||
* @param taskPriority taskPriority |
||||
* @return dispatch result, return true if dispatch success, return false if dispatch failed. |
||||
*/ |
||||
protected void dispatchTask(TaskPriority taskPriority) throws ExecuteException { |
||||
TaskMetrics.incTaskDispatch(); |
||||
WorkflowExecuteRunnable workflowExecuteRunnable = |
||||
processInstanceExecCacheManager.getByProcessInstanceId(taskPriority.getProcessInstanceId()); |
||||
if (workflowExecuteRunnable == null) { |
||||
log.error("Cannot find the related processInstance of the task, taskPriority: {}", taskPriority); |
||||
return; |
||||
} |
||||
Optional<TaskInstance> taskInstanceOptional = |
||||
workflowExecuteRunnable.getTaskInstance(taskPriority.getTaskId()); |
||||
if (!taskInstanceOptional.isPresent()) { |
||||
log.error("Cannot find the task instance from related processInstance, taskPriority: {}", |
||||
taskPriority); |
||||
// we return true, so that we will drop this task.
|
||||
return; |
||||
} |
||||
TaskInstance taskInstance = taskInstanceOptional.get(); |
||||
TaskExecutionContext context = taskPriority.getTaskExecutionContext(); |
||||
ExecutionContext executionContext = ExecutionContext.builder() |
||||
.taskInstance(taskInstance) |
||||
.workerGroup(context.getWorkerGroup()) |
||||
.executorType(ExecutorType.WORKER) |
||||
.message(toCommand(context)) |
||||
.build(); |
||||
|
||||
if (isTaskNeedToCheck(taskPriority)) { |
||||
if (taskInstanceIsFinalState(taskPriority.getTaskId())) { |
||||
// when task finish, ignore this task, there is no need to dispatch anymore
|
||||
log.info("Task {} is already finished, no need to dispatch, task instance id: {}", |
||||
taskInstance.getName(), taskInstance.getId()); |
||||
return; |
||||
} |
||||
} |
||||
|
||||
// check task is cache execution, and decide whether to dispatch
|
||||
if (checkIsCacheExecution(taskInstance, context)) { |
||||
return; |
||||
} |
||||
|
||||
dispatcher.dispatch(executionContext); |
||||
log.info("Master success dispatch task to worker, taskInstanceId: {}, worker: {}", |
||||
taskPriority.getTaskId(), |
||||
executionContext.getHost()); |
||||
addDispatchEvent(context, executionContext); |
||||
} |
||||
|
||||
/** |
||||
* add dispatch event |
||||
*/ |
||||
private void addDispatchEvent(TaskExecutionContext context, ExecutionContext executionContext) { |
||||
TaskEvent taskEvent = TaskEvent.newDispatchEvent(context.getProcessInstanceId(), context.getTaskInstanceId(), |
||||
executionContext.getHost().getAddress()); |
||||
taskEventService.addEvent(taskEvent); |
||||
} |
||||
|
||||
private void addDispatchFailedEvent(TaskPriority taskPriority) { |
||||
TaskExecutionContext taskExecutionContext = taskPriority.getTaskExecutionContext(); |
||||
TaskEvent taskEvent = TaskEvent.builder() |
||||
.processInstanceId(taskPriority.getProcessInstanceId()) |
||||
.taskInstanceId(taskPriority.getTaskId()) |
||||
.state(TaskExecutionStatus.FAILURE) |
||||
.logPath(taskExecutionContext.getLogPath()) |
||||
.executePath(taskExecutionContext.getExecutePath()) |
||||
.appIds(taskExecutionContext.getAppIds()) |
||||
.processId(taskExecutionContext.getProcessId()) |
||||
.varPool(taskExecutionContext.getVarPool()) |
||||
.startTime(DateUtils.timeStampToDate(taskExecutionContext.getStartTime())) |
||||
.endTime(new Date()) |
||||
.event(TaskEventType.RESULT) |
||||
.build(); |
||||
taskEventService.addEvent(taskEvent); |
||||
} |
||||
|
||||
private Message toCommand(TaskExecutionContext taskExecutionContext) { |
||||
// todo: we didn't set the host here, since right now we didn't need to retry this message.
|
||||
TaskDispatchMessage requestCommand = new TaskDispatchMessage(taskExecutionContext, |
||||
masterConfig.getMasterAddress(), |
||||
taskExecutionContext.getHost(), |
||||
System.currentTimeMillis()); |
||||
return requestCommand.convert2Command(); |
||||
} |
||||
|
||||
/** |
||||
* taskInstance is final state |
||||
* success,failure,kill,stop,pause,threadwaiting is final state |
||||
* |
||||
* @param taskInstanceId taskInstanceId |
||||
* @return taskInstance is final state |
||||
*/ |
||||
public boolean taskInstanceIsFinalState(int taskInstanceId) { |
||||
TaskInstance taskInstance = taskInstanceDao.findTaskInstanceById(taskInstanceId); |
||||
return taskInstance.getState().isFinished(); |
||||
} |
||||
|
||||
/** |
||||
* check if task need to check state, if true, refresh the checkpoint |
||||
*/ |
||||
private boolean isTaskNeedToCheck(TaskPriority taskPriority) { |
||||
long now = System.currentTimeMillis(); |
||||
if (now - taskPriority.getCheckpoint() > Constants.SECOND_TIME_MILLIS) { |
||||
taskPriority.setCheckpoint(now); |
||||
return true; |
||||
} |
||||
return false; |
||||
} |
||||
|
||||
/** |
||||
* check if task is cache execution |
||||
* if the task is defined as cache execution, and we find the cache task instance is finished yet, we will not dispatch this task |
||||
* @param taskInstance taskInstance |
||||
* @param context context |
||||
* @return true if we will not dispatch this task, false if we will dispatch this task |
||||
*/ |
||||
private boolean checkIsCacheExecution(TaskInstance taskInstance, TaskExecutionContext context) { |
||||
try { |
||||
// check if task is defined as a cache task
|
||||
if (taskInstance.getIsCache().equals(Flag.NO)) { |
||||
return false; |
||||
} |
||||
// check if task is cache execution
|
||||
String cacheKey = TaskCacheUtils.generateCacheKey(taskInstance, context, storageOperate); |
||||
TaskInstance cacheTaskInstance = taskInstanceDao.findTaskInstanceByCacheKey(cacheKey); |
||||
// if we can find the cache task instance, we will add cache event, and return true.
|
||||
if (cacheTaskInstance != null) { |
||||
log.info("Task {} is cache, no need to dispatch, task instance id: {}", |
||||
taskInstance.getName(), taskInstance.getId()); |
||||
addCacheEvent(taskInstance, cacheTaskInstance); |
||||
taskInstance.setCacheKey(TaskCacheUtils.generateTagCacheKey(cacheTaskInstance.getId(), cacheKey)); |
||||
return true; |
||||
} else { |
||||
// if we can not find cache task, update cache key, and return false. the task will be dispatched
|
||||
taskInstance.setCacheKey(TaskCacheUtils.generateTagCacheKey(taskInstance.getId(), cacheKey)); |
||||
} |
||||
} catch (Exception e) { |
||||
log.error("checkIsCacheExecution error", e); |
||||
} |
||||
return false; |
||||
} |
||||
|
||||
private void addCacheEvent(TaskInstance taskInstance, TaskInstance cacheTaskInstance) { |
||||
if (cacheTaskInstance == null) { |
||||
return; |
||||
} |
||||
TaskEvent taskEvent = TaskEvent.newCacheEvent(taskInstance.getProcessInstanceId(), taskInstance.getId(), |
||||
cacheTaskInstance.getId()); |
||||
taskEventService.addEvent(taskEvent); |
||||
} |
||||
} |
@ -1,118 +0,0 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.dispatch; |
||||
|
||||
import org.apache.dolphinscheduler.remote.utils.Host; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.executor.ExecutorManager; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.host.HostManager; |
||||
|
||||
import org.apache.commons.lang3.StringUtils; |
||||
|
||||
import java.util.concurrent.ConcurrentHashMap; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.InitializingBean; |
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Service; |
||||
|
||||
/** |
||||
* executor dispatcher |
||||
*/ |
||||
@Service |
||||
@Slf4j |
||||
public class ExecutorDispatcher implements InitializingBean { |
||||
|
||||
/** |
||||
* netty executor manager |
||||
*/ |
||||
@Autowired |
||||
private NettyExecutorManager nettyExecutorManager; |
||||
|
||||
/** |
||||
* round robin host manager |
||||
*/ |
||||
@Autowired |
||||
private HostManager hostManager; |
||||
|
||||
/** |
||||
* executor manager |
||||
*/ |
||||
private final ConcurrentHashMap<ExecutorType, ExecutorManager<Boolean>> executorManagers; |
||||
|
||||
/** |
||||
* constructor |
||||
*/ |
||||
public ExecutorDispatcher() { |
||||
this.executorManagers = new ConcurrentHashMap<>(); |
||||
} |
||||
|
||||
/** |
||||
* task dispatch |
||||
* |
||||
* @param context context |
||||
* @return result |
||||
* @throws ExecuteException if error throws ExecuteException |
||||
*/ |
||||
public void dispatch(final ExecutionContext context) throws ExecuteException { |
||||
// get executor manager
|
||||
ExecutorManager<Boolean> executorManager = this.executorManagers.get(context.getExecutorType()); |
||||
if (executorManager == null) { |
||||
throw new ExecuteException("no ExecutorManager for type : " + context.getExecutorType()); |
||||
} |
||||
|
||||
// host select
|
||||
Host host = hostManager.select(context); |
||||
if (StringUtils.isEmpty(host.getAddress())) { |
||||
log.warn("fail to execute : {} due to no suitable worker, current task needs worker group {} to execute", |
||||
context.getMessage(), context.getWorkerGroup()); |
||||
throw new ExecuteException("no suitable worker"); |
||||
} |
||||
context.setHost(host); |
||||
executorManager.beforeExecute(context); |
||||
try { |
||||
// task execute
|
||||
executorManager.execute(context); |
||||
} finally { |
||||
executorManager.afterExecute(context); |
||||
} |
||||
} |
||||
|
||||
/** |
||||
* register init |
||||
* @throws Exception if error throws Exception |
||||
*/ |
||||
@Override |
||||
public void afterPropertiesSet() throws Exception { |
||||
register(ExecutorType.WORKER, nettyExecutorManager); |
||||
register(ExecutorType.CLIENT, nettyExecutorManager); |
||||
} |
||||
|
||||
/** |
||||
* register |
||||
* @param type executor type |
||||
* @param executorManager executorManager |
||||
*/ |
||||
public void register(ExecutorType type, ExecutorManager executorManager) { |
||||
executorManagers.put(type, executorManager); |
||||
} |
||||
} |
@ -1,84 +0,0 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.event; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.TaskEventType; |
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance; |
||||
import org.apache.dolphinscheduler.remote.command.task.TaskRejectMessageAck; |
||||
import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; |
||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; |
||||
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class TaskRejectByWorkerEventHandler implements TaskEventHandler { |
||||
|
||||
@Autowired |
||||
private ProcessInstanceExecCacheManager processInstanceExecCacheManager; |
||||
|
||||
@Autowired |
||||
private MasterConfig masterConfig; |
||||
|
||||
@Override |
||||
public void handleTaskEvent(TaskEvent taskEvent) throws TaskEventHandleError { |
||||
int taskInstanceId = taskEvent.getTaskInstanceId(); |
||||
int processInstanceId = taskEvent.getProcessInstanceId(); |
||||
|
||||
WorkflowExecuteRunnable workflowExecuteRunnable = this.processInstanceExecCacheManager.getByProcessInstanceId( |
||||
processInstanceId); |
||||
if (workflowExecuteRunnable == null) { |
||||
sendAckToWorker(taskEvent); |
||||
throw new TaskEventHandleError( |
||||
"Handle task reject event error, cannot find related workflow instance from cache, will discard this event"); |
||||
} |
||||
TaskInstance taskInstance = workflowExecuteRunnable.getTaskInstance(taskInstanceId).orElseThrow(() -> { |
||||
sendAckToWorker(taskEvent); |
||||
return new TaskEventHandleError( |
||||
"Handle task reject event error, cannot find the taskInstance from cache, will discord this event"); |
||||
}); |
||||
try { |
||||
// todo: If the worker submit multiple reject response to master, the task instance may be dispatch
|
||||
// multiple,
|
||||
// we need to control the worker overload by master rather than worker
|
||||
// if the task resubmit and the worker failover, this task may be dispatch twice?
|
||||
// todo: we need to clear the taskInstance host and rollback the status to submit.
|
||||
workflowExecuteRunnable.resubmit(taskInstance.getTaskCode()); |
||||
sendAckToWorker(taskEvent); |
||||
} catch (Exception ex) { |
||||
throw new TaskEventHandleError("Handle task reject event error", ex); |
||||
} |
||||
|
||||
} |
||||
|
||||
public void sendAckToWorker(TaskEvent taskEvent) { |
||||
TaskRejectMessageAck taskRejectMessageAck = new TaskRejectMessageAck(true, |
||||
taskEvent.getTaskInstanceId(), |
||||
masterConfig.getMasterAddress(), |
||||
taskEvent.getWorkerAddress(), |
||||
System.currentTimeMillis()); |
||||
taskEvent.getChannel().writeAndFlush(taskRejectMessageAck.convert2Command()); |
||||
} |
||||
|
||||
@Override |
||||
public TaskEventType getHandleEventType() { |
||||
return TaskEventType.WORKER_REJECT; |
||||
} |
||||
} |
@ -0,0 +1,26 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.exception; |
||||
|
||||
public class LogicTaskFactoryNotFoundException extends MasterException { |
||||
|
||||
public LogicTaskFactoryNotFoundException(String message) { |
||||
super(message); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,30 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.exception; |
||||
|
||||
public class LogicTaskInitializeException extends MasterException { |
||||
|
||||
public LogicTaskInitializeException(String message) { |
||||
super(message); |
||||
} |
||||
|
||||
public LogicTaskInitializeException(String message, Throwable cause) { |
||||
super(message, cause); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,29 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.exception; |
||||
|
||||
public class MasterTaskExecuteException extends MasterException { |
||||
|
||||
public MasterTaskExecuteException(String message) { |
||||
super(message); |
||||
} |
||||
|
||||
public MasterTaskExecuteException(String message, Throwable cause) { |
||||
super(message, cause); |
||||
} |
||||
} |
@ -0,0 +1,29 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.exception; |
||||
|
||||
public class TaskDispatchException extends MasterException { |
||||
|
||||
public TaskDispatchException(String message) { |
||||
super(message); |
||||
} |
||||
|
||||
public TaskDispatchException(String message, Throwable cause) { |
||||
super(message, cause); |
||||
} |
||||
} |
@ -0,0 +1,30 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.exception; |
||||
|
||||
public class TaskExecuteRunnableCreateException extends MasterException { |
||||
|
||||
public TaskExecuteRunnableCreateException(String message) { |
||||
super(message); |
||||
} |
||||
|
||||
public TaskExecuteRunnableCreateException(String message, Throwable throwable) { |
||||
super(message, throwable); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,26 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.exception; |
||||
|
||||
public class TaskExecutionContextCreateException extends MasterException { |
||||
|
||||
public TaskExecutionContextCreateException(String message) { |
||||
super(message); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,29 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.exception; |
||||
|
||||
public class WorkflowCreateException extends MasterException { |
||||
|
||||
public WorkflowCreateException(String message) { |
||||
super(message); |
||||
} |
||||
|
||||
public WorkflowCreateException(String message, Throwable throwable) { |
||||
super(message, throwable); |
||||
} |
||||
} |
@ -0,0 +1,130 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.processor; |
||||
|
||||
import org.apache.dolphinscheduler.common.utils.DateUtils; |
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; |
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; |
||||
import org.apache.dolphinscheduler.remote.command.Message; |
||||
import org.apache.dolphinscheduler.remote.command.MessageType; |
||||
import org.apache.dolphinscheduler.remote.command.task.TaskDispatchRequest; |
||||
import org.apache.dolphinscheduler.remote.command.task.TaskDispatchResponse; |
||||
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; |
||||
import org.apache.dolphinscheduler.server.master.runner.MasterDelayTaskExecuteRunnableDelayQueue; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.MasterDelayTaskExecuteRunnable; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnableFactoryBuilder; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutionContextHolder; |
||||
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager; |
||||
|
||||
import java.util.concurrent.TimeUnit; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
import io.netty.channel.Channel; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class MasterTaskDispatchProcessor implements MasterRpcProcessor { |
||||
|
||||
@Autowired |
||||
private MasterTaskExecuteRunnableFactoryBuilder masterTaskExecuteRunnableFactoryBuilder; |
||||
|
||||
@Autowired |
||||
private MasterMessageSenderManager masterMessageSenderManager; |
||||
|
||||
@Autowired |
||||
private MasterDelayTaskExecuteRunnableDelayQueue masterDelayTaskExecuteRunnableDelayQueue; |
||||
|
||||
@Override |
||||
public void process(Channel channel, Message message) { |
||||
TaskDispatchRequest taskDispatchRequest = JSONUtils.parseObject(message.getBody(), TaskDispatchRequest.class); |
||||
log.info("Receive task dispatch request, command: {}", taskDispatchRequest); |
||||
TaskExecutionContext taskExecutionContext = taskDispatchRequest.getTaskExecutionContext(); |
||||
taskExecutionContext.setLogPath(LogUtils.getTaskInstanceLogFullPath(taskExecutionContext)); |
||||
try ( |
||||
final LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC( |
||||
taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId()); |
||||
final LogUtils.MDCAutoClosableContext mdcAutoClosableContext1 = |
||||
LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath())) { |
||||
MasterTaskExecutionContextHolder.putTaskExecutionContext(taskExecutionContext); |
||||
// todo: calculate the delay in master dispatcher then we don't need to use a queue to store the task
|
||||
long remainTime = |
||||
DateUtils.getRemainTime(DateUtils.timeStampToDate(taskExecutionContext.getFirstSubmitTime()), |
||||
TimeUnit.SECONDS.toMillis(taskExecutionContext.getDelayTime())); |
||||
|
||||
// todo: move this to the master delay queue
|
||||
if (remainTime > 0) { |
||||
log.info("Current taskInstance: {} is choose delay execution, delay time: {}ms, remainTime: {}ms", |
||||
taskExecutionContext.getTaskName(), |
||||
TimeUnit.SECONDS.toMillis(taskExecutionContext.getDelayTime()), remainTime); |
||||
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.DELAY_EXECUTION); |
||||
masterMessageSenderManager.getMasterTaskExecuteResultMessageSender().sendMessage(taskExecutionContext); |
||||
} |
||||
|
||||
MasterDelayTaskExecuteRunnable masterDelayTaskExecuteRunnable = masterTaskExecuteRunnableFactoryBuilder |
||||
.createWorkerDelayTaskExecuteRunnableFactory(taskExecutionContext.getTaskType()) |
||||
.createWorkerTaskExecuteRunnable(taskExecutionContext); |
||||
if (masterDelayTaskExecuteRunnableDelayQueue |
||||
.submitMasterDelayTaskExecuteRunnable(masterDelayTaskExecuteRunnable)) { |
||||
log.info( |
||||
"Submit task: {} to MasterDelayTaskExecuteRunnableDelayQueue success", |
||||
taskExecutionContext.getTaskName()); |
||||
sendDispatchSuccessResult(channel, message, taskExecutionContext); |
||||
} else { |
||||
log.error( |
||||
"Submit task: {} to MasterDelayTaskExecuteRunnableDelayQueue failed, current task waiting queue size: {} is full", |
||||
taskExecutionContext.getTaskName(), masterDelayTaskExecuteRunnableDelayQueue.size()); |
||||
sendDispatchRejectResult(channel, message, taskExecutionContext); |
||||
} |
||||
} catch (Exception ex) { |
||||
log.error("Handle task dispatch request error, command: {}", taskDispatchRequest, ex); |
||||
sendDispatchFailedResult(channel, message, taskExecutionContext, ex); |
||||
} |
||||
} |
||||
|
||||
private void sendDispatchSuccessResult(Channel channel, Message dispatchRequest, |
||||
TaskExecutionContext taskExecutionContext) { |
||||
TaskDispatchResponse taskDispatchResponse = |
||||
TaskDispatchResponse.success(taskExecutionContext.getTaskInstanceId()); |
||||
channel.writeAndFlush(taskDispatchResponse.convert2Command(dispatchRequest.getOpaque())); |
||||
} |
||||
|
||||
private void sendDispatchRejectResult(Channel channel, Message dispatchRequest, |
||||
TaskExecutionContext taskExecutionContext) { |
||||
TaskDispatchResponse taskDispatchResponse = |
||||
TaskDispatchResponse.failed(taskExecutionContext.getTaskInstanceId(), "Task dispatch queue is full"); |
||||
channel.writeAndFlush(taskDispatchResponse.convert2Command(dispatchRequest.getOpaque())); |
||||
} |
||||
|
||||
private void sendDispatchFailedResult(Channel channel, Message dispatchRequest, |
||||
TaskExecutionContext taskExecutionContext, Throwable throwable) { |
||||
TaskDispatchResponse taskDispatchResponse = |
||||
TaskDispatchResponse.failed(taskExecutionContext.getTaskInstanceId(), throwable.getMessage()); |
||||
channel.writeAndFlush(taskDispatchResponse.convert2Command(dispatchRequest.getOpaque())); |
||||
} |
||||
|
||||
@Override |
||||
public MessageType getCommandType() { |
||||
return MessageType.TASK_DISPATCH_REQUEST; |
||||
} |
||||
} |
@ -0,0 +1,83 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.processor; |
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; |
||||
import org.apache.dolphinscheduler.remote.command.Message; |
||||
import org.apache.dolphinscheduler.remote.command.MessageType; |
||||
import org.apache.dolphinscheduler.remote.command.task.TaskKillRequest; |
||||
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; |
||||
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; |
||||
import org.apache.dolphinscheduler.server.master.runner.MasterDelayTaskExecuteRunnableDelayQueue; |
||||
import org.apache.dolphinscheduler.server.master.runner.MasterTaskExecuteRunnableThreadPool; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnable; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutionContextHolder; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
import io.netty.channel.Channel; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class MasterTaskKillProcessor implements MasterRpcProcessor { |
||||
|
||||
@Autowired |
||||
private MasterTaskExecuteRunnableThreadPool masterTaskExecuteRunnableThreadPool; |
||||
|
||||
@Autowired |
||||
private MasterDelayTaskExecuteRunnableDelayQueue masterDelayTaskExecuteRunnableDelayQueue; |
||||
|
||||
@Override |
||||
public void process(Channel channel, Message message) { |
||||
TaskKillRequest taskKillRequest = JSONUtils.parseObject(message.getBody(), TaskKillRequest.class); |
||||
log.info("Master receive task kill request: {}", taskKillRequest); |
||||
int taskInstanceId = taskKillRequest.getTaskInstanceId(); |
||||
try (LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setTaskInstanceIdMDC(taskInstanceId)) { |
||||
TaskExecutionContext taskExecutionContext = |
||||
MasterTaskExecutionContextHolder.getTaskExecutionContext(taskInstanceId); |
||||
if (taskExecutionContext == null) { |
||||
log.error("Cannot find the TaskExecutionContext, this task may already been killed"); |
||||
return; |
||||
} |
||||
MasterTaskExecuteRunnable masterTaskExecuteRunnable = |
||||
masterTaskExecuteRunnableThreadPool.getMasterTaskExecuteRunnable(taskInstanceId); |
||||
if (masterTaskExecuteRunnable == null) { |
||||
log.error("Cannot find the MasterTaskExecuteRunnable, this task may already been killed"); |
||||
return; |
||||
} |
||||
try { |
||||
masterTaskExecuteRunnable.cancelTask(); |
||||
masterDelayTaskExecuteRunnableDelayQueue |
||||
.removeMasterDelayTaskExecuteRunnable(masterTaskExecuteRunnable); |
||||
} catch (MasterTaskExecuteException e) { |
||||
log.error("Cancel MasterTaskExecuteRunnable failed ", e); |
||||
} |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public MessageType getCommandType() { |
||||
return MessageType.TASK_KILL_REQUEST; |
||||
} |
||||
|
||||
} |
@ -0,0 +1,68 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.processor; |
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; |
||||
import org.apache.dolphinscheduler.remote.command.Message; |
||||
import org.apache.dolphinscheduler.remote.command.MessageType; |
||||
import org.apache.dolphinscheduler.remote.command.task.TaskPauseRequest; |
||||
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor; |
||||
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; |
||||
import org.apache.dolphinscheduler.server.master.runner.MasterTaskExecuteRunnableThreadPool; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnable; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
import io.netty.channel.Channel; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class MasterTaskPauseProcessor implements MasterRpcProcessor { |
||||
|
||||
@Autowired |
||||
private MasterTaskExecuteRunnableThreadPool masterTaskExecuteRunnableThreadPool; |
||||
|
||||
@Override |
||||
public void process(Channel channel, Message message) { |
||||
TaskPauseRequest taskPauseRequest = JSONUtils.parseObject(message.getBody(), TaskPauseRequest.class); |
||||
MasterTaskExecuteRunnable masterTaskExecuteRunnable = |
||||
masterTaskExecuteRunnableThreadPool.getMasterTaskExecuteRunnable(taskPauseRequest.getTaskInstanceId()); |
||||
if (masterTaskExecuteRunnable == null) { |
||||
log.info("Cannot find the MasterTaskExecuteRunnable"); |
||||
return; |
||||
} |
||||
TaskExecutionContext taskExecutionContext = masterTaskExecuteRunnable.getTaskExecutionContext(); |
||||
try ( |
||||
LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC( |
||||
taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId())) { |
||||
masterTaskExecuteRunnable.pauseTask(); |
||||
} catch (MasterTaskExecuteException e) { |
||||
log.error("Pause MasterTaskExecuteRunnable failed", e); |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public MessageType getCommandType() { |
||||
return MessageType.PAUSE_TASK_INSTANCE; |
||||
} |
||||
} |
@ -1,68 +0,0 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.processor; |
||||
|
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; |
||||
import org.apache.dolphinscheduler.remote.command.Message; |
||||
import org.apache.dolphinscheduler.remote.command.MessageType; |
||||
import org.apache.dolphinscheduler.remote.command.task.TaskRejectMessage; |
||||
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; |
||||
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; |
||||
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
import io.netty.channel.Channel; |
||||
|
||||
/** |
||||
* task recall processor |
||||
*/ |
||||
@Component |
||||
@Slf4j |
||||
public class TaskRecallProcessor implements NettyRequestProcessor { |
||||
|
||||
@Autowired |
||||
private TaskEventService taskEventService; |
||||
|
||||
/** |
||||
* task ack process |
||||
* |
||||
* @param channel channel channel |
||||
* @param message command TaskExecuteAckCommand |
||||
*/ |
||||
@Override |
||||
public void process(Channel channel, Message message) { |
||||
TaskRejectMessage recallCommand = JSONUtils.parseObject(message.getBody(), TaskRejectMessage.class); |
||||
TaskEvent taskEvent = TaskEvent.newRecallEvent(recallCommand, channel); |
||||
try ( |
||||
final LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC( |
||||
recallCommand.getProcessInstanceId(), recallCommand.getTaskInstanceId())) { |
||||
log.info("Receive task recall command: {}", recallCommand); |
||||
taskEventService.addEvent(taskEvent); |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public MessageType getCommandType() { |
||||
return MessageType.TASK_REJECT; |
||||
} |
||||
} |
@ -0,0 +1,132 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner; |
||||
|
||||
import static com.google.common.base.Preconditions.checkNotNull; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.TaskEventType; |
||||
import org.apache.dolphinscheduler.common.utils.DateUtils; |
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; |
||||
import org.apache.dolphinscheduler.remote.command.Message; |
||||
import org.apache.dolphinscheduler.remote.command.task.TaskDispatchRequest; |
||||
import org.apache.dolphinscheduler.remote.command.task.TaskDispatchResponse; |
||||
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; |
||||
import org.apache.dolphinscheduler.remote.utils.Host; |
||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; |
||||
import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException; |
||||
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; |
||||
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; |
||||
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; |
||||
import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatcher; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable; |
||||
|
||||
import java.util.Date; |
||||
import java.util.Optional; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
@Slf4j |
||||
public abstract class BaseTaskDispatcher implements TaskDispatcher { |
||||
|
||||
protected final TaskEventService taskEventService; |
||||
protected final MasterConfig masterConfig; |
||||
protected final MasterRpcClient masterRpcClient; |
||||
|
||||
protected BaseTaskDispatcher(TaskEventService taskEventService, |
||||
MasterConfig masterConfig, |
||||
MasterRpcClient masterRpcClient) { |
||||
this.taskEventService = checkNotNull(taskEventService); |
||||
this.masterConfig = checkNotNull(masterConfig); |
||||
this.masterRpcClient = checkNotNull(masterRpcClient); |
||||
} |
||||
|
||||
@Override |
||||
public void dispatchTask(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException { |
||||
Host taskInstanceDispatchHost; |
||||
try { |
||||
taskInstanceDispatchHost = getTaskInstanceDispatchHost(taskExecuteRunnable) |
||||
.orElseThrow(() -> new TaskDispatchException("Cannot find the host to execute task.")); |
||||
} catch (WorkerGroupNotFoundException workerGroupNotFoundException) { |
||||
log.error("Dispatch task: {} failed, worker group not found.", |
||||
taskExecuteRunnable.getTaskExecutionContext().getTaskName(), workerGroupNotFoundException); |
||||
addDispatchFailedEvent(taskExecuteRunnable); |
||||
return; |
||||
} |
||||
taskExecuteRunnable.getTaskExecutionContext().setHost(taskInstanceDispatchHost.getAddress()); |
||||
doDispatch(taskExecuteRunnable); |
||||
taskExecuteRunnable.getTaskInstance().setHost(taskInstanceDispatchHost.getAddress()); |
||||
log.info("Success dispatch task {} to {}.", taskExecuteRunnable.getTaskExecutionContext().getTaskName(), |
||||
taskInstanceDispatchHost.getAddress()); |
||||
addDispatchEvent(taskExecuteRunnable); |
||||
} |
||||
|
||||
protected abstract Optional<Host> getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecutionContext) throws TaskDispatchException, WorkerGroupNotFoundException; |
||||
|
||||
protected void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException { |
||||
TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); |
||||
TaskDispatchRequest taskDispatchRequest = new TaskDispatchRequest(taskExecutionContext); |
||||
try { |
||||
Message message = masterRpcClient.sendSyncCommand(Host.of(taskExecutionContext.getHost()), |
||||
taskDispatchRequest.convert2Command()); |
||||
TaskDispatchResponse taskDispatchResponse = |
||||
JSONUtils.parseObject(message.getBody(), TaskDispatchResponse.class); |
||||
if (!taskDispatchResponse.isDispatchSuccess()) { |
||||
throw new TaskDispatchException(String.format("Dispatch task to %s failed, response is: %s", |
||||
taskExecutionContext.getHost(), taskDispatchResponse)); |
||||
} |
||||
} catch (InterruptedException e) { |
||||
// This exception should only happen when we close the server.
|
||||
Thread.currentThread().interrupt(); |
||||
throw new TaskDispatchException(String.format("Dispatch task to %s failed, get response failed", |
||||
taskExecutionContext.getHost()), e); |
||||
} catch (RemotingException e) { |
||||
throw new TaskDispatchException(String.format("Dispatch task to %s failed", |
||||
taskExecutionContext.getHost()), e); |
||||
} |
||||
} |
||||
|
||||
protected void addDispatchEvent(TaskExecuteRunnable taskExecuteRunnable) { |
||||
TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); |
||||
TaskEvent taskEvent = TaskEvent.newDispatchEvent( |
||||
taskExecutionContext.getProcessInstanceId(), |
||||
taskExecutionContext.getTaskInstanceId(), |
||||
taskExecutionContext.getHost()); |
||||
taskEventService.addEvent(taskEvent); |
||||
} |
||||
|
||||
private void addDispatchFailedEvent(TaskExecuteRunnable taskExecuteRunnable) { |
||||
TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); |
||||
TaskEvent taskEvent = TaskEvent.builder() |
||||
.processInstanceId(taskExecutionContext.getProcessInstanceId()) |
||||
.taskInstanceId(taskExecutionContext.getTaskInstanceId()) |
||||
.state(TaskExecutionStatus.FAILURE) |
||||
.logPath(taskExecutionContext.getLogPath()) |
||||
.executePath(taskExecutionContext.getExecutePath()) |
||||
.appIds(taskExecutionContext.getAppIds()) |
||||
.processId(taskExecutionContext.getProcessId()) |
||||
.varPool(taskExecutionContext.getVarPool()) |
||||
.startTime(DateUtils.timeStampToDate(taskExecutionContext.getStartTime())) |
||||
.endTime(new Date()) |
||||
.event(TaskEventType.RESULT) |
||||
.build(); |
||||
taskEventService.addEvent(taskEvent); |
||||
} |
||||
} |
@ -0,0 +1,46 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner; |
||||
|
||||
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; |
||||
|
||||
import java.util.concurrent.PriorityBlockingQueue; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class GlobalTaskDispatchWaitingQueue { |
||||
|
||||
private final PriorityBlockingQueue<DefaultTaskExecuteRunnable> queue = new PriorityBlockingQueue<>(); |
||||
|
||||
public void submitNeedToDispatchTaskExecuteRunnable(DefaultTaskExecuteRunnable priorityTaskExecuteRunnable) { |
||||
queue.put(priorityTaskExecuteRunnable); |
||||
} |
||||
|
||||
public DefaultTaskExecuteRunnable takeNeedToDispatchTaskExecuteRunnable() throws InterruptedException { |
||||
return queue.take(); |
||||
} |
||||
|
||||
public int getWaitingDispatchTaskNumber() { |
||||
return queue.size(); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,99 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner; |
||||
|
||||
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; |
||||
import org.apache.dolphinscheduler.common.thread.ThreadUtils; |
||||
import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatchFactory; |
||||
import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatcher; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; |
||||
|
||||
import java.util.concurrent.atomic.AtomicBoolean; |
||||
import java.util.concurrent.atomic.AtomicInteger; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class GlobalTaskDispatchWaitingQueueLooper extends BaseDaemonThread implements AutoCloseable { |
||||
|
||||
@Autowired |
||||
private GlobalTaskDispatchWaitingQueue globalTaskDispatchWaitingQueue; |
||||
|
||||
@Autowired |
||||
private TaskDispatchFactory taskDispatchFactory; |
||||
|
||||
private final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false); |
||||
|
||||
private final AtomicInteger DISPATCHED_TIMES = new AtomicInteger(); |
||||
|
||||
private static final Integer MAX_DISPATCHED_FAILED_TIMES = 100; |
||||
|
||||
public GlobalTaskDispatchWaitingQueueLooper() { |
||||
super("GlobalTaskDispatchWaitingQueueLooper"); |
||||
} |
||||
|
||||
@Override |
||||
public synchronized void start() { |
||||
if (!RUNNING_FLAG.compareAndSet(false, true)) { |
||||
log.error("The GlobalTaskDispatchWaitingQueueLooper already started, will not start again"); |
||||
return; |
||||
} |
||||
log.info("GlobalTaskDispatchWaitingQueueLooper starting..."); |
||||
super.start(); |
||||
log.info("GlobalTaskDispatchWaitingQueueLooper started..."); |
||||
} |
||||
|
||||
@Override |
||||
public void run() { |
||||
DefaultTaskExecuteRunnable defaultTaskExecuteRunnable; |
||||
while (RUNNING_FLAG.get()) { |
||||
try { |
||||
defaultTaskExecuteRunnable = globalTaskDispatchWaitingQueue.takeNeedToDispatchTaskExecuteRunnable(); |
||||
} catch (InterruptedException e) { |
||||
log.warn("Get waiting dispatch task failed, the current thread has been interrupted, will stop loop"); |
||||
Thread.currentThread().interrupt(); |
||||
break; |
||||
} |
||||
try { |
||||
final TaskDispatcher taskDispatcher = taskDispatchFactory |
||||
.getTaskDispatcher(defaultTaskExecuteRunnable.getTaskInstance().getTaskType()); |
||||
taskDispatcher.dispatchTask(defaultTaskExecuteRunnable); |
||||
DISPATCHED_TIMES.set(0); |
||||
} catch (Exception e) { |
||||
globalTaskDispatchWaitingQueue.submitNeedToDispatchTaskExecuteRunnable(defaultTaskExecuteRunnable); |
||||
if (DISPATCHED_TIMES.incrementAndGet() > MAX_DISPATCHED_FAILED_TIMES) { |
||||
ThreadUtils.sleep(10 * 1000L); |
||||
} |
||||
log.error("Dispatch task failed", e); |
||||
} |
||||
} |
||||
log.info("GlobalTaskDispatchWaitingQueueLooper started..."); |
||||
} |
||||
|
||||
@Override |
||||
public void close() throws Exception { |
||||
if (RUNNING_FLAG.compareAndSet(true, false)) { |
||||
log.info("GlobalTaskDispatchWaitingQueueLooper stopping..."); |
||||
log.info("GlobalTaskDispatchWaitingQueueLooper stopped..."); |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,54 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner; |
||||
|
||||
import org.apache.dolphinscheduler.server.master.runner.execute.MasterDelayTaskExecuteRunnable; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnable; |
||||
|
||||
import java.util.concurrent.DelayQueue; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
/** |
||||
* |
||||
*/ |
||||
@Component |
||||
public class MasterDelayTaskExecuteRunnableDelayQueue { |
||||
|
||||
private final DelayQueue<MasterDelayTaskExecuteRunnable> masterDelayTaskExecuteRunnableDelayQueue = |
||||
new DelayQueue<>(); |
||||
|
||||
public boolean submitMasterDelayTaskExecuteRunnable(MasterDelayTaskExecuteRunnable masterDelayTaskExecuteRunnable) { |
||||
return masterDelayTaskExecuteRunnableDelayQueue.offer(masterDelayTaskExecuteRunnable); |
||||
} |
||||
|
||||
public MasterDelayTaskExecuteRunnable takeMasterDelayTaskExecuteRunnable() throws InterruptedException { |
||||
return masterDelayTaskExecuteRunnableDelayQueue.take(); |
||||
} |
||||
|
||||
// todo: if we move the delay process to master, than we don't need this method, since dispatchProcess can directly
|
||||
// submit to thread pool
|
||||
public boolean removeMasterDelayTaskExecuteRunnable(MasterTaskExecuteRunnable masterTaskExecuteRunnable) { |
||||
return masterDelayTaskExecuteRunnableDelayQueue.remove(masterTaskExecuteRunnable); |
||||
} |
||||
|
||||
public int size() { |
||||
return masterDelayTaskExecuteRunnableDelayQueue.size(); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,81 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner; |
||||
|
||||
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.MasterDelayTaskExecuteRunnable; |
||||
|
||||
import java.util.concurrent.atomic.AtomicBoolean; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class MasterDelayTaskExecuteRunnableDelayQueueLooper extends BaseDaemonThread implements AutoCloseable { |
||||
|
||||
@Autowired |
||||
private MasterDelayTaskExecuteRunnableDelayQueue masterDelayTaskExecuteRunnableDelayQueue; |
||||
|
||||
@Autowired |
||||
private MasterTaskExecuteRunnableThreadPool masterTaskExecuteRunnableThreadPool; |
||||
|
||||
private final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false); |
||||
|
||||
public MasterDelayTaskExecuteRunnableDelayQueueLooper() { |
||||
super("MasterDelayTaskExecuteRunnableDelayQueueLooper"); |
||||
} |
||||
|
||||
@Override |
||||
public synchronized void start() { |
||||
if (!RUNNING_FLAG.compareAndSet(false, true)) { |
||||
log.error("The MasterDelayTaskExecuteRunnableDelayQueueLooper already started, will not start again"); |
||||
return; |
||||
} |
||||
log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper starting..."); |
||||
super.start(); |
||||
masterTaskExecuteRunnableThreadPool.start(); |
||||
log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper started..."); |
||||
} |
||||
|
||||
@Override |
||||
public void run() { |
||||
while (RUNNING_FLAG.get()) { |
||||
try { |
||||
final MasterDelayTaskExecuteRunnable masterDelayTaskExecuteRunnable = |
||||
masterDelayTaskExecuteRunnableDelayQueue.takeMasterDelayTaskExecuteRunnable(); |
||||
masterTaskExecuteRunnableThreadPool.submitMasterTaskExecuteRunnable(masterDelayTaskExecuteRunnable); |
||||
} catch (InterruptedException ex) { |
||||
Thread.currentThread().interrupt(); |
||||
log.warn("MasterDelayTaskExecuteRunnableDelayQueueLooper has been interrupted, will stop loop"); |
||||
break; |
||||
} |
||||
} |
||||
log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper stop loop..."); |
||||
} |
||||
|
||||
@Override |
||||
public void close() throws Exception { |
||||
if (RUNNING_FLAG.compareAndSet(true, false)) { |
||||
log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper stopping..."); |
||||
log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper stopped..."); |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,89 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner; |
||||
|
||||
import org.apache.dolphinscheduler.common.thread.ThreadUtils; |
||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnable; |
||||
|
||||
import java.util.Map; |
||||
import java.util.concurrent.ConcurrentHashMap; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
import com.google.common.util.concurrent.FutureCallback; |
||||
import com.google.common.util.concurrent.Futures; |
||||
import com.google.common.util.concurrent.ListenableFuture; |
||||
import com.google.common.util.concurrent.ListeningExecutorService; |
||||
import com.google.common.util.concurrent.MoreExecutors; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class MasterTaskExecuteRunnableThreadPool { |
||||
|
||||
@Autowired |
||||
private MasterConfig masterConfig; |
||||
|
||||
private static final Map<Integer, MasterTaskExecuteRunnable> SUBMITTED_MASTER_TASK_MAP = new ConcurrentHashMap<>(); |
||||
|
||||
private ListeningExecutorService listeningExecutorService; |
||||
|
||||
public synchronized void start() { |
||||
log.info("MasterTaskExecuteRunnableThreadPool starting..."); |
||||
this.listeningExecutorService = MoreExecutors.listeningDecorator(ThreadUtils.newDaemonFixedThreadExecutor( |
||||
"MasterTaskExecuteRunnableThread", masterConfig.getMasterTaskExecuteThreadPoolSize())); |
||||
log.info("MasterTaskExecuteRunnableThreadPool started..."); |
||||
} |
||||
|
||||
public void submitMasterTaskExecuteRunnable(MasterTaskExecuteRunnable masterTaskExecuteRunnable) { |
||||
ListenableFuture<?> future = listeningExecutorService.submit(masterTaskExecuteRunnable); |
||||
Futures.addCallback(future, new MasterTaskExecuteCallback(masterTaskExecuteRunnable), |
||||
this.listeningExecutorService); |
||||
SUBMITTED_MASTER_TASK_MAP.put(masterTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId(), |
||||
masterTaskExecuteRunnable); |
||||
} |
||||
|
||||
public MasterTaskExecuteRunnable getMasterTaskExecuteRunnable(Integer taskInstanceId) { |
||||
return SUBMITTED_MASTER_TASK_MAP.get(taskInstanceId); |
||||
} |
||||
|
||||
private static class MasterTaskExecuteCallback implements FutureCallback { |
||||
|
||||
private MasterTaskExecuteRunnable masterTaskExecuteRunnable; |
||||
|
||||
public MasterTaskExecuteCallback(MasterTaskExecuteRunnable masterTaskExecuteRunnable) { |
||||
this.masterTaskExecuteRunnable = masterTaskExecuteRunnable; |
||||
} |
||||
|
||||
@Override |
||||
public void onSuccess(Object result) { |
||||
log.info("MasterTaskExecuteRunnable execute success, will remove this task"); |
||||
SUBMITTED_MASTER_TASK_MAP.remove(masterTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId()); |
||||
} |
||||
|
||||
@Override |
||||
public void onFailure(Throwable t) { |
||||
log.info("MasterTaskExecuteRunnable execute failed, will remove this task"); |
||||
SUBMITTED_MASTER_TASK_MAP.remove(masterTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId()); |
||||
} |
||||
} |
||||
|
||||
} |
@ -0,0 +1,62 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner; |
||||
|
||||
import org.apache.dolphinscheduler.server.master.runner.execute.AsyncMasterTaskDelayQueueLooper; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class MasterTaskExecutorBootstrap implements AutoCloseable { |
||||
|
||||
@Autowired |
||||
private GlobalTaskDispatchWaitingQueueLooper globalTaskDispatchWaitingQueueLooper; |
||||
|
||||
@Autowired |
||||
private MasterDelayTaskExecuteRunnableDelayQueueLooper masterDelayTaskExecuteRunnableDelayQueueLooper; |
||||
|
||||
@Autowired |
||||
private AsyncMasterTaskDelayQueueLooper asyncMasterTaskDelayQueueLooper; |
||||
|
||||
public synchronized void start() { |
||||
log.info("MasterTaskExecutorBootstrap starting..."); |
||||
globalTaskDispatchWaitingQueueLooper.start(); |
||||
masterDelayTaskExecuteRunnableDelayQueueLooper.start(); |
||||
asyncMasterTaskDelayQueueLooper.start(); |
||||
log.info("MasterTaskExecutorBootstrap started..."); |
||||
} |
||||
|
||||
@Override |
||||
public void close() throws Exception { |
||||
log.info("MasterTaskExecutorBootstrap closing..."); |
||||
try ( |
||||
final GlobalTaskDispatchWaitingQueueLooper globalTaskDispatchWaitingQueueLooper1 = |
||||
globalTaskDispatchWaitingQueueLooper; |
||||
final MasterDelayTaskExecuteRunnableDelayQueueLooper masterDelayTaskExecuteRunnableDelayQueueLooper1 = |
||||
masterDelayTaskExecuteRunnableDelayQueueLooper; |
||||
final AsyncMasterTaskDelayQueueLooper asyncMasterTaskDelayQueueLooper1 = |
||||
asyncMasterTaskDelayQueueLooper) { |
||||
// closed the resource
|
||||
} |
||||
log.info("MasterTaskExecutorBootstrap closed..."); |
||||
} |
||||
} |
@ -0,0 +1,134 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.SlotCheckState; |
||||
import org.apache.dolphinscheduler.dao.entity.Command; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; |
||||
import org.apache.dolphinscheduler.dao.repository.TaskDefinitionLogDao; |
||||
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; |
||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||
import org.apache.dolphinscheduler.server.master.exception.WorkflowCreateException; |
||||
import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; |
||||
import org.apache.dolphinscheduler.server.master.registry.ServerNodeManager; |
||||
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnableFactory; |
||||
import org.apache.dolphinscheduler.service.alert.ProcessAlertManager; |
||||
import org.apache.dolphinscheduler.service.command.CommandService; |
||||
import org.apache.dolphinscheduler.service.expand.CuringParamsService; |
||||
import org.apache.dolphinscheduler.service.process.ProcessService; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class WorkflowExecuteRunnableFactory { |
||||
|
||||
@Autowired |
||||
private ServerNodeManager serverNodeManager; |
||||
|
||||
@Autowired |
||||
private CommandService commandService; |
||||
|
||||
@Autowired |
||||
private ProcessService processService; |
||||
|
||||
@Autowired |
||||
private ProcessInstanceDao processInstanceDao; |
||||
|
||||
@Autowired |
||||
private MasterRpcClient masterRpcClient; |
||||
|
||||
@Autowired |
||||
private ProcessAlertManager processAlertManager; |
||||
|
||||
@Autowired |
||||
private StateWheelExecuteThread stateWheelExecuteThread; |
||||
|
||||
@Autowired |
||||
private CuringParamsService curingGlobalParamsService; |
||||
|
||||
@Autowired |
||||
private TaskInstanceDao taskInstanceDao; |
||||
|
||||
@Autowired |
||||
private MasterConfig masterConfig; |
||||
|
||||
@Autowired |
||||
private TaskDefinitionLogDao taskDefinitionLogDao; |
||||
|
||||
@Autowired |
||||
private DefaultTaskExecuteRunnableFactory defaultTaskExecuteRunnableFactory; |
||||
|
||||
public WorkflowExecuteRunnable createWorkflowExecuteRunnable(Command command) throws WorkflowCreateException { |
||||
try { |
||||
ProcessInstance workflowInstance = createWorkflowInstance(command); |
||||
return new WorkflowExecuteRunnable(workflowInstance, |
||||
commandService, |
||||
processService, |
||||
processInstanceDao, |
||||
masterRpcClient, |
||||
processAlertManager, |
||||
masterConfig, |
||||
stateWheelExecuteThread, |
||||
curingGlobalParamsService, |
||||
taskInstanceDao, |
||||
taskDefinitionLogDao, |
||||
defaultTaskExecuteRunnableFactory); |
||||
} catch (Exception ex) { |
||||
throw new WorkflowCreateException("Create workflow execute runnable failed", ex); |
||||
} |
||||
} |
||||
|
||||
private ProcessInstance createWorkflowInstance(Command command) throws Exception { |
||||
long commandTransformStartTime = System.currentTimeMillis(); |
||||
// Note: this check is not safe, the slot may change after command transform.
|
||||
// We use the database transaction in `handleCommand` so that we can guarantee the command will
|
||||
// always be executed
|
||||
// by only one master
|
||||
SlotCheckState slotCheckState = slotCheck(command); |
||||
if (slotCheckState.equals(SlotCheckState.CHANGE) || slotCheckState.equals(SlotCheckState.INJECT)) { |
||||
log.info("Master handle command {} skip, slot check state: {}", command.getId(), slotCheckState); |
||||
throw new RuntimeException("Slot check failed the current state: " + slotCheckState); |
||||
} |
||||
ProcessInstance processInstance = processService.handleCommand(masterConfig.getMasterAddress(), command); |
||||
log.info("Master handle command {} end, create process instance {}", command.getId(), processInstance.getId()); |
||||
ProcessInstanceMetrics |
||||
.recordProcessInstanceGenerateTime(System.currentTimeMillis() - commandTransformStartTime); |
||||
return processInstance; |
||||
} |
||||
|
||||
private SlotCheckState slotCheck(Command command) { |
||||
int slot = serverNodeManager.getSlot(); |
||||
int masterSize = serverNodeManager.getMasterSize(); |
||||
SlotCheckState state; |
||||
if (masterSize <= 0) { |
||||
state = SlotCheckState.CHANGE; |
||||
} else if (command.getId() % masterSize == slot) { |
||||
state = SlotCheckState.PASS; |
||||
} else { |
||||
state = SlotCheckState.INJECT; |
||||
} |
||||
return state; |
||||
} |
||||
|
||||
} |
@ -0,0 +1,50 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.dispatcher; |
||||
|
||||
import org.apache.dolphinscheduler.remote.utils.Host; |
||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; |
||||
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; |
||||
import org.apache.dolphinscheduler.server.master.runner.BaseTaskDispatcher; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable; |
||||
|
||||
import java.util.Optional; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class MasterTaskDispatcher extends BaseTaskDispatcher { |
||||
|
||||
private final Optional<Host> masterTaskExecuteHost; |
||||
|
||||
public MasterTaskDispatcher(TaskEventService taskEventService, |
||||
MasterConfig masterConfig, |
||||
MasterRpcClient masterRpcClient) { |
||||
super(taskEventService, masterConfig, masterRpcClient); |
||||
masterTaskExecuteHost = Optional.of(Host.of(masterConfig.getMasterAddress())); |
||||
} |
||||
|
||||
@Override |
||||
protected Optional<Host> getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecutionContext) { |
||||
return masterTaskExecuteHost; |
||||
} |
||||
} |
@ -0,0 +1,41 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.dispatcher; |
||||
|
||||
import org.apache.dolphinscheduler.server.master.utils.TaskUtils; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class TaskDispatchFactory { |
||||
|
||||
@Autowired |
||||
private MasterTaskDispatcher masterTaskDispatcher; |
||||
|
||||
@Autowired |
||||
private WorkerTaskDispatcher workerTaskDispatcher; |
||||
|
||||
public TaskDispatcher getTaskDispatcher(String taskType) { |
||||
return TaskUtils.isMasterTask(taskType) ? masterTaskDispatcher : workerTaskDispatcher; |
||||
} |
||||
|
||||
} |
@ -0,0 +1,31 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.dispatcher; |
||||
|
||||
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; |
||||
import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable; |
||||
|
||||
/** |
||||
* Used to do task dispatcher. |
||||
*/ |
||||
public interface TaskDispatcher { |
||||
|
||||
void dispatchTask(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException, WorkerGroupNotFoundException; |
||||
|
||||
} |
@ -0,0 +1,57 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.dispatcher; |
||||
|
||||
import static com.google.common.base.Preconditions.checkNotNull; |
||||
|
||||
import org.apache.dolphinscheduler.remote.utils.Host; |
||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; |
||||
import org.apache.dolphinscheduler.server.master.dispatch.host.HostManager; |
||||
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; |
||||
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; |
||||
import org.apache.dolphinscheduler.server.master.runner.BaseTaskDispatcher; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable; |
||||
|
||||
import java.util.Optional; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class WorkerTaskDispatcher extends BaseTaskDispatcher { |
||||
|
||||
private final HostManager hostManager; |
||||
|
||||
public WorkerTaskDispatcher(TaskEventService taskEventService, |
||||
MasterConfig masterConfig, |
||||
MasterRpcClient masterRpcClient, |
||||
HostManager hostManager) { |
||||
super(taskEventService, masterConfig, masterRpcClient); |
||||
this.hostManager = checkNotNull(hostManager); |
||||
} |
||||
|
||||
@Override |
||||
protected Optional<Host> getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecuteRunnable) throws WorkerGroupNotFoundException { |
||||
String workerGroup = taskExecuteRunnable.getTaskExecutionContext().getWorkerGroup(); |
||||
return hostManager.select(workerGroup); |
||||
|
||||
} |
||||
} |
@ -0,0 +1,60 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; |
||||
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager; |
||||
import org.apache.dolphinscheduler.server.master.runner.task.IAsyncLogicTask; |
||||
import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
@Slf4j |
||||
public class AsyncMasterDelayTaskExecuteRunnable extends MasterDelayTaskExecuteRunnable { |
||||
|
||||
private final AsyncMasterTaskDelayQueue asyncMasterTaskDelayQueue; |
||||
|
||||
public AsyncMasterDelayTaskExecuteRunnable(TaskExecutionContext taskExecutionContext, |
||||
LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder, |
||||
MasterMessageSenderManager masterMessageSenderManager, |
||||
AsyncMasterTaskDelayQueue asyncTaskDelayQueue) { |
||||
super(taskExecutionContext, logicTaskPluginFactoryBuilder, masterMessageSenderManager); |
||||
this.asyncMasterTaskDelayQueue = asyncTaskDelayQueue; |
||||
} |
||||
|
||||
@Override |
||||
protected void executeTask() throws MasterTaskExecuteException { |
||||
if (logicTask == null) { |
||||
throw new MasterTaskExecuteException("The task plugin instance is null"); |
||||
} |
||||
final IAsyncLogicTask iAsyncLogicTask = (IAsyncLogicTask) logicTask; |
||||
// we execute the handle method here, but for async task, this method will not block
|
||||
// submit the task to async task queue
|
||||
final AsyncTaskExecutionContext asyncTaskExecutionContext = new AsyncTaskExecutionContext( |
||||
taskExecutionContext, |
||||
iAsyncLogicTask.getAsyncTaskExecuteFunction(), |
||||
new AsyncTaskCallbackFunctionImpl(this)); |
||||
asyncMasterTaskDelayQueue.addAsyncTask(asyncTaskExecutionContext); |
||||
} |
||||
|
||||
@Override |
||||
protected void afterExecute() { |
||||
// Do nothing, async task is not finished yet
|
||||
} |
||||
} |
@ -0,0 +1,49 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager; |
||||
import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class AsyncMasterDelayTaskExecuteRunnableFactory |
||||
implements |
||||
MasterDelayTaskExecuteRunnableFactory<AsyncMasterDelayTaskExecuteRunnable> { |
||||
|
||||
@Autowired |
||||
private LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder; |
||||
|
||||
@Autowired |
||||
private MasterMessageSenderManager masterMessageSenderManager; |
||||
|
||||
@Autowired |
||||
private AsyncMasterTaskDelayQueue asyncTaskDelayQueue; |
||||
|
||||
@Override |
||||
public AsyncMasterDelayTaskExecuteRunnable createWorkerTaskExecuteRunnable(TaskExecutionContext taskExecutionContext) { |
||||
return new AsyncMasterDelayTaskExecuteRunnable(taskExecutionContext, |
||||
logicTaskPluginFactoryBuilder, |
||||
masterMessageSenderManager, |
||||
asyncTaskDelayQueue); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,46 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import java.util.concurrent.DelayQueue; |
||||
|
||||
import javax.annotation.Nullable; |
||||
|
||||
import lombok.NonNull; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class AsyncMasterTaskDelayQueue { |
||||
|
||||
private final DelayQueue<AsyncTaskExecutionContext> asyncTaskCheckDelayQueue = new DelayQueue<>(); |
||||
|
||||
public void addAsyncTask(@NonNull AsyncTaskExecutionContext asyncTaskExecutionContext) { |
||||
asyncTaskExecutionContext.refreshStartTime(); |
||||
asyncTaskCheckDelayQueue.add(asyncTaskExecutionContext); |
||||
} |
||||
|
||||
public @Nullable AsyncTaskExecutionContext pollAsyncTask() throws InterruptedException { |
||||
return asyncTaskCheckDelayQueue.take(); |
||||
} |
||||
|
||||
public int getAsyncTaskRunningNum() { |
||||
return asyncTaskCheckDelayQueue.size(); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,132 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; |
||||
import org.apache.dolphinscheduler.common.thread.ThreadUtils; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; |
||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||
|
||||
import java.util.concurrent.ExecutorService; |
||||
import java.util.concurrent.atomic.AtomicBoolean; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class AsyncMasterTaskDelayQueueLooper extends BaseDaemonThread implements AutoCloseable { |
||||
|
||||
@Autowired |
||||
private AsyncMasterTaskDelayQueue asyncMasterTaskDelayQueue; |
||||
|
||||
@Autowired |
||||
private MasterConfig masterConfig; |
||||
|
||||
private static final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false); |
||||
|
||||
private ExecutorService asyncTaskStateCheckThreadPool; |
||||
|
||||
public AsyncMasterTaskDelayQueueLooper() { |
||||
super("AsyncMasterTaskDelayQueueLooper"); |
||||
} |
||||
|
||||
@Override |
||||
public synchronized void start() { |
||||
if (!RUNNING_FLAG.compareAndSet(false, true)) { |
||||
log.info("The AsyncMasterTaskDelayQueueLooper has already been started, will not start again"); |
||||
return; |
||||
} |
||||
|
||||
log.info("AsyncMasterTaskDelayQueueLooper starting..."); |
||||
super.start(); |
||||
log.info("AsyncMasterTaskDelayQueueLooper started..."); |
||||
} |
||||
|
||||
@Override |
||||
public void run() { |
||||
asyncTaskStateCheckThreadPool = ThreadUtils.newDaemonFixedThreadExecutor("AsyncTaskStateCheckThreadPool", |
||||
masterConfig.getMasterAsyncTaskStateCheckThreadPoolSize()); |
||||
while (RUNNING_FLAG.get()) { |
||||
AsyncTaskExecutionContext asyncTaskExecutionContext; |
||||
try { |
||||
asyncTaskExecutionContext = asyncMasterTaskDelayQueue.pollAsyncTask(); |
||||
} catch (InterruptedException e) { |
||||
log.error("AsyncConditionTaskLooper has been interrupted, will break this loop", e); |
||||
Thread.currentThread().interrupt(); |
||||
break; |
||||
} |
||||
final TaskExecutionContext taskExecutionContext = asyncTaskExecutionContext.getTaskExecutionContext(); |
||||
try ( |
||||
LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC( |
||||
taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId()); |
||||
LogUtils.MDCAutoClosableContext mdcAutoClosableContext1 = |
||||
LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath())) { |
||||
|
||||
if (MasterTaskExecutionContextHolder |
||||
.getTaskExecutionContext(taskExecutionContext.getTaskInstanceId()) == null) { |
||||
log.warn( |
||||
"Cannot find the taskInstance from TaskExecutionContextCacheManager, the task may already been killed, will stop the async master task"); |
||||
continue; |
||||
} |
||||
asyncTaskStateCheckThreadPool.submit(() -> { |
||||
final AsyncTaskExecuteFunction asyncTaskExecuteFunction = |
||||
asyncTaskExecutionContext.getAsyncTaskExecuteFunction(); |
||||
final AsyncTaskCallbackFunction asyncTaskCallbackFunction = |
||||
asyncTaskExecutionContext.getAsyncTaskCallbackFunction(); |
||||
try ( |
||||
LogUtils.MDCAutoClosableContext mdcAutoClosableContext2 = |
||||
LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath())) { |
||||
AsyncTaskExecuteFunction.AsyncTaskExecutionStatus asyncTaskExecutionStatus = |
||||
asyncTaskExecuteFunction.getAsyncTaskExecutionStatus(); |
||||
switch (asyncTaskExecutionStatus) { |
||||
case RUNNING: |
||||
// If the task status is running, means the task real status is not finished. We will
|
||||
// put it back to the queue to get the status again.
|
||||
asyncMasterTaskDelayQueue.addAsyncTask(asyncTaskExecutionContext); |
||||
break; |
||||
case SUCCESS: |
||||
asyncTaskCallbackFunction.executeSuccess(); |
||||
break; |
||||
case FAILED: |
||||
asyncTaskCallbackFunction.executeFailed(); |
||||
break; |
||||
} |
||||
} catch (Exception ex) { |
||||
asyncTaskCallbackFunction.executeThrowing(ex); |
||||
} |
||||
}); |
||||
} |
||||
} |
||||
log.info("AsyncMasterTaskDelayQueueLooper closed..."); |
||||
} |
||||
|
||||
@Override |
||||
public void close() throws Exception { |
||||
if (!RUNNING_FLAG.compareAndSet(true, false)) { |
||||
log.warn("The AsyncMasterTaskDelayQueueLooper is not started, will not close"); |
||||
return; |
||||
} |
||||
log.info("AsyncMasterTaskDelayQueueLooper closing..."); |
||||
asyncTaskStateCheckThreadPool.shutdown(); |
||||
log.info("AsyncMasterTaskDelayQueueLooper closed..."); |
||||
} |
||||
} |
@ -0,0 +1,30 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; |
||||
|
||||
public interface AsyncTaskCallbackFunction { |
||||
|
||||
void executeSuccess() throws MasterTaskExecuteException; |
||||
|
||||
void executeFailed() throws MasterTaskExecuteException; |
||||
|
||||
void executeThrowing(Throwable throwable); |
||||
|
||||
} |
@ -0,0 +1,66 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContextCacheManager; |
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; |
||||
import org.apache.dolphinscheduler.plugin.task.api.log.TaskInstanceLogHeader; |
||||
|
||||
import lombok.NonNull; |
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
@Slf4j |
||||
public class AsyncTaskCallbackFunctionImpl implements AsyncTaskCallbackFunction { |
||||
|
||||
private final AsyncMasterDelayTaskExecuteRunnable asyncMasterDelayTaskExecuteRunnable; |
||||
|
||||
public AsyncTaskCallbackFunctionImpl(@NonNull AsyncMasterDelayTaskExecuteRunnable asyncMasterDelayTaskExecuteRunnable) { |
||||
this.asyncMasterDelayTaskExecuteRunnable = asyncMasterDelayTaskExecuteRunnable; |
||||
} |
||||
|
||||
@Override |
||||
public void executeSuccess() { |
||||
asyncMasterDelayTaskExecuteRunnable.getTaskExecutionContext() |
||||
.setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS); |
||||
executeFinished(); |
||||
} |
||||
|
||||
@Override |
||||
public void executeFailed() { |
||||
asyncMasterDelayTaskExecuteRunnable.getTaskExecutionContext() |
||||
.setCurrentExecutionStatus(TaskExecutionStatus.FAILURE); |
||||
executeFinished(); |
||||
} |
||||
|
||||
@Override |
||||
public void executeThrowing(Throwable throwable) { |
||||
asyncMasterDelayTaskExecuteRunnable.afterThrowing(throwable); |
||||
} |
||||
|
||||
private void executeFinished() { |
||||
TaskInstanceLogHeader.printFinalizeTaskHeader(); |
||||
TaskExecutionContextCacheManager.removeByTaskInstanceId( |
||||
asyncMasterDelayTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId()); |
||||
log.info("Task execute finished, removed the TaskExecutionContext"); |
||||
asyncMasterDelayTaskExecuteRunnable.sendTaskResult(); |
||||
log.info( |
||||
"Execute task finished, will send the task execute result to master, the current task execute result is {}", |
||||
asyncMasterDelayTaskExecuteRunnable.getTaskExecutionContext().getCurrentExecutionStatus().name()); |
||||
asyncMasterDelayTaskExecuteRunnable.closeLogAppender(); |
||||
} |
||||
} |
@ -0,0 +1,43 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import java.time.Duration; |
||||
|
||||
import lombok.NonNull; |
||||
|
||||
public interface AsyncTaskExecuteFunction { |
||||
|
||||
@NonNull |
||||
AsyncTaskExecutionStatus getAsyncTaskExecutionStatus(); |
||||
|
||||
@NonNull |
||||
Duration getAsyncTaskStateCheckInterval(); |
||||
|
||||
enum AsyncTaskExecutionStatus { |
||||
|
||||
RUNNING, |
||||
|
||||
SUCCESS, |
||||
|
||||
FAILED, |
||||
; |
||||
|
||||
} |
||||
|
||||
} |
@ -0,0 +1,84 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy; |
||||
|
||||
import java.util.concurrent.Delayed; |
||||
import java.util.concurrent.TimeUnit; |
||||
|
||||
import lombok.Data; |
||||
import lombok.NonNull; |
||||
|
||||
@Data |
||||
public class AsyncTaskExecutionContext implements Delayed { |
||||
|
||||
private final TaskExecutionContext taskExecutionContext; |
||||
|
||||
private final AsyncTaskExecuteFunction asyncTaskExecuteFunction; |
||||
|
||||
private final AsyncTaskCallbackFunction asyncTaskCallbackFunction; |
||||
|
||||
private long currentStartTime; |
||||
private int executeTimes; |
||||
private final long executeInterval; |
||||
private long timeout; |
||||
|
||||
public AsyncTaskExecutionContext(@NonNull TaskExecutionContext taskExecutionContext, |
||||
@NonNull AsyncTaskExecuteFunction asyncTaskExecuteFunction, |
||||
@NonNull AsyncTaskCallbackFunction asyncTaskCallbackFunction) { |
||||
this.taskExecutionContext = taskExecutionContext; |
||||
this.asyncTaskExecuteFunction = asyncTaskExecuteFunction; |
||||
this.asyncTaskCallbackFunction = asyncTaskCallbackFunction; |
||||
this.currentStartTime = 0; |
||||
this.executeTimes = 0; |
||||
if (TaskTimeoutStrategy.FAILED.equals(taskExecutionContext.getTaskTimeoutStrategy()) |
||||
|| TaskTimeoutStrategy.WARNFAILED.equals(taskExecutionContext.getTaskTimeoutStrategy())) { |
||||
// will timeout
|
||||
this.timeout = taskExecutionContext.getStartTime() |
||||
+ TimeUnit.SECONDS.toMillis(taskExecutionContext.getTaskTimeout()); |
||||
} else { |
||||
this.timeout = TimeUnit.SECONDS.toMillis(Integer.MAX_VALUE); |
||||
} |
||||
this.executeInterval = Math.max(asyncTaskExecuteFunction.getAsyncTaskStateCheckInterval().toMillis(), 1000L); |
||||
} |
||||
|
||||
public void refreshStartTime() { |
||||
if (executeTimes == 0) { |
||||
// The first time doesn't have delay
|
||||
executeTimes++; |
||||
} else { |
||||
currentStartTime = System.currentTimeMillis(); |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public long getDelay(TimeUnit unit) { |
||||
long nextExecuteTimeDelay = Math.min(currentStartTime + executeInterval, timeout) - System.currentTimeMillis(); |
||||
return unit.convert(nextExecuteTimeDelay, TimeUnit.MILLISECONDS); |
||||
} |
||||
|
||||
@Override |
||||
public int compareTo(Delayed o) { |
||||
if (o == null) { |
||||
return 1; |
||||
} |
||||
return Long.compare(this.getDelay(TimeUnit.SECONDS), o.getDelay(TimeUnit.SECONDS)); |
||||
} |
||||
} |
@ -0,0 +1,58 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import static com.google.common.base.Preconditions.checkNotNull; |
||||
|
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.server.master.runner.operator.TaskOperatorManager; |
||||
|
||||
public class DefaultTaskExecuteRunnable extends PriorityTaskExecuteRunnable { |
||||
|
||||
private final TaskOperatorManager taskOperatorManager; |
||||
|
||||
public DefaultTaskExecuteRunnable(ProcessInstance workflowInstance, |
||||
TaskInstance taskInstance, |
||||
TaskExecutionContext taskExecutionContext, |
||||
TaskOperatorManager taskOperatorManager) { |
||||
super(workflowInstance, taskInstance, taskExecutionContext); |
||||
this.taskOperatorManager = checkNotNull(taskOperatorManager); |
||||
} |
||||
|
||||
@Override |
||||
public void dispatch() { |
||||
taskOperatorManager.getTaskDispatchOperator().handle(this); |
||||
} |
||||
|
||||
@Override |
||||
public void kill() { |
||||
taskOperatorManager.getTaskKillOperator().handle(this); |
||||
} |
||||
|
||||
@Override |
||||
public void pause() { |
||||
taskOperatorManager.getTaskPauseOperator().handle(this); |
||||
} |
||||
|
||||
@Override |
||||
public void timeout() { |
||||
taskOperatorManager.getTaskTimeoutOperator().handle(this); |
||||
} |
||||
} |
@ -0,0 +1,59 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance; |
||||
import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; |
||||
import org.apache.dolphinscheduler.server.master.exception.TaskExecuteRunnableCreateException; |
||||
import org.apache.dolphinscheduler.server.master.exception.TaskExecutionContextCreateException; |
||||
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable; |
||||
import org.apache.dolphinscheduler.server.master.runner.operator.TaskOperatorManager; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class DefaultTaskExecuteRunnableFactory implements TaskExecuteRunnableFactory<DefaultTaskExecuteRunnable> { |
||||
|
||||
@Autowired |
||||
private ProcessInstanceExecCacheManager processInstanceExecCacheManager; |
||||
|
||||
@Autowired |
||||
private TaskExecutionContextFactory taskExecutionContextFactory; |
||||
|
||||
@Autowired |
||||
private TaskOperatorManager taskOperatorManager; |
||||
|
||||
@Override |
||||
public DefaultTaskExecuteRunnable createTaskExecuteRunnable(TaskInstance taskInstance) throws TaskExecuteRunnableCreateException { |
||||
WorkflowExecuteRunnable workflowExecuteRunnable = |
||||
processInstanceExecCacheManager.getByProcessInstanceId(taskInstance.getProcessInstanceId()); |
||||
try { |
||||
return new DefaultTaskExecuteRunnable( |
||||
workflowExecuteRunnable.getProcessInstance(), |
||||
taskInstance, |
||||
taskExecutionContextFactory.createTaskExecutionContext(taskInstance), |
||||
taskOperatorManager); |
||||
} catch (TaskExecutionContextCreateException ex) { |
||||
throw new TaskExecuteRunnableCreateException("Create DefaultTaskExecuteRunnable failed", ex); |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,68 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.common.utils.DateUtils; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager; |
||||
import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder; |
||||
|
||||
import java.util.concurrent.Delayed; |
||||
import java.util.concurrent.TimeUnit; |
||||
|
||||
public abstract class MasterDelayTaskExecuteRunnable extends MasterTaskExecuteRunnable implements Delayed { |
||||
|
||||
public MasterDelayTaskExecuteRunnable(TaskExecutionContext taskExecutionContext, |
||||
LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder, |
||||
MasterMessageSenderManager masterMessageSenderManager) { |
||||
super(taskExecutionContext, logicTaskPluginFactoryBuilder, masterMessageSenderManager); |
||||
} |
||||
|
||||
@Override |
||||
public boolean equals(Object obj) { |
||||
if (!(obj instanceof MasterDelayTaskExecuteRunnable)) { |
||||
return false; |
||||
} |
||||
MasterDelayTaskExecuteRunnable other = (MasterDelayTaskExecuteRunnable) obj; |
||||
return other.getTaskExecutionContext().getTaskInstanceId() == this.getTaskExecutionContext() |
||||
.getTaskInstanceId(); |
||||
} |
||||
|
||||
@Override |
||||
public int hashCode() { |
||||
return this.getTaskExecutionContext().getTaskInstanceId(); |
||||
} |
||||
|
||||
@Override |
||||
public long getDelay(TimeUnit unit) { |
||||
TaskExecutionContext taskExecutionContext = getTaskExecutionContext(); |
||||
return unit.convert( |
||||
DateUtils.getRemainTime( |
||||
taskExecutionContext.getFirstSubmitTime(), taskExecutionContext.getDelayTime() * 60L), |
||||
TimeUnit.SECONDS); |
||||
} |
||||
|
||||
@Override |
||||
public int compareTo(Delayed o) { |
||||
if (o == null) { |
||||
return 1; |
||||
} |
||||
return Long.compare(this.getDelay(TimeUnit.MILLISECONDS), o.getDelay(TimeUnit.MILLISECONDS)); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,26 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
|
||||
public interface MasterDelayTaskExecuteRunnableFactory<T extends MasterDelayTaskExecuteRunnable> { |
||||
|
||||
T createWorkerTaskExecuteRunnable(TaskExecutionContext taskExecutionContext); |
||||
|
||||
} |
@ -0,0 +1,173 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import static ch.qos.logback.classic.ClassicConstants.FINALIZE_SESSION_MARKER; |
||||
|
||||
import org.apache.dolphinscheduler.common.log.remote.RemoteLogUtils; |
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContextCacheManager; |
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; |
||||
import org.apache.dolphinscheduler.plugin.task.api.log.TaskInstanceLogHeader; |
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils; |
||||
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; |
||||
import org.apache.dolphinscheduler.server.master.exception.LogicTaskFactoryNotFoundException; |
||||
import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException; |
||||
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; |
||||
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager; |
||||
import org.apache.dolphinscheduler.server.master.runner.task.ILogicTask; |
||||
import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
@Slf4j |
||||
public abstract class MasterTaskExecuteRunnable implements Runnable { |
||||
|
||||
protected final TaskExecutionContext taskExecutionContext; |
||||
protected final LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder; |
||||
protected final MasterMessageSenderManager masterMessageSenderManager; |
||||
protected ILogicTask logicTask; |
||||
|
||||
public MasterTaskExecuteRunnable(TaskExecutionContext taskExecutionContext, |
||||
LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder, |
||||
MasterMessageSenderManager masterMessageSenderManager) { |
||||
this.taskExecutionContext = taskExecutionContext; |
||||
this.logicTaskPluginFactoryBuilder = logicTaskPluginFactoryBuilder; |
||||
this.masterMessageSenderManager = masterMessageSenderManager; |
||||
} |
||||
|
||||
protected abstract void executeTask() throws MasterTaskExecuteException; |
||||
|
||||
protected abstract void afterExecute() throws MasterTaskExecuteException; |
||||
|
||||
protected void afterThrowing(Throwable throwable) { |
||||
try { |
||||
cancelTask(); |
||||
log.info("Get a exception when execute the task, canceled the task"); |
||||
} catch (Exception e) { |
||||
log.error("Cancel task failed,", e); |
||||
} |
||||
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.FAILURE); |
||||
sendTaskResult(); |
||||
log.info( |
||||
"Get a exception when execute the task, sent the task execute result to master, the current task execute result is {}", |
||||
taskExecutionContext.getCurrentExecutionStatus()); |
||||
TaskExecutionContextCacheManager.removeByTaskInstanceId(taskExecutionContext.getTaskInstanceId()); |
||||
log.info("Get a exception when execute the task, removed the TaskExecutionContext"); |
||||
} |
||||
|
||||
public void cancelTask() throws MasterTaskExecuteException { |
||||
if (logicTask != null) { |
||||
logicTask.kill(); |
||||
} |
||||
} |
||||
|
||||
public void pauseTask() throws MasterTaskExecuteException { |
||||
if (logicTask != null) { |
||||
logicTask.pause(); |
||||
} |
||||
} |
||||
|
||||
public TaskExecutionContext getTaskExecutionContext() { |
||||
return taskExecutionContext; |
||||
} |
||||
|
||||
@Override |
||||
public void run() { |
||||
try ( |
||||
final LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC( |
||||
taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId()); |
||||
final LogUtils.MDCAutoClosableContext mdcAutoClosableContext1 = |
||||
LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath())) { |
||||
TaskInstanceLogHeader.printInitializeTaskContextHeader(); |
||||
initializeTask(); |
||||
|
||||
TaskInstanceLogHeader.printLoadTaskInstancePluginHeader(); |
||||
beforeExecute(); |
||||
|
||||
TaskInstanceLogHeader.printExecuteTaskHeader(); |
||||
executeTask(); |
||||
|
||||
afterExecute(); |
||||
} catch (Throwable ex) { |
||||
log.error("Task execute failed, due to meet an exception", ex); |
||||
afterThrowing(ex); |
||||
closeLogAppender(); |
||||
} |
||||
} |
||||
|
||||
protected void initializeTask() { |
||||
log.info("Begin to initialize task"); |
||||
|
||||
long taskStartTime = System.currentTimeMillis(); |
||||
taskExecutionContext.setStartTime(taskStartTime); |
||||
log.info("Set task startTime: {}", taskStartTime); |
||||
|
||||
String taskAppId = String.format("%s_%s", taskExecutionContext.getProcessInstanceId(), |
||||
taskExecutionContext.getTaskInstanceId()); |
||||
taskExecutionContext.setTaskAppId(taskAppId); |
||||
log.info("Set task appId: {}", taskAppId); |
||||
|
||||
log.info("End initialize task {}", JSONUtils.toPrettyJsonString(taskExecutionContext)); |
||||
} |
||||
|
||||
protected void beforeExecute() throws LogicTaskFactoryNotFoundException, LogicTaskInitializeException, RemotingException { |
||||
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.RUNNING_EXECUTION); |
||||
masterMessageSenderManager.getMasterTaskExecuteRunningMessageSender().sendMessage(taskExecutionContext); |
||||
log.info("Send task status {} to master {}", taskExecutionContext.getCurrentExecutionStatus().name(), |
||||
taskExecutionContext.getWorkflowInstanceHost()); |
||||
|
||||
logicTask = logicTaskPluginFactoryBuilder.createILogicTaskPluginFactory(taskExecutionContext.getTaskType()) |
||||
.createLogicTask(taskExecutionContext); |
||||
log.info("Initialized task plugin instance: {} successfully", taskExecutionContext.getTaskType()); |
||||
|
||||
logicTask.getTaskParameters().setVarPool(taskExecutionContext.getVarPool()); |
||||
log.info("Initialize taskVarPool: {} successfully", taskExecutionContext.getVarPool()); |
||||
|
||||
} |
||||
|
||||
protected void closeLogAppender() { |
||||
try { |
||||
if (RemoteLogUtils.isRemoteLoggingEnable()) { |
||||
RemoteLogUtils.sendRemoteLog(taskExecutionContext.getLogPath()); |
||||
log.info("Send task log {} to remote storage successfully", taskExecutionContext.getLogPath()); |
||||
} |
||||
} catch (Exception ex) { |
||||
log.error("Send task log {} to remote storage failed", taskExecutionContext.getLogPath(), ex); |
||||
} finally { |
||||
log.info(FINALIZE_SESSION_MARKER, FINALIZE_SESSION_MARKER.toString()); |
||||
} |
||||
} |
||||
|
||||
protected void sendTaskResult() { |
||||
try { |
||||
taskExecutionContext.setEndTime(System.currentTimeMillis()); |
||||
taskExecutionContext.setVarPool(JSONUtils.toJsonString(logicTask.getTaskParameters().getVarPool())); |
||||
masterMessageSenderManager.getMasterTaskExecuteResultMessageSender().sendMessage(taskExecutionContext); |
||||
log.info("Send task status: {} to master: {} successfully", |
||||
taskExecutionContext.getCurrentExecutionStatus().name(), |
||||
taskExecutionContext.getWorkflowInstanceHost()); |
||||
} catch (Exception ex) { |
||||
log.error("Send task status: {} to master: {} failed", |
||||
taskExecutionContext.getCurrentExecutionStatus().name(), |
||||
taskExecutionContext.getWorkflowInstanceHost(), ex); |
||||
} |
||||
} |
||||
|
||||
} |
@ -0,0 +1,49 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.server.master.runner.task.dependent.DependentLogicTask; |
||||
import org.apache.dolphinscheduler.server.master.runner.task.subworkflow.SubWorkflowLogicTask; |
||||
|
||||
import java.util.Set; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
import com.google.common.collect.Sets; |
||||
|
||||
@Component |
||||
public class MasterTaskExecuteRunnableFactoryBuilder { |
||||
|
||||
@Autowired |
||||
private AsyncMasterDelayTaskExecuteRunnableFactory asyncMasterDelayTaskExecuteRunnableFactory; |
||||
|
||||
@Autowired |
||||
private SyncMasterDelayTaskExecuteRunnableFactory syncMasterDelayTaskExecuteRunnableFactory; |
||||
|
||||
private static final Set<String> ASYNC_TASK_TYPE = Sets.newHashSet( |
||||
DependentLogicTask.TASK_TYPE, |
||||
SubWorkflowLogicTask.TASK_TYPE); |
||||
|
||||
public MasterDelayTaskExecuteRunnableFactory<? extends MasterDelayTaskExecuteRunnable> createWorkerDelayTaskExecuteRunnableFactory(String taskType) { |
||||
if (ASYNC_TASK_TYPE.contains(taskType)) { |
||||
return asyncMasterDelayTaskExecuteRunnableFactory; |
||||
} |
||||
return syncMasterDelayTaskExecuteRunnableFactory; |
||||
} |
||||
} |
@ -0,0 +1,47 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
|
||||
import java.util.Map; |
||||
import java.util.concurrent.ConcurrentHashMap; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
@Slf4j |
||||
public class MasterTaskExecutionContextHolder { |
||||
|
||||
private static final Map<Integer, TaskExecutionContext> TASK_EXECUTION_CONTEXT_MAP = new ConcurrentHashMap<>(); |
||||
|
||||
public static void putTaskExecutionContext(TaskExecutionContext taskExecutionContext) { |
||||
if (TASK_EXECUTION_CONTEXT_MAP.containsKey(taskExecutionContext.getTaskInstanceId())) { |
||||
log.error("The TaskExecutionContext {} already exists in the MasterTaskExecutionContextHolder", |
||||
taskExecutionContext); |
||||
} |
||||
TASK_EXECUTION_CONTEXT_MAP.put(taskExecutionContext.getTaskInstanceId(), taskExecutionContext); |
||||
} |
||||
|
||||
public static TaskExecutionContext getTaskExecutionContext(int taskInstanceId) { |
||||
return TASK_EXECUTION_CONTEXT_MAP.get(taskInstanceId); |
||||
} |
||||
|
||||
public static void removeTaskExecutionContext(int taskInstanceId) { |
||||
TASK_EXECUTION_CONTEXT_MAP.remove(taskInstanceId); |
||||
} |
||||
} |
@ -0,0 +1,97 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import static com.google.common.base.Preconditions.checkNotNull; |
||||
|
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
|
||||
import org.jetbrains.annotations.NotNull; |
||||
|
||||
public abstract class PriorityTaskExecuteRunnable implements TaskExecuteRunnable, Comparable<TaskExecuteRunnable> { |
||||
|
||||
private final ProcessInstance workflowInstance; |
||||
private final TaskInstance taskInstance; |
||||
private final TaskExecutionContext taskExecutionContext; |
||||
|
||||
public PriorityTaskExecuteRunnable(ProcessInstance workflowInstance, |
||||
TaskInstance taskInstance, |
||||
TaskExecutionContext taskExecutionContext) { |
||||
this.taskInstance = checkNotNull(taskInstance); |
||||
this.workflowInstance = checkNotNull(workflowInstance); |
||||
this.taskExecutionContext = checkNotNull(taskExecutionContext); |
||||
} |
||||
|
||||
@Override |
||||
public ProcessInstance getWorkflowInstance() { |
||||
return workflowInstance; |
||||
} |
||||
|
||||
@Override |
||||
public TaskInstance getTaskInstance() { |
||||
return taskInstance; |
||||
} |
||||
|
||||
@Override |
||||
public TaskExecutionContext getTaskExecutionContext() { |
||||
return taskExecutionContext; |
||||
} |
||||
|
||||
@Override |
||||
public int compareTo(@NotNull TaskExecuteRunnable other) { |
||||
int workflowInstancePriorityCompareResult = workflowInstance.getProcessInstancePriority().getCode() |
||||
- other.getWorkflowInstance().getProcessInstancePriority().getCode(); |
||||
if (workflowInstancePriorityCompareResult != 0) { |
||||
return workflowInstancePriorityCompareResult; |
||||
} |
||||
int workflowInstanceIdCompareResult = workflowInstance.getId() - other.getWorkflowInstance().getId(); |
||||
if (workflowInstanceIdCompareResult != 0) { |
||||
return workflowInstanceIdCompareResult; |
||||
} |
||||
int taskInstancePriorityCompareResult = taskInstance.getTaskInstancePriority().getCode() |
||||
- other.getTaskInstance().getTaskInstancePriority().getCode(); |
||||
if (taskInstancePriorityCompareResult > 0) { |
||||
return taskInstancePriorityCompareResult; |
||||
} |
||||
// larger number, higher priority
|
||||
int taskGroupPriorityCompareResult = |
||||
taskInstance.getTaskGroupPriority() - other.getTaskInstance().getTaskGroupPriority(); |
||||
if (taskGroupPriorityCompareResult != 0) { |
||||
return -taskGroupPriorityCompareResult; |
||||
} |
||||
// The task instance shouldn't be equals
|
||||
return taskInstance.getId() - other.getTaskInstance().getId(); |
||||
} |
||||
|
||||
@Override |
||||
public boolean equals(Object obj) { |
||||
if (obj instanceof PriorityTaskExecuteRunnable) { |
||||
PriorityTaskExecuteRunnable other = (PriorityTaskExecuteRunnable) obj; |
||||
return compareTo(other) == 0; |
||||
} |
||||
return false; |
||||
} |
||||
|
||||
@Override |
||||
public int hashCode() { |
||||
return taskInstance.getId(); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,60 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.plugin.task.api.log.TaskInstanceLogHeader; |
||||
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException; |
||||
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager; |
||||
import org.apache.dolphinscheduler.server.master.runner.task.ISyncLogicTask; |
||||
import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
@Slf4j |
||||
public class SyncMasterDelayTaskExecuteRunnable extends MasterDelayTaskExecuteRunnable { |
||||
|
||||
public SyncMasterDelayTaskExecuteRunnable(TaskExecutionContext taskExecutionContext, |
||||
LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder, |
||||
MasterMessageSenderManager masterMessageSenderManager) { |
||||
super(taskExecutionContext, logicTaskPluginFactoryBuilder, masterMessageSenderManager); |
||||
} |
||||
|
||||
@Override |
||||
protected void executeTask() throws MasterTaskExecuteException { |
||||
if (logicTask == null) { |
||||
throw new MasterTaskExecuteException("The task plugin instance is null"); |
||||
} |
||||
ISyncLogicTask iSyncLogicTask = (ISyncLogicTask) logicTask; |
||||
iSyncLogicTask.handle(); |
||||
} |
||||
|
||||
protected void afterExecute() throws MasterTaskExecuteException { |
||||
TaskInstanceLogHeader.printFinalizeTaskHeader(); |
||||
// todo: move this to constructor to avoid check every time
|
||||
if (logicTask == null) { |
||||
throw new MasterTaskExecuteException("The current task instance is null"); |
||||
} |
||||
sendTaskResult(); |
||||
log.info( |
||||
"Execute task finished, will send the task execute result to master, the current task execute result is {}", |
||||
taskExecutionContext.getCurrentExecutionStatus().name()); |
||||
closeLogAppender(); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,45 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager; |
||||
import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class SyncMasterDelayTaskExecuteRunnableFactory |
||||
implements |
||||
MasterDelayTaskExecuteRunnableFactory<SyncMasterDelayTaskExecuteRunnable> { |
||||
|
||||
@Autowired |
||||
private LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder; |
||||
@Autowired |
||||
private MasterMessageSenderManager masterMessageSenderManager; |
||||
|
||||
@Override |
||||
public SyncMasterDelayTaskExecuteRunnable createWorkerTaskExecuteRunnable(TaskExecutionContext taskExecutionContext) { |
||||
return new SyncMasterDelayTaskExecuteRunnable(taskExecutionContext, logicTaskPluginFactoryBuilder, |
||||
masterMessageSenderManager); |
||||
} |
||||
} |
@ -0,0 +1,43 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance; |
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
|
||||
/** |
||||
* This interface is used to define a task which is executing. |
||||
* todo: split to MasterTaskExecuteRunnable and WorkerTaskExecuteRunnable |
||||
*/ |
||||
public interface TaskExecuteRunnable extends Comparable<TaskExecuteRunnable> { |
||||
|
||||
void dispatch(); |
||||
|
||||
void kill(); |
||||
|
||||
void pause(); |
||||
|
||||
void timeout(); |
||||
|
||||
ProcessInstance getWorkflowInstance(); |
||||
|
||||
TaskInstance getTaskInstance(); |
||||
|
||||
TaskExecutionContext getTaskExecutionContext(); |
||||
} |
@ -0,0 +1,27 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.execute; |
||||
|
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance; |
||||
import org.apache.dolphinscheduler.server.master.exception.TaskExecuteRunnableCreateException; |
||||
|
||||
public interface TaskExecuteRunnableFactory<T extends TaskExecuteRunnable> { |
||||
|
||||
T createTaskExecuteRunnable(TaskInstance taskInstance) throws TaskExecuteRunnableCreateException; |
||||
|
||||
} |
@ -0,0 +1,50 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.message; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.remote.command.BaseMessage; |
||||
import org.apache.dolphinscheduler.remote.command.MessageType; |
||||
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; |
||||
|
||||
// todo: unify with WorkerMessageSender
|
||||
public interface MasterMessageSender<T extends BaseMessage> { |
||||
|
||||
/** |
||||
* Send the message |
||||
* |
||||
* @throws RemotingException Cannot connect to the target host. |
||||
*/ |
||||
void sendMessage(T message) throws RemotingException; |
||||
|
||||
default void sendMessage(TaskExecutionContext taskExecutionContext) throws RemotingException { |
||||
T message = buildMessage(taskExecutionContext); |
||||
sendMessage(message); |
||||
} |
||||
|
||||
/** |
||||
* Build the message from task context and message received address. |
||||
*/ |
||||
T buildMessage(TaskExecutionContext taskExecutionContext); |
||||
|
||||
/** |
||||
* The message type can be sent by this sender. |
||||
*/ |
||||
MessageType getMessageType(); |
||||
|
||||
} |
@ -0,0 +1,50 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.message; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class MasterMessageSenderManager { |
||||
|
||||
@Autowired |
||||
private MasterTaskExecuteResultMessageSender masterTaskExecuteResultMessageSender; |
||||
|
||||
@Autowired |
||||
private MasterTaskExecuteRunningMessageSender masterTaskExecuteRunningMessageSender; |
||||
|
||||
@Autowired |
||||
private MasterTaskUpdateRuntimeMessageSender masterTaskUpdateRuntimeMessageSender; |
||||
|
||||
public MasterTaskExecuteResultMessageSender getMasterTaskExecuteResultMessageSender() { |
||||
return masterTaskExecuteResultMessageSender; |
||||
} |
||||
|
||||
public MasterTaskExecuteRunningMessageSender getMasterTaskExecuteRunningMessageSender() { |
||||
return masterTaskExecuteRunningMessageSender; |
||||
} |
||||
|
||||
public MasterTaskUpdateRuntimeMessageSender getMasterTaskUpdatePidMessageSender() { |
||||
return masterTaskUpdateRuntimeMessageSender; |
||||
} |
||||
|
||||
} |
@ -0,0 +1,70 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.message; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.remote.command.MessageType; |
||||
import org.apache.dolphinscheduler.remote.command.task.TaskExecuteResultMessage; |
||||
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; |
||||
import org.apache.dolphinscheduler.remote.utils.Host; |
||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class MasterTaskExecuteResultMessageSender implements MasterMessageSender<TaskExecuteResultMessage> { |
||||
|
||||
@Autowired |
||||
private MasterConfig masterConfig; |
||||
|
||||
@Autowired |
||||
private MasterRpcClient masterRpcClient; |
||||
|
||||
@Override |
||||
public void sendMessage(TaskExecuteResultMessage message) throws RemotingException { |
||||
masterRpcClient.send(Host.of(message.getMessageReceiverAddress()), message.convert2Command()); |
||||
} |
||||
|
||||
@Override |
||||
public TaskExecuteResultMessage buildMessage(TaskExecutionContext taskExecutionContext) { |
||||
TaskExecuteResultMessage taskExecuteResultMessage = |
||||
new TaskExecuteResultMessage(masterConfig.getMasterAddress(), |
||||
taskExecutionContext.getWorkflowInstanceHost(), |
||||
System.currentTimeMillis()); |
||||
taskExecuteResultMessage.setProcessInstanceId(taskExecutionContext.getProcessInstanceId()); |
||||
taskExecuteResultMessage.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); |
||||
taskExecuteResultMessage.setStatus(taskExecutionContext.getCurrentExecutionStatus().getCode()); |
||||
taskExecuteResultMessage.setLogPath(taskExecutionContext.getLogPath()); |
||||
taskExecuteResultMessage.setExecutePath(taskExecutionContext.getExecutePath()); |
||||
taskExecuteResultMessage.setAppIds(taskExecutionContext.getAppIds()); |
||||
taskExecuteResultMessage.setProcessId(taskExecutionContext.getProcessId()); |
||||
taskExecuteResultMessage.setHost(taskExecutionContext.getHost()); |
||||
taskExecuteResultMessage.setStartTime(taskExecutionContext.getStartTime()); |
||||
taskExecuteResultMessage.setEndTime(taskExecutionContext.getEndTime()); |
||||
taskExecuteResultMessage.setVarPool(taskExecutionContext.getVarPool()); |
||||
taskExecuteResultMessage.setExecutePath(taskExecutionContext.getExecutePath()); |
||||
return taskExecuteResultMessage; |
||||
} |
||||
|
||||
@Override |
||||
public MessageType getMessageType() { |
||||
return MessageType.TASK_EXECUTE_RESULT_MESSAGE; |
||||
} |
||||
} |
@ -0,0 +1,68 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.message; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.remote.command.MessageType; |
||||
import org.apache.dolphinscheduler.remote.command.task.TaskExecuteRunningMessage; |
||||
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; |
||||
import org.apache.dolphinscheduler.remote.utils.Host; |
||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; |
||||
|
||||
import lombok.NonNull; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class MasterTaskExecuteRunningMessageSender implements MasterMessageSender<TaskExecuteRunningMessage> { |
||||
|
||||
@Autowired |
||||
private MasterRpcClient masterRpcClient; |
||||
|
||||
@Autowired |
||||
private MasterConfig masterConfig; |
||||
|
||||
@Override |
||||
public void sendMessage(TaskExecuteRunningMessage message) throws RemotingException { |
||||
masterRpcClient.send(Host.of(message.getMessageReceiverAddress()), message.convert2Command()); |
||||
} |
||||
|
||||
@Override |
||||
public TaskExecuteRunningMessage buildMessage(@NonNull TaskExecutionContext taskExecutionContext) { |
||||
TaskExecuteRunningMessage taskExecuteRunningMessage = |
||||
new TaskExecuteRunningMessage(masterConfig.getMasterAddress(), |
||||
taskExecutionContext.getWorkflowInstanceHost(), |
||||
System.currentTimeMillis()); |
||||
taskExecuteRunningMessage.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); |
||||
taskExecuteRunningMessage.setProcessInstanceId(taskExecutionContext.getProcessInstanceId()); |
||||
taskExecuteRunningMessage.setStatus(taskExecutionContext.getCurrentExecutionStatus()); |
||||
taskExecuteRunningMessage.setLogPath(taskExecutionContext.getLogPath()); |
||||
taskExecuteRunningMessage.setHost(taskExecutionContext.getHost()); |
||||
taskExecuteRunningMessage.setStartTime(taskExecutionContext.getStartTime()); |
||||
taskExecuteRunningMessage.setExecutePath(taskExecutionContext.getExecutePath()); |
||||
taskExecuteRunningMessage.setAppIds(taskExecutionContext.getAppIds()); |
||||
return taskExecuteRunningMessage; |
||||
} |
||||
|
||||
@Override |
||||
public MessageType getMessageType() { |
||||
return MessageType.TASK_EXECUTE_RUNNING_MESSAGE; |
||||
} |
||||
} |
@ -0,0 +1,64 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.message; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
||||
import org.apache.dolphinscheduler.remote.command.MessageType; |
||||
import org.apache.dolphinscheduler.remote.command.task.TaskUpdateRuntimeMessage; |
||||
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; |
||||
import org.apache.dolphinscheduler.remote.utils.Host; |
||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; |
||||
|
||||
import lombok.NonNull; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class MasterTaskUpdateRuntimeMessageSender implements MasterMessageSender<TaskUpdateRuntimeMessage> { |
||||
|
||||
@Autowired |
||||
private MasterRpcClient masterRpcClient; |
||||
|
||||
@Autowired |
||||
private MasterConfig masterConfig; |
||||
|
||||
@Override |
||||
public void sendMessage(TaskUpdateRuntimeMessage message) throws RemotingException { |
||||
masterRpcClient.send(Host.of(message.getMessageReceiverAddress()), message.convert2Command()); |
||||
} |
||||
|
||||
@Override |
||||
public TaskUpdateRuntimeMessage buildMessage(@NonNull TaskExecutionContext taskExecutionContext) { |
||||
TaskUpdateRuntimeMessage taskUpdatePidRequest = |
||||
new TaskUpdateRuntimeMessage(masterConfig.getMasterAddress(), |
||||
taskExecutionContext.getWorkflowInstanceHost(), |
||||
System.currentTimeMillis()); |
||||
taskUpdatePidRequest.setTaskInstanceId(taskExecutionContext.getTaskInstanceId()); |
||||
taskUpdatePidRequest.setProcessInstanceId(taskExecutionContext.getProcessInstanceId()); |
||||
taskUpdatePidRequest.setHost(taskExecutionContext.getHost()); |
||||
taskUpdatePidRequest.setStartTime(taskExecutionContext.getStartTime()); |
||||
return taskUpdatePidRequest; |
||||
} |
||||
|
||||
@Override |
||||
public MessageType getMessageType() { |
||||
return MessageType.TASK_UPDATE_RUNTIME_MESSAGE; |
||||
} |
||||
} |
@ -0,0 +1,36 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.operator; |
||||
|
||||
import org.apache.dolphinscheduler.server.master.runner.GlobalTaskDispatchWaitingQueue; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class TaskDispatchOperator implements TaskOperator { |
||||
|
||||
@Autowired |
||||
private GlobalTaskDispatchWaitingQueue globalTaskDispatchWaitingQueue; |
||||
|
||||
@Override |
||||
public void handle(DefaultTaskExecuteRunnable taskExecuteRunnable) { |
||||
globalTaskDispatchWaitingQueue.submitNeedToDispatchTaskExecuteRunnable(taskExecuteRunnable); |
||||
} |
||||
} |
@ -0,0 +1,79 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.operator; |
||||
|
||||
import org.apache.dolphinscheduler.dao.entity.TaskInstance; |
||||
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao; |
||||
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; |
||||
import org.apache.dolphinscheduler.remote.command.task.TaskKillRequest; |
||||
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; |
||||
import org.apache.dolphinscheduler.remote.utils.Host; |
||||
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient; |
||||
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; |
||||
|
||||
import org.apache.commons.lang3.StringUtils; |
||||
|
||||
import java.util.Date; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class TaskKillOperator implements TaskOperator { |
||||
|
||||
@Autowired |
||||
private TaskInstanceDao taskInstanceDao; |
||||
|
||||
@Autowired |
||||
private MasterRpcClient masterRpcClient; |
||||
|
||||
@Override |
||||
public void handle(DefaultTaskExecuteRunnable taskExecuteRunnable) { |
||||
TaskInstance taskInstance = taskExecuteRunnable.getTaskInstance(); |
||||
log.info("Begin to kill task instance: {}", taskInstance.getName()); |
||||
if (taskInstance.getState().isFinished()) { |
||||
log.info("The task stance {} is finished, no need to kill", taskInstance.getName()); |
||||
return; |
||||
} |
||||
try { |
||||
killTaskInstanceInDB(taskInstance); |
||||
killRemoteTaskInstanceInThreadPool(taskInstance); |
||||
} catch (Exception ex) { |
||||
// todo: do we need to throw this exception?
|
||||
log.error("Kill task instance {} failed", taskInstance.getName(), ex); |
||||
} |
||||
} |
||||
|
||||
private void killTaskInstanceInDB(TaskInstance taskInstance) { |
||||
taskInstance.setState(TaskExecutionStatus.KILL); |
||||
taskInstance.setEndTime(new Date()); |
||||
taskInstanceDao.updateTaskInstance(taskInstance); |
||||
} |
||||
|
||||
private void killRemoteTaskInstanceInThreadPool(TaskInstance taskInstance) throws RemotingException { |
||||
if (StringUtils.isEmpty(taskInstance.getHost())) { |
||||
return; |
||||
} |
||||
TaskKillRequest killCommand = new TaskKillRequest(taskInstance.getId()); |
||||
masterRpcClient.send(Host.of(taskInstance.getHost()), killCommand.convert2Command()); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,26 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.server.master.runner.operator; |
||||
|
||||
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable; |
||||
|
||||
public interface TaskOperator { |
||||
|
||||
void handle(DefaultTaskExecuteRunnable taskExecuteRunnable); |
||||
|
||||
} |
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in new issue