|
|
@ -17,19 +17,19 @@ |
|
|
|
|
|
|
|
|
|
|
|
package org.apache.dolphinscheduler.server.master.service; |
|
|
|
package org.apache.dolphinscheduler.server.master.service; |
|
|
|
|
|
|
|
|
|
|
|
import static com.google.common.base.Preconditions.checkNotNull; |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.common.Constants; |
|
|
|
import org.apache.dolphinscheduler.common.Constants; |
|
|
|
import org.apache.dolphinscheduler.common.enums.NodeType; |
|
|
|
import org.apache.dolphinscheduler.common.enums.NodeType; |
|
|
|
import org.apache.dolphinscheduler.common.enums.StateEvent; |
|
|
|
import org.apache.dolphinscheduler.common.enums.StateEvent; |
|
|
|
import org.apache.dolphinscheduler.common.enums.StateEventType; |
|
|
|
import org.apache.dolphinscheduler.common.enums.StateEventType; |
|
|
|
import org.apache.dolphinscheduler.common.model.Server; |
|
|
|
import org.apache.dolphinscheduler.common.model.Server; |
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.common.utils.LoggerUtils; |
|
|
|
import org.apache.dolphinscheduler.common.utils.NetUtils; |
|
|
|
import org.apache.dolphinscheduler.common.utils.NetUtils; |
|
|
|
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
|
|
|
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
|
|
|
import org.apache.dolphinscheduler.dao.entity.TaskInstance; |
|
|
|
import org.apache.dolphinscheduler.dao.entity.TaskInstance; |
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus; |
|
|
|
import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus; |
|
|
|
import org.apache.dolphinscheduler.server.builder.TaskExecutionContextBuilder; |
|
|
|
import org.apache.dolphinscheduler.server.builder.TaskExecutionContextBuilder; |
|
|
|
|
|
|
|
import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager; |
|
|
|
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
|
|
|
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
|
|
|
import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; |
|
|
|
import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics; |
|
|
|
import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; |
|
|
|
import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics; |
|
|
@ -49,6 +49,7 @@ import java.util.Iterator; |
|
|
|
import java.util.List; |
|
|
|
import java.util.List; |
|
|
|
import java.util.Map; |
|
|
|
import java.util.Map; |
|
|
|
import java.util.concurrent.TimeUnit; |
|
|
|
import java.util.concurrent.TimeUnit; |
|
|
|
|
|
|
|
import java.util.stream.Collectors; |
|
|
|
|
|
|
|
|
|
|
|
import org.slf4j.Logger; |
|
|
|
import org.slf4j.Logger; |
|
|
|
import org.slf4j.LoggerFactory; |
|
|
|
import org.slf4j.LoggerFactory; |
|
|
@ -56,6 +57,7 @@ import org.springframework.stereotype.Component; |
|
|
|
|
|
|
|
|
|
|
|
import io.micrometer.core.annotation.Counted; |
|
|
|
import io.micrometer.core.annotation.Counted; |
|
|
|
import io.micrometer.core.annotation.Timed; |
|
|
|
import io.micrometer.core.annotation.Timed; |
|
|
|
|
|
|
|
import lombok.NonNull; |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
|
* failover service |
|
|
|
* failover service |
|
|
@ -67,15 +69,20 @@ public class FailoverService { |
|
|
|
private final MasterConfig masterConfig; |
|
|
|
private final MasterConfig masterConfig; |
|
|
|
private final ProcessService processService; |
|
|
|
private final ProcessService processService; |
|
|
|
private final WorkflowExecuteThreadPool workflowExecuteThreadPool; |
|
|
|
private final WorkflowExecuteThreadPool workflowExecuteThreadPool; |
|
|
|
|
|
|
|
private final ProcessInstanceExecCacheManager cacheManager; |
|
|
|
public FailoverService(RegistryClient registryClient, |
|
|
|
private final String localAddress; |
|
|
|
MasterConfig masterConfig, |
|
|
|
|
|
|
|
ProcessService processService, |
|
|
|
public FailoverService(@NonNull RegistryClient registryClient, |
|
|
|
WorkflowExecuteThreadPool workflowExecuteThreadPool) { |
|
|
|
@NonNull MasterConfig masterConfig, |
|
|
|
this.registryClient = checkNotNull(registryClient); |
|
|
|
@NonNull ProcessService processService, |
|
|
|
this.masterConfig = checkNotNull(masterConfig); |
|
|
|
@NonNull WorkflowExecuteThreadPool workflowExecuteThreadPool, |
|
|
|
this.processService = checkNotNull(processService); |
|
|
|
@NonNull ProcessInstanceExecCacheManager cacheManager) { |
|
|
|
this.workflowExecuteThreadPool = checkNotNull(workflowExecuteThreadPool); |
|
|
|
this.registryClient = registryClient; |
|
|
|
|
|
|
|
this.masterConfig = masterConfig; |
|
|
|
|
|
|
|
this.processService = processService; |
|
|
|
|
|
|
|
this.workflowExecuteThreadPool = workflowExecuteThreadPool; |
|
|
|
|
|
|
|
this.cacheManager = cacheManager; |
|
|
|
|
|
|
|
this.localAddress = NetUtils.getAddr(masterConfig.getListenPort()); |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
@ -88,7 +95,7 @@ public class FailoverService { |
|
|
|
if (CollectionUtils.isEmpty(hosts)) { |
|
|
|
if (CollectionUtils.isEmpty(hosts)) { |
|
|
|
return; |
|
|
|
return; |
|
|
|
} |
|
|
|
} |
|
|
|
LOGGER.info("Master failover service {} begin to failover hosts:{}", getLocalAddress(), hosts); |
|
|
|
LOGGER.info("Master failover service {} begin to failover hosts:{}", localAddress, hosts); |
|
|
|
|
|
|
|
|
|
|
|
for (String host : hosts) { |
|
|
|
for (String host : hosts) { |
|
|
|
failoverMasterWithLock(host); |
|
|
|
failoverMasterWithLock(host); |
|
|
@ -174,11 +181,10 @@ public class FailoverService { |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
|
* failover worker tasks |
|
|
|
* Do the worker failover. Will find the SUBMITTED_SUCCESS/DISPATCH/RUNNING_EXECUTION/DELAY_EXECUTION/READY_PAUSE/READY_STOP tasks belong the given worker, |
|
|
|
|
|
|
|
* and failover these tasks. |
|
|
|
* <p> |
|
|
|
* <p> |
|
|
|
* 1. kill yarn job if there are yarn jobs in tasks. |
|
|
|
* Note: When we do worker failover, the master will only failover the processInstance belongs to the current master. |
|
|
|
* 2. change task state from running to need failover. |
|
|
|
|
|
|
|
* 3. failover all tasks when workerHost is null |
|
|
|
|
|
|
|
* |
|
|
|
* |
|
|
|
* @param workerHost worker host |
|
|
|
* @param workerHost worker host |
|
|
|
*/ |
|
|
|
*/ |
|
|
@ -188,29 +194,40 @@ public class FailoverService { |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
long startTime = System.currentTimeMillis(); |
|
|
|
long startTime = System.currentTimeMillis(); |
|
|
|
List<TaskInstance> needFailoverTaskInstanceList = processService.queryNeedFailoverTaskInstances(workerHost); |
|
|
|
// we query the task instance from cache, so that we can directly update the cache
|
|
|
|
Map<Integer, ProcessInstance> processInstanceCacheMap = new HashMap<>(); |
|
|
|
final List<TaskInstance> needFailoverTaskInstanceList = cacheManager.getAll() |
|
|
|
|
|
|
|
.stream() |
|
|
|
|
|
|
|
.flatMap(workflowExecuteRunnable -> workflowExecuteRunnable.getAllTaskInstances().stream()) |
|
|
|
|
|
|
|
.filter(taskInstance -> |
|
|
|
|
|
|
|
workerHost.equals(taskInstance.getHost()) && ExecutionStatus.isNeedFailoverWorkflowInstanceState(taskInstance.getState())) |
|
|
|
|
|
|
|
.collect(Collectors.toList()); |
|
|
|
|
|
|
|
final Map<Integer, ProcessInstance> processInstanceCacheMap = new HashMap<>(); |
|
|
|
LOGGER.info("start worker[{}] failover, task list size:{}", workerHost, needFailoverTaskInstanceList.size()); |
|
|
|
LOGGER.info("start worker[{}] failover, task list size:{}", workerHost, needFailoverTaskInstanceList.size()); |
|
|
|
List<Server> workerServers = registryClient.getServerList(NodeType.WORKER); |
|
|
|
final List<Server> workerServers = registryClient.getServerList(NodeType.WORKER); |
|
|
|
for (TaskInstance taskInstance : needFailoverTaskInstanceList) { |
|
|
|
for (TaskInstance taskInstance : needFailoverTaskInstanceList) { |
|
|
|
ProcessInstance processInstance = processInstanceCacheMap.get(taskInstance.getProcessInstanceId()); |
|
|
|
LoggerUtils.setWorkflowAndTaskInstanceIDMDC(taskInstance.getProcessInstanceId(), taskInstance.getId()); |
|
|
|
if (processInstance == null) { |
|
|
|
try { |
|
|
|
processInstance = processService.findProcessInstanceDetailById(taskInstance.getProcessInstanceId()); |
|
|
|
ProcessInstance processInstance = processInstanceCacheMap.get(taskInstance.getProcessInstanceId()); |
|
|
|
if (processInstance == null) { |
|
|
|
if (processInstance == null) { |
|
|
|
LOGGER.error("failover task instance error, processInstance {} of taskInstance {} is null", |
|
|
|
processInstance = cacheManager.getByProcessInstanceId(taskInstance.getProcessInstanceId()).getProcessInstance(); |
|
|
|
taskInstance.getProcessInstanceId(), taskInstance.getId()); |
|
|
|
if (processInstance == null) { |
|
|
|
|
|
|
|
LOGGER.error("failover task instance error, processInstance {} of taskInstance {} is null", |
|
|
|
|
|
|
|
taskInstance.getProcessInstanceId(), taskInstance.getId()); |
|
|
|
|
|
|
|
continue; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
processInstanceCacheMap.put(processInstance.getId(), processInstance); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
// only failover the task owned myself if worker down.
|
|
|
|
|
|
|
|
if (!StringUtils.equalsIgnoreCase(processInstance.getHost(), localAddress)) { |
|
|
|
continue; |
|
|
|
continue; |
|
|
|
} |
|
|
|
} |
|
|
|
processInstanceCacheMap.put(processInstance.getId(), processInstance); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
// only failover the task owned myself if worker down.
|
|
|
|
LOGGER.info("failover task instance id: {}, process instance id: {}", taskInstance.getId(), taskInstance.getProcessInstanceId()); |
|
|
|
if (!processInstance.getHost().equalsIgnoreCase(getLocalAddress())) { |
|
|
|
failoverTaskInstance(processInstance, taskInstance, workerServers); |
|
|
|
continue; |
|
|
|
} finally { |
|
|
|
|
|
|
|
LoggerUtils.removeWorkflowAndTaskInstanceIdMDC(); |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
LOGGER.info("failover task instance id: {}, process instance id: {}", taskInstance.getId(), taskInstance.getProcessInstanceId()); |
|
|
|
|
|
|
|
failoverTaskInstance(processInstance, taskInstance, workerServers); |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
LOGGER.info("end worker[{}] failover, useTime:{}ms", workerHost, System.currentTimeMillis() - startTime); |
|
|
|
LOGGER.info("end worker[{}] failover, useTime:{}ms", workerHost, System.currentTimeMillis() - startTime); |
|
|
|
} |
|
|
|
} |
|
|
@ -221,17 +238,14 @@ public class FailoverService { |
|
|
|
* 1. kill yarn job if run on worker and there are yarn jobs in tasks. |
|
|
|
* 1. kill yarn job if run on worker and there are yarn jobs in tasks. |
|
|
|
* 2. change task state from running to need failover. |
|
|
|
* 2. change task state from running to need failover. |
|
|
|
* 3. try to notify local master |
|
|
|
* 3. try to notify local master |
|
|
|
|
|
|
|
* |
|
|
|
* @param processInstance |
|
|
|
* @param processInstance |
|
|
|
* @param taskInstance |
|
|
|
* @param taskInstance |
|
|
|
* @param servers if failover master, servers container master servers and worker servers; if failover worker, servers contain worker servers. |
|
|
|
* @param servers if failover master, servers container master servers and worker servers; if failover worker, servers contain worker servers. |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private void failoverTaskInstance(ProcessInstance processInstance, TaskInstance taskInstance, List<Server> servers) { |
|
|
|
private void failoverTaskInstance(@NonNull ProcessInstance processInstance, TaskInstance taskInstance, List<Server> servers) { |
|
|
|
if (processInstance == null) { |
|
|
|
|
|
|
|
LOGGER.error("failover task instance error, processInstance {} of taskInstance {} is null", |
|
|
|
|
|
|
|
taskInstance.getProcessInstanceId(), taskInstance.getId()); |
|
|
|
|
|
|
|
return; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
if (!checkTaskInstanceNeedFailover(servers, taskInstance)) { |
|
|
|
if (!checkTaskInstanceNeedFailover(servers, taskInstance)) { |
|
|
|
|
|
|
|
LOGGER.info("The taskInstance doesn't need to failover"); |
|
|
|
return; |
|
|
|
return; |
|
|
|
} |
|
|
|
} |
|
|
|
TaskMetrics.incTaskFailover(); |
|
|
|
TaskMetrics.incTaskFailover(); |
|
|
@ -240,6 +254,7 @@ public class FailoverService { |
|
|
|
taskInstance.setProcessInstance(processInstance); |
|
|
|
taskInstance.setProcessInstance(processInstance); |
|
|
|
|
|
|
|
|
|
|
|
if (!isMasterTask) { |
|
|
|
if (!isMasterTask) { |
|
|
|
|
|
|
|
LOGGER.info("The failover taskInstance is not master task"); |
|
|
|
TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get() |
|
|
|
TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get() |
|
|
|
.buildTaskInstanceRelatedInfo(taskInstance) |
|
|
|
.buildTaskInstanceRelatedInfo(taskInstance) |
|
|
|
.buildProcessInstanceRelatedInfo(processInstance) |
|
|
|
.buildProcessInstanceRelatedInfo(processInstance) |
|
|
@ -249,6 +264,8 @@ public class FailoverService { |
|
|
|
// only kill yarn job if exists , the local thread has exited
|
|
|
|
// only kill yarn job if exists , the local thread has exited
|
|
|
|
ProcessUtils.killYarnJob(taskExecutionContext); |
|
|
|
ProcessUtils.killYarnJob(taskExecutionContext); |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
} else { |
|
|
|
|
|
|
|
LOGGER.info("The failover taskInstance is a master task"); |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
taskInstance.setState(ExecutionStatus.NEED_FAULT_TOLERANCE); |
|
|
|
taskInstance.setState(ExecutionStatus.NEED_FAULT_TOLERANCE); |
|
|
@ -278,7 +295,7 @@ public class FailoverService { |
|
|
|
while (iterator.hasNext()) { |
|
|
|
while (iterator.hasNext()) { |
|
|
|
String host = iterator.next(); |
|
|
|
String host = iterator.next(); |
|
|
|
if (registryClient.checkNodeExists(host, NodeType.MASTER)) { |
|
|
|
if (registryClient.checkNodeExists(host, NodeType.MASTER)) { |
|
|
|
if (!getLocalAddress().equals(host)) { |
|
|
|
if (!localAddress.equals(host)) { |
|
|
|
iterator.remove(); |
|
|
|
iterator.remove(); |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
@ -390,11 +407,8 @@ public class FailoverService { |
|
|
|
return serverStartupTime; |
|
|
|
return serverStartupTime; |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
public String getLocalAddress() { |
|
|
|
* get local address |
|
|
|
return localAddress; |
|
|
|
*/ |
|
|
|
|
|
|
|
String getLocalAddress() { |
|
|
|
|
|
|
|
return NetUtils.getAddr(masterConfig.getListenPort()); |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|