分布式调度框架。
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

378 lines
13 KiB

/*
* 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.zk;
import static org.apache.dolphinscheduler.common.Constants.SLEEP_TIME_MILLIS;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.ZKNodeType;
import org.apache.dolphinscheduler.common.model.Server;
#2499 bug fix (#2505) * dispatch task fail will set task status failed * 1,no worker condition , master will while ture wait for worker startup 2,worker response task status sync wait for result * 1,no worker condition , master will while ture wait for worker startup 2,worker response task status sync wait for result * 1,no worker condition , master will while ture wait for worker startup 2,worker response task status sync wait for result * 1,no worker condition , master will while ture wait for worker startup 2,worker response task status sync wait for result * 1,no worker condition , master will while ture wait for worker startup 2,worker response task status sync wait for result * 1,no worker condition , master will while ture wait for worker startup 2,worker response task status sync wait for result * 1,no worker condition , master will while ture wait for worker startup 2,worker response task status sync wait for result * 1,no worker condition , master will while ture wait for worker startup 2,worker response task status sync wait for result * 1,no worker condition , master will while ture wait for worker startup 2,worker response task status sync wait for result * 1,no worker condition , master will while ture wait for worker startup 2,worker response task status sync wait for result * 1,no worker condition , master will while ture wait for worker startup 2,worker response task status sync wait for result * 1,task status statistics and process status statistics bug fix (#2357) 2,worker group bug fix * 1,task status statistics and process status statistics bug fix (#2357) 2,worker group bug fix * 1,task status statistics and process status statistics bug fix (#2357) 2,worker group bug fix * 1,task status statistics and process status statistics bug fix (#2357) 2,worker group bug fix * send mail error, #2466 bug fix * send mail error, #2466 bug fix * send mail error, #2466 bug fix * send mail error, #2466 bug fix * #2486 bug fix * host and workergroup compatible * EnterpriseWeChatUtils modify * EnterpriseWeChatUtils modify * EnterpriseWeChatUtils modify * #2499 bug fix Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn>
4 years ago
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.common.utils.NetUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.builder.TaskExecutionContextBuilder;
4 years ago
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.registry.MasterRegistry;
import org.apache.dolphinscheduler.server.utils.ProcessUtils;
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.zk.AbstractZKClient;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
import org.apache.curator.framework.recipes.locks.InterProcessMutex;
import java.util.Date;
import java.util.List;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
/**
* zookeeper master client
* <p>
* single instance
*/
@Component
public class ZKMasterClient extends AbstractZKClient {
/**
* logger
*/
private static final Logger logger = LoggerFactory.getLogger(ZKMasterClient.class);
/**
* process service
*/
@Autowired
private ProcessService processService;
/**
* master registry
*/
@Autowired
private MasterRegistry masterRegistry;
public void start() {
InterProcessMutex mutex = null;
try {
// create distributed lock with the root node path of the lock space as /dolphinscheduler/lock/failover/master
String znodeLock = getMasterStartUpLockPath();
mutex = new InterProcessMutex(getZkClient(), znodeLock);
mutex.acquire();
// Master registry
masterRegistry.registry();
// init system znode
this.initSystemZNode();
while (!checkZKNodeExists(NetUtils.getHost(), ZKNodeType.MASTER)) {
ThreadUtils.sleep(SLEEP_TIME_MILLIS);
}
// self tolerant
if (getActiveMasterNum() == 1) {
removeZKNodePath(null, ZKNodeType.MASTER, true);
removeZKNodePath(null, ZKNodeType.WORKER, true);
}
registerListener();
} catch (Exception e) {
logger.error("master start up exception", e);
} finally {
releaseMutex(mutex);
}
}
@Override
public void close() {
super.close();
masterRegistry.unRegistry();
}
/**
* handle path events that this class cares about
*
* @param client zkClient
* @param event path event
* @param path zk path
*/
@Override
protected void dataChanged(CuratorFramework client, TreeCacheEvent event, String path) {
//monitor master
if (path.startsWith(getZNodeParentPath(ZKNodeType.MASTER) + Constants.SINGLE_SLASH)) {
handleMasterEvent(event, path);
} else if (path.startsWith(getZNodeParentPath(ZKNodeType.WORKER) + Constants.SINGLE_SLASH)) {
//monitor worker
handleWorkerEvent(event, path);
}
}
/**
* remove zookeeper node path
*
* @param path zookeeper node path
* @param zkNodeType zookeeper node type
* @param failover is failover
*/
private void removeZKNodePath(String path, ZKNodeType zkNodeType, boolean failover) {
logger.info("{} node deleted : {}", zkNodeType.toString(), path);
InterProcessMutex mutex = null;
try {
String failoverPath = getFailoverLockPath(zkNodeType);
// create a distributed lock
mutex = new InterProcessMutex(getZkClient(), failoverPath);
mutex.acquire();
String serverHost = null;
if (StringUtils.isNotEmpty(path)) {
serverHost = getHostByEventDataPath(path);
if (StringUtils.isEmpty(serverHost)) {
logger.error("server down error: unknown path: {}", path);
return;
}
// handle dead server
handleDeadServer(path, zkNodeType, Constants.ADD_ZK_OP);
}
//failover server
if (failover) {
failoverServerWhenDown(serverHost, zkNodeType);
}
} catch (Exception e) {
logger.error("{} server failover failed.", zkNodeType.toString());
logger.error("failover exception ", e);
} finally {
releaseMutex(mutex);
}
}
/**
* failover server when server down
*
* @param serverHost server host
* @param zkNodeType zookeeper node type
* @throws Exception exception
*/
private void failoverServerWhenDown(String serverHost, ZKNodeType zkNodeType) throws Exception {
if (StringUtils.isEmpty(serverHost)) {
return;
}
switch (zkNodeType) {
case MASTER:
failoverMaster(serverHost);
break;
case WORKER:
failoverWorker(serverHost, true);
break;
default:
break;
}
}
/**
* get failover lock path
*
* @param zkNodeType zookeeper node type
* @return fail over lock path
*/
private String getFailoverLockPath(ZKNodeType zkNodeType) {
switch (zkNodeType) {
case MASTER:
return getMasterFailoverLockPath();
case WORKER:
return getWorkerFailoverLockPath();
default:
return "";
}
}
/**
* monitor master
*
* @param event event
* @param path path
*/
public void handleMasterEvent(TreeCacheEvent event, String path) {
switch (event.getType()) {
case NODE_ADDED:
logger.info("master node added : {}", path);
break;
case NODE_REMOVED:
removeZKNodePath(path, ZKNodeType.MASTER, true);
break;
default:
break;
}
}
/**
* monitor worker
*
* @param event event
* @param path path
*/
public void handleWorkerEvent(TreeCacheEvent event, String path) {
switch (event.getType()) {
case NODE_ADDED:
logger.info("worker node added : {}", path);
break;
case NODE_REMOVED:
logger.info("worker node deleted : {}", path);
removeZKNodePath(path, ZKNodeType.WORKER, true);
break;
default:
break;
}
}
/**
* task needs failover if task start before worker starts
*
* @param taskInstance task instance
* @return true if task instance need fail over
*/
private boolean checkTaskInstanceNeedFailover(TaskInstance taskInstance) throws Exception {
boolean taskNeedFailover = true;
//now no host will execute this task instance,so no need to failover the task
if (taskInstance.getHost() == null) {
return false;
}
// if the worker node exists in zookeeper, we must check the task starts after the worker
if (checkZKNodeExists(taskInstance.getHost(), ZKNodeType.WORKER)) {
//if task start after worker starts, there is no need to failover the task.
if (checkTaskAfterWorkerStart(taskInstance)) {
taskNeedFailover = false;
}
}
return taskNeedFailover;
}
/**
* check task start after the worker server starts.
*
* @param taskInstance task instance
* @return true if task instance start time after worker server start date
*/
private boolean checkTaskAfterWorkerStart(TaskInstance taskInstance) {
if (StringUtils.isEmpty(taskInstance.getHost())) {
return false;
}
Date workerServerStartDate = null;
List<Server> workerServers = getServersList(ZKNodeType.WORKER);
for (Server workerServer : workerServers) {
if (taskInstance.getHost().equals(workerServer.getHost() + Constants.COLON + workerServer.getPort())) {
workerServerStartDate = workerServer.getCreateTime();
break;
}
}
if (workerServerStartDate != null) {
return taskInstance.getStartTime().after(workerServerStartDate);
}
return false;
}
/**
* failover worker tasks
*
* 1. kill yarn job if there are yarn jobs in tasks.
* 2. change task state from running to need failover.
* 3. failover all tasks when workerHost is null
* @param workerHost worker host
*/
/**
* failover worker tasks
* <p>
* 1. kill yarn job if there are yarn jobs in tasks.
* 2. change task state from running to need failover.
* 3. failover all tasks when workerHost is null
*
* @param workerHost worker host
* @param needCheckWorkerAlive need check worker alive
* @throws Exception exception
*/
private void failoverWorker(String workerHost, boolean needCheckWorkerAlive) throws Exception {
workerHost = Host.of(workerHost).getAddress();
logger.info("start worker[{}] failover ...", workerHost);
List<TaskInstance> needFailoverTaskInstanceList = processService.queryNeedFailoverTaskInstances(workerHost);
for (TaskInstance taskInstance : needFailoverTaskInstanceList) {
if (needCheckWorkerAlive) {
if (!checkTaskInstanceNeedFailover(taskInstance)) {
continue;
}
}
ProcessInstance processInstance = processService.findProcessInstanceDetailById(taskInstance.getProcessInstanceId());
if (processInstance != null) {
taskInstance.setProcessInstance(processInstance);
}
TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get()
.buildTaskInstanceRelatedInfo(taskInstance)
.buildProcessInstanceRelatedInfo(processInstance)
.create();
// only kill yarn job if exists , the local thread has exited
ProcessUtils.killYarnJob(taskExecutionContext);
taskInstance.setState(ExecutionStatus.NEED_FAULT_TOLERANCE);
processService.saveTaskInstance(taskInstance);
}
logger.info("end worker[{}] failover ...", workerHost);
}
/**
* failover master tasks
*
* @param masterHost master host
*/
private void failoverMaster(String masterHost) {
logger.info("start master failover ...");
List<ProcessInstance> needFailoverProcessInstanceList = processService.queryNeedFailoverProcessInstances(masterHost);
logger.info("failover process list size:{} ", needFailoverProcessInstanceList.size());
//updateProcessInstance host is null and insert into command
for (ProcessInstance processInstance : needFailoverProcessInstanceList) {
logger.info("failover process instance id: {} host:{}",
processInstance.getId(), processInstance.getHost());
if (Constants.NULL.equals(processInstance.getHost())) {
continue;
}
processService.processNeedFailoverProcessInstances(processInstance);
}
logger.info("master failover end");
}
public InterProcessMutex blockAcquireMutex() throws Exception {
InterProcessMutex mutex = new InterProcessMutex(getZkClient(), getMasterLockPath());
mutex.acquire();
return mutex;
}
}