Browse Source

master add kill task logic (#2058)

* 1, master persistent task
2. extract  master and worker communication model

* 1, master persistent task
2. extract  master and worker communication model

* 1, master persistent task
2. extract  master and worker communication model

* add license

* modify javadoc error

* TaskExecutionContext create modify

* buildAckCommand taskInstanceId not set modify

* java doc error modify

* add comment

* ExecutorManager interface add generic type

* add TaskInstanceCacheManager receive Worker report result

* TaskInstance setExecutePath

* add TaskInstanceCacheManager to receive Worker Task result report

* TaskInstanceCacheManager add remove method

* add license

* add dispatcht task method

* AbstractCommandExecutor remove db access

* AbstractCommandExecutor remove db access

* AbstractCommandExecutor remove db access

* AbstractCommandExecutor remove db access

* AbstractCommandExecutor remove db access

* AbstractCommandExecutor remove db access

* AbstractCommandExecutor remove db access

* taskInstanceCache is null ,need load from db

* taskInstanceCache is null ,need load from db

* taskInstanceCache is null ,need load from db

* 1,worker TaskPros use TaskExecutionContext replase
2,Master kill Task , KillTaskProcessor modify

* worker remove db

* ShellTask modify

* master persistence processId and appIds

* master persistence processId and appIds

* master add kill task logic

* master add kill task logic

* master add kill task logic

* javadoc error modify
pull/2/head
qiaozhanwei 4 years ago committed by GitHub
parent
commit
bb3885cfe2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 10
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/DbType.java
  2. 2
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/KillTaskRequestCommand.java
  3. 7
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/builder/TaskExecutionContextBuilder.java
  4. 116
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/entity/DataxTaskExecutionContext.java
  5. 64
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/entity/SQLTaskExecutionContext.java
  6. 37
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/entity/TaskExecutionContext.java
  7. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java
  8. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/cache/TaskInstanceCacheManager.java
  9. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/TaskInstanceCacheManagerImpl.java
  10. 4
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutorDispatcher.java
  11. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/context/ExecutionContext.java
  12. 1
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/executor/AbstractExecutorManager.java
  13. 4
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/executor/ExecutorManager.java
  14. 6
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManager.java
  15. 119
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyKillManager.java
  16. 6
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobin.java
  17. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/future/TaskFuture.java
  18. 61
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskKillResponseProcessor.java
  19. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskResponseProcessor.java
  20. 3
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistry.java
  21. 4
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java
  22. 5
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java
  23. 42
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java
  24. 5
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/Monitor.java
  25. 6
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/registry/ZookeeperNodeManager.java
  26. 6
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/registry/ZookeeperRegistryCenter.java
  27. 3
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ParamUtils.java
  28. 3
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java
  29. 3
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/cache/TaskExecutionContextCacheManager.java
  30. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/cache/impl/TaskExecutionContextCacheManagerImpl.java
  31. 3
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskExecuteProcessor.java
  32. 49
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskKillProcessor.java
  33. 4
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistry.java
  34. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskExecuteThread.java
  35. 23
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java
  36. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractTask.java
  37. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java
  38. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/PythonCommandExecutor.java
  39. 3
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/ShellCommandExecutor.java
  40. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskManager.java
  41. 28
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskProps.java
  42. 56
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java
  43. 4
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/flink/FlinkTask.java
  44. 12
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java
  45. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/mr/MapReduceTask.java
  46. 3
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java
  47. 5
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java
  48. 17
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java
  49. 3
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/spark/SparkTask.java
  50. 65
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java
  51. 6
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java
  52. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKWorkerClient.java
  53. 4
      dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java

10
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/DbType.java

@ -57,4 +57,14 @@ public enum DbType {
public String getDescp() { public String getDescp() {
return descp; return descp;
} }
public static DbType of(int type){
for(DbType ty : values()){
if(ty.getCode() == type){
return ty;
}
}
throw new IllegalArgumentException("invalid type : " + type);
}
} }

2
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/KillTaskRequestCommand.java

@ -1 +1 @@
/* * 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.remote.command; import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; import java.io.Serializable; /** * kill task request command */ public class KillTaskRequestCommand implements Serializable { /** * taskInstanceId */ private int taskInstanceId; /** * processId */ private int processId; /** * host */ private String host; /** InstanceId */ private String tenantCode; /** * logPath */ private String logPath; /** * executePath */ InstanceId /** InstanceId * processId InstanceId private int processId; } InstanceId private String host; this.logPath = logPath; } public int getTaskInstanceId() { */ */ } */ private int taskInstanceId; this.taskInstanceId = taskInstanceId; } */ /** */ * processId } */ private int processId; */ * host } */ private String host; private int taskInstanceId; } public void setHost(String host) { this.host = host; } public String getTenantCode() { return tenantCode; InstanceId * host private int taskInstanceId; /** this.tenantCode = tenantCode; } private int taskInstanceId; private int processId; return executePath; } public void setExecutePath(String executePath) { this.executePath = executePath; } /** InstanceId */ private int taskInstanceId; */ /** * processId private int processId; command.setBody(body); return command; } } /* * 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.remote.command; import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; import java.io.Serializable; /** * kill task request command */ public class KillTaskRequestCommand implements Serializable { /** * task /** InstanceId /** */ /** private int taskInstanceId; /** /** /** /** * processId /** private int processId; /** /** * host /** private String host; } /** * processId /** private int processId; /** * processId InstanceId /** private String host; } /** * processId */ * processId private int taskInstanceId; * processId * processId /** /** */ public Command convert2Command(){ * processId private int processId; * processId * host * processId private String host; private int processId; private int processId; InstanceId /** private int processId; /** private int processId; */ private int processId; private int taskInstanceId; private int processId; private int processId; /** private int processId; * processId private int processId; private int processId; > }

7
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/builder/TaskExecutionContextBuilder.java

@ -17,10 +17,11 @@
package org.apache.dolphinscheduler.server.builder; package org.apache.dolphinscheduler.server.builder;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
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.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
/** /**
* TaskExecutionContext builder * TaskExecutionContext builder
@ -47,6 +48,8 @@ public class TaskExecutionContextBuilder {
taskExecutionContext.setLogPath(taskInstance.getLogPath()); taskExecutionContext.setLogPath(taskInstance.getLogPath());
taskExecutionContext.setExecutePath(taskInstance.getExecutePath()); taskExecutionContext.setExecutePath(taskInstance.getExecutePath());
taskExecutionContext.setTaskJson(taskInstance.getTaskJson()); taskExecutionContext.setTaskJson(taskInstance.getTaskJson());
taskExecutionContext.setHost(taskInstance.getHost());
taskExecutionContext.setWorkerGroup(Constants.DEFAULT_WORKER_GROUP);
return this; return this;
} }
@ -54,7 +57,7 @@ public class TaskExecutionContextBuilder {
/** /**
* build processInstance related info * build processInstance related info
* *
* @param processInstance * @param processInstance processInstance
* @return TaskExecutionContextBuilder * @return TaskExecutionContextBuilder
*/ */
public TaskExecutionContextBuilder buildProcessInstanceRelatedInfo(ProcessInstance processInstance){ public TaskExecutionContextBuilder buildProcessInstanceRelatedInfo(ProcessInstance processInstance){

116
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/entity/DataxTaskExecutionContext.java

@ -0,0 +1,116 @@
/*
* 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.entity;
import java.io.Serializable;
/**
* master/worker task transport
*/
public class DataxTaskExecutionContext implements Serializable{
/**
* dataSourceId
*/
private int dataSourceId;
/**
* sourcetype
*/
private int sourcetype;
/**
* sourceConnectionParams
*/
private String sourceConnectionParams;
/**
* dataTargetId
*/
private int dataTargetId;
/**
* targetType
*/
private int targetType;
/**
* targetConnectionParams
*/
private String targetConnectionParams;
public int getDataSourceId() {
return dataSourceId;
}
public void setDataSourceId(int dataSourceId) {
this.dataSourceId = dataSourceId;
}
public int getSourcetype() {
return sourcetype;
}
public void setSourcetype(int sourcetype) {
this.sourcetype = sourcetype;
}
public String getSourceConnectionParams() {
return sourceConnectionParams;
}
public void setSourceConnectionParams(String sourceConnectionParams) {
this.sourceConnectionParams = sourceConnectionParams;
}
public int getDataTargetId() {
return dataTargetId;
}
public void setDataTargetId(int dataTargetId) {
this.dataTargetId = dataTargetId;
}
public int getTargetType() {
return targetType;
}
public void setTargetType(int targetType) {
this.targetType = targetType;
}
public String getTargetConnectionParams() {
return targetConnectionParams;
}
public void setTargetConnectionParams(String targetConnectionParams) {
this.targetConnectionParams = targetConnectionParams;
}
@Override
public String toString() {
return "DataxTaskExecutionContext{" +
"dataSourceId=" + dataSourceId +
", sourcetype=" + sourcetype +
", sourceConnectionParams='" + sourceConnectionParams + '\'' +
", dataTargetId=" + dataTargetId +
", targetType=" + targetType +
", targetConnectionParams='" + targetConnectionParams + '\'' +
'}';
}
}

64
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/entity/SQLTaskExecutionContext.java

@ -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.entity;
import org.apache.dolphinscheduler.dao.entity.UdfFunc;
import java.io.Serializable;
import java.util.List;
/**
* SQL Task ExecutionContext
*/
public class SQLTaskExecutionContext implements Serializable {
/**
* warningGroupId
*/
private int warningGroupId;
/**
* udf function list
*/
private List<UdfFunc> udfFuncList;
public int getWarningGroupId() {
return warningGroupId;
}
public void setWarningGroupId(int warningGroupId) {
this.warningGroupId = warningGroupId;
}
public List<UdfFunc> getUdfFuncList() {
return udfFuncList;
}
public void setUdfFuncList(List<UdfFunc> udfFuncList) {
this.udfFuncList = udfFuncList;
}
@Override
public String toString() {
return "SQLTaskExecutionContext{" +
"warningGroupId=" + warningGroupId +
", udfFuncList=" + udfFuncList +
'}';
}
}

37
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/entity/TaskExecutionContext.java → dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/entity/TaskExecutionContext.java

@ -15,12 +15,10 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.dolphinscheduler.remote.entity; package org.apache.dolphinscheduler.server.entity;
import java.io.Serializable; import java.io.Serializable;
import java.util.Date; import java.util.Date;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map; import java.util.Map;
/** /**
@ -144,7 +142,6 @@ public class TaskExecutionContext implements Serializable{
*/ */
private Map<String, String> definedParams; private Map<String, String> definedParams;
/** /**
* task AppId * task AppId
*/ */
@ -165,6 +162,20 @@ public class TaskExecutionContext implements Serializable{
*/ */
private String workerGroup; private String workerGroup;
/**
* sql TaskExecutionContext
*/
private SQLTaskExecutionContext sqlTaskExecutionContext;
/**
* datax TaskExecutionContext
*/
private DataxTaskExecutionContext dataxTaskExecutionContext;
public String getWorkerGroup() { public String getWorkerGroup() {
return workerGroup; return workerGroup;
} }
@ -373,6 +384,21 @@ public class TaskExecutionContext implements Serializable{
this.appIds = appIds; this.appIds = appIds;
} }
public SQLTaskExecutionContext getSqlTaskExecutionContext() {
return sqlTaskExecutionContext;
}
public void setSqlTaskExecutionContext(SQLTaskExecutionContext sqlTaskExecutionContext) {
this.sqlTaskExecutionContext = sqlTaskExecutionContext;
}
public DataxTaskExecutionContext getDataxTaskExecutionContext() {
return dataxTaskExecutionContext;
}
public void setDataxTaskExecutionContext(DataxTaskExecutionContext dataxTaskExecutionContext) {
this.dataxTaskExecutionContext = dataxTaskExecutionContext;
}
@Override @Override
public String toString() { public String toString() {
@ -402,6 +428,9 @@ public class TaskExecutionContext implements Serializable{
", taskAppId='" + taskAppId + '\'' + ", taskAppId='" + taskAppId + '\'' +
", taskTimeoutStrategy=" + taskTimeoutStrategy + ", taskTimeoutStrategy=" + taskTimeoutStrategy +
", taskTimeout=" + taskTimeout + ", taskTimeout=" + taskTimeout +
", workerGroup='" + workerGroup + '\'' +
", sqlTaskExecutionContext=" + sqlTaskExecutionContext +
", dataxTaskExecutionContext=" + dataxTaskExecutionContext +
'}'; '}';
} }
} }

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java

@ -27,6 +27,7 @@ import org.apache.dolphinscheduler.remote.command.CommandType;
import org.apache.dolphinscheduler.remote.config.NettyServerConfig; import org.apache.dolphinscheduler.remote.config.NettyServerConfig;
import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.processor.TaskAckProcessor; import org.apache.dolphinscheduler.server.master.processor.TaskAckProcessor;
import org.apache.dolphinscheduler.server.master.processor.TaskKillResponseProcessor;
import org.apache.dolphinscheduler.server.master.processor.TaskResponseProcessor; import org.apache.dolphinscheduler.server.master.processor.TaskResponseProcessor;
import org.apache.dolphinscheduler.server.master.registry.MasterRegistry; import org.apache.dolphinscheduler.server.master.registry.MasterRegistry;
import org.apache.dolphinscheduler.server.master.runner.MasterSchedulerThread; import org.apache.dolphinscheduler.server.master.runner.MasterSchedulerThread;
@ -127,6 +128,7 @@ public class MasterServer implements IStoppable {
this.nettyRemotingServer = new NettyRemotingServer(serverConfig); this.nettyRemotingServer = new NettyRemotingServer(serverConfig);
this.nettyRemotingServer.registerProcessor(CommandType.EXECUTE_TASK_RESPONSE, new TaskResponseProcessor()); this.nettyRemotingServer.registerProcessor(CommandType.EXECUTE_TASK_RESPONSE, new TaskResponseProcessor());
this.nettyRemotingServer.registerProcessor(CommandType.EXECUTE_TASK_ACK, new TaskAckProcessor()); this.nettyRemotingServer.registerProcessor(CommandType.EXECUTE_TASK_ACK, new TaskAckProcessor());
this.nettyRemotingServer.registerProcessor(CommandType.KILL_TASK_RESPONSE, new TaskKillResponseProcessor());
this.nettyRemotingServer.start(); this.nettyRemotingServer.start();
// //

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/cache/TaskInstanceCacheManager.java vendored

@ -20,7 +20,7 @@ package org.apache.dolphinscheduler.server.master.cache;
import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.remote.command.ExecuteTaskAckCommand; import org.apache.dolphinscheduler.remote.command.ExecuteTaskAckCommand;
import org.apache.dolphinscheduler.remote.command.ExecuteTaskResponseCommand; import org.apache.dolphinscheduler.remote.command.ExecuteTaskResponseCommand;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
/** /**
* task instance state manager * task instance state manager

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/TaskInstanceCacheManagerImpl.java vendored

@ -20,7 +20,7 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.remote.command.ExecuteTaskAckCommand; import org.apache.dolphinscheduler.remote.command.ExecuteTaskAckCommand;
import org.apache.dolphinscheduler.remote.command.ExecuteTaskResponseCommand; import org.apache.dolphinscheduler.remote.command.ExecuteTaskResponseCommand;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.cache.TaskInstanceCacheManager; import org.apache.dolphinscheduler.server.master.cache.TaskInstanceCacheManager;
import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.process.ProcessService;
import org.springframework.beans.factory.annotation.Autowired; import org.springframework.beans.factory.annotation.Autowired;

4
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutorDispatcher.java

@ -67,8 +67,10 @@ public class ExecutorDispatcher implements InitializingBean {
} }
/** /**
* task dispatch * task dispatch
*
* @param context context * @param context context
* @return result
* @throws ExecuteException * @throws ExecuteException
*/ */
public Boolean dispatch(final ExecutionContext context) throws ExecuteException { public Boolean dispatch(final ExecutionContext context) throws ExecuteException {

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/context/ExecutionContext.java

@ -17,8 +17,8 @@
package org.apache.dolphinscheduler.server.master.dispatch.context; package org.apache.dolphinscheduler.server.master.dispatch.context;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.remote.utils.Host; import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType; import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType;
/** /**

1
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/executor/AbstractExecutorManager.java

@ -27,6 +27,7 @@ public abstract class AbstractExecutorManager<T> implements ExecutorManager<T>{
/** /**
* before execute , add time monitor timeout * before execute , add time monitor timeout
*
* @param context context * @param context context
* @throws ExecuteException * @throws ExecuteException
*/ */

4
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/executor/ExecutorManager.java

@ -27,6 +27,7 @@ public interface ExecutorManager<T> {
/** /**
* before execute * before execute
*
* @param executeContext executeContext * @param executeContext executeContext
* @throws ExecuteException * @throws ExecuteException
*/ */
@ -35,12 +36,13 @@ public interface ExecutorManager<T> {
/** /**
* execute task * execute task
* @param context context * @param context context
* @return T
* @throws ExecuteException * @throws ExecuteException
*/ */
T execute(ExecutionContext context) throws ExecuteException; T execute(ExecutionContext context) throws ExecuteException;
/** /**
* after execute * after execute
* @param context context * @param context context
* @throws ExecuteException * @throws ExecuteException
*/ */

6
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManager.java

@ -23,9 +23,9 @@ import org.apache.dolphinscheduler.remote.command.Command;
import org.apache.dolphinscheduler.remote.command.CommandType; import org.apache.dolphinscheduler.remote.command.CommandType;
import org.apache.dolphinscheduler.remote.command.ExecuteTaskRequestCommand; import org.apache.dolphinscheduler.remote.command.ExecuteTaskRequestCommand;
import org.apache.dolphinscheduler.remote.config.NettyClientConfig; import org.apache.dolphinscheduler.remote.config.NettyClientConfig;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
import org.apache.dolphinscheduler.remote.utils.Host; import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; 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.enums.ExecutorType;
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException; import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException;
@ -72,9 +72,11 @@ public class NettyExecutorManager extends AbstractExecutorManager<Boolean>{
this.nettyRemotingClient.registerProcessor(CommandType.EXECUTE_TASK_ACK, new TaskAckProcessor()); this.nettyRemotingClient.registerProcessor(CommandType.EXECUTE_TASK_ACK, new TaskAckProcessor());
} }
/** /**
* execute logic * execute logic
* @param context context * @param context context
* @return result
* @throws ExecuteException * @throws ExecuteException
*/ */
@Override @Override

119
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyKillManager.java

@ -0,0 +1,119 @@
/*
* 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.executor;
import org.apache.dolphinscheduler.remote.NettyRemotingClient;
import org.apache.dolphinscheduler.remote.command.Command;
import org.apache.dolphinscheduler.remote.command.CommandType;
import org.apache.dolphinscheduler.remote.command.KillTaskRequestCommand;
import org.apache.dolphinscheduler.remote.config.NettyClientConfig;
import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext;
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException;
import org.apache.dolphinscheduler.server.master.processor.TaskKillResponseProcessor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.stereotype.Service;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
/**
* netty executor manager
*/
@Service
public class NettyKillManager extends AbstractExecutorManager<Boolean>{
private final Logger logger = LoggerFactory.getLogger(NettyKillManager.class);
/**
* netty remote client
*/
private final NettyRemotingClient nettyRemotingClient;
public NettyKillManager(){
final NettyClientConfig clientConfig = new NettyClientConfig();
this.nettyRemotingClient = new NettyRemotingClient(clientConfig);
/**
* register KILL_TASK_RESPONSE command type TaskKillResponseProcessor
*/
this.nettyRemotingClient.registerProcessor(CommandType.KILL_TASK_RESPONSE, new TaskKillResponseProcessor());
}
/**
* execute logic
*
* @param context context
* @return result
* @throws ExecuteException
*/
@Override
public Boolean execute(ExecutionContext context) throws ExecuteException {
Host host = context.getHost();
Command command = buildCommand(context);
try {
doExecute(host, command);
return true;
}catch (ExecuteException ex) {
logger.error(String.format("execute context : %s error", context.getContext()), ex);
return false;
}
}
private Command buildCommand(ExecutionContext context) {
KillTaskRequestCommand requestCommand = new KillTaskRequestCommand();
TaskExecutionContext taskExecutionContext = context.getContext();
requestCommand.setTaskExecutionContext(FastJsonSerializer.serializeToString(taskExecutionContext));
return requestCommand.convert2Command();
}
/**
* execute logic
* @param host host
* @param command command
* @throws ExecuteException
*/
private void doExecute(final Host host, final Command command) throws ExecuteException {
/**
* retry countdefault retry 3
*/
int retryCount = 3;
boolean success = false;
do {
try {
nettyRemotingClient.send(host, command);
success = true;
} catch (Exception ex) {
logger.error(String.format("send command : %s to %s error", command, host), ex);
retryCount--;
try {
Thread.sleep(100);
} catch (InterruptedException ignore) {}
}
} while (retryCount >= 0 && !success);
if (!success) {
throw new ExecuteException(String.format("send command : %s to %s error", command, host));
}
}
}

6
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/assign/LowerWeightRoundRobin.java

@ -24,6 +24,12 @@ import java.util.Collection;
*/ */
public class LowerWeightRoundRobin implements Selector<HostWeight>{ public class LowerWeightRoundRobin implements Selector<HostWeight>{
/**
* select
* @param sources sources
* @return HostWeight
*/
@Override
public HostWeight select(Collection<HostWeight> sources){ public HostWeight select(Collection<HostWeight> sources){
int totalWeight = 0; int totalWeight = 0;
int lowWeight = 0; int lowWeight = 0;

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/future/TaskFuture.java

@ -68,7 +68,7 @@ public class TaskFuture {
} }
/** /**
* wait for response * wait for response
* @return command * @return command
* @throws InterruptedException * @throws InterruptedException
*/ */

61
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskKillResponseProcessor.java

@ -0,0 +1,61 @@
/*
* 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 io.netty.channel.Channel;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.utils.Preconditions;
import org.apache.dolphinscheduler.remote.command.Command;
import org.apache.dolphinscheduler.remote.command.CommandType;
import org.apache.dolphinscheduler.remote.command.ExecuteTaskResponseCommand;
import org.apache.dolphinscheduler.remote.command.KillTaskResponseCommand;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
import org.apache.dolphinscheduler.server.master.cache.TaskInstanceCacheManager;
import org.apache.dolphinscheduler.server.master.cache.impl.TaskInstanceCacheManagerImpl;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillTaskResponse;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* task response processor
*/
public class TaskKillResponseProcessor implements NettyRequestProcessor {
private final Logger logger = LoggerFactory.getLogger(TaskKillResponseProcessor.class);
/**
* task final result response
* need master process , state persistence
*
* @param channel channel
* @param command command
*/
@Override
public void process(Channel channel, Command command) {
Preconditions.checkArgument(CommandType.KILL_TASK_RESPONSE == command.getType(), String.format("invalid command type : %s", command.getType()));
KillTaskResponseCommand responseCommand = FastJsonSerializer.deserialize(command.getBody(), KillTaskResponseCommand.class);
logger.info("received command : {}", responseCommand);
logger.info("已经接受到了worker杀任务的回应");
}
}

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskResponseProcessor.java

@ -72,6 +72,8 @@ public class TaskResponseProcessor implements NettyRequestProcessor {
processService.changeTaskState(ExecutionStatus.of(responseCommand.getStatus()), processService.changeTaskState(ExecutionStatus.of(responseCommand.getStatus()),
responseCommand.getEndTime(), responseCommand.getEndTime(),
responseCommand.getProcessId(),
responseCommand.getAppIds(),
responseCommand.getTaskInstanceId()); responseCommand.getTaskInstanceId());
} }

3
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistry.java

@ -67,9 +67,10 @@ public class MasterRegistry {
private final String startTime; private final String startTime;
/** /**
* construct * construct
* @param zookeeperRegistryCenter zookeeperRegistryCenter * @param zookeeperRegistryCenter zookeeperRegistryCenter
* @param port port * @param port port
* @param heartBeatInterval heartBeatInterval
*/ */
public MasterRegistry(ZookeeperRegistryCenter zookeeperRegistryCenter, int port, long heartBeatInterval){ public MasterRegistry(ZookeeperRegistryCenter zookeeperRegistryCenter, int port, long heartBeatInterval){
this.zookeeperRegistryCenter = zookeeperRegistryCenter; this.zookeeperRegistryCenter = zookeeperRegistryCenter;

4
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java

@ -24,8 +24,8 @@ import org.apache.dolphinscheduler.dao.AlertDao;
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.dao.entity.Tenant; import org.apache.dolphinscheduler.dao.entity.Tenant;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.builder.TaskExecutionContextBuilder; import org.apache.dolphinscheduler.server.builder.TaskExecutionContextBuilder;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher; import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher;
import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext;
@ -147,7 +147,7 @@ public class MasterBaseTaskExecThread implements Callable<Boolean> {
* @param taskInstance taskInstance * @param taskInstance taskInstance
* @return TaskExecutionContext * @return TaskExecutionContext
*/ */
private TaskExecutionContext getTaskExecutionContext(TaskInstance taskInstance){ protected TaskExecutionContext getTaskExecutionContext(TaskInstance taskInstance){
taskInstance = processService.getTaskInstanceDetailByTaskId(taskInstance.getId()); taskInstance = processService.getTaskInstanceDetailByTaskId(taskInstance.getId());
Integer userId = taskInstance.getProcessDefine() == null ? 0 : taskInstance.getProcessDefine().getUserId(); Integer userId = taskInstance.getProcessDefine() == null ? 0 : taskInstance.getProcessDefine().getUserId();

5
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java

@ -142,8 +142,9 @@ public class MasterExecThread implements Runnable {
/** /**
* constructor of MasterExecThread * constructor of MasterExecThread
* @param processInstance process instance * @param processInstance processInstance
* @param processService process dao * @param processService processService
* @param nettyRemotingClient nettyRemotingClient
*/ */
public MasterExecThread(ProcessInstance processInstance, ProcessService processService, NettyRemotingClient nettyRemotingClient){ public MasterExecThread(ProcessInstance processInstance, ProcessService processService, NettyRemotingClient nettyRemotingClient){
this.processService = processService; this.processService = processService;

42
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java

@ -26,15 +26,19 @@ import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
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 com.alibaba.fastjson.JSONObject; import com.alibaba.fastjson.JSONObject;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.cache.TaskInstanceCacheManager; import org.apache.dolphinscheduler.server.master.cache.TaskInstanceCacheManager;
import org.apache.dolphinscheduler.server.master.cache.impl.TaskInstanceCacheManagerImpl; import org.apache.dolphinscheduler.server.master.cache.impl.TaskInstanceCacheManagerImpl;
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.executor.NettyKillManager;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.Date; import java.util.Date;
import static org.apache.dolphinscheduler.common.Constants.DOLPHINSCHEDULER_TASKS_KILL;
/** /**
* master task exec thread * master task exec thread
@ -52,6 +56,9 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread {
*/ */
private TaskInstanceCacheManager taskInstanceCacheManager; private TaskInstanceCacheManager taskInstanceCacheManager;
private NettyKillManager nettyKillManager;
/** /**
* constructor of MasterTaskExecThread * constructor of MasterTaskExecThread
* @param taskInstance task instance * @param taskInstance task instance
@ -60,6 +67,7 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread {
public MasterTaskExecThread(TaskInstance taskInstance, ProcessInstance processInstance){ public MasterTaskExecThread(TaskInstance taskInstance, ProcessInstance processInstance){
super(taskInstance, processInstance); super(taskInstance, processInstance);
this.taskInstanceCacheManager = SpringApplicationContext.getBean(TaskInstanceCacheManagerImpl.class); this.taskInstanceCacheManager = SpringApplicationContext.getBean(TaskInstanceCacheManagerImpl.class);
this.nettyKillManager = SpringApplicationContext.getBean(NettyKillManager.class);
} }
/** /**
@ -78,6 +86,7 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread {
/** /**
* TODO submit task instance and wait complete * TODO submit task instance and wait complete
*
* @return true is task quit is true * @return true is task quit is true
*/ */
@Override @Override
@ -99,14 +108,14 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread {
} }
/** /**
* TODO 在这里轮询数据库 * TODO polling db
* *
* wait task quit * wait task quit
* @return true if task quit success * @return true if task quit success
*/ */
public Boolean waitTaskQuit(){ public Boolean waitTaskQuit(){
// query new state // query new state
taskInstance = taskInstanceCacheManager.getByTaskInstanceId(taskInstance.getId()); taskInstance = processService.findTaskInstanceById(taskInstance.getId());
logger.info("wait task: process id: {}, task id:{}, task name:{} complete", logger.info("wait task: process id: {}, task id:{}, task name:{} complete",
this.taskInstance.getProcessInstanceId(), this.taskInstance.getId(), this.taskInstance.getName()); this.taskInstance.getProcessInstanceId(), this.taskInstance.getId(), this.taskInstance.getName());
// task time out // task time out
@ -147,7 +156,7 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread {
} }
} }
// updateProcessInstance task instance // updateProcessInstance task instance
taskInstance = taskInstanceCacheManager.getByTaskInstanceId(taskInstance.getId()); taskInstance = processService.findTaskInstanceById(taskInstance.getId());
processInstance = processService.findProcessInstanceById(processInstance.getId()); processInstance = processService.findProcessInstanceById(processInstance.getId());
Thread.sleep(Constants.SLEEP_TIME_MILLIS); Thread.sleep(Constants.SLEEP_TIME_MILLIS);
} catch (Exception e) { } catch (Exception e) {
@ -163,23 +172,26 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread {
/** /**
* TODO Kill 任务 * TODO Kill TASK
* *
* task instance add queue , waiting worker to kill * task instance add queue , waiting worker to kill
*/ */
private void cancelTaskInstance(){ private void cancelTaskInstance() throws Exception{
if(alreadyKilled){ if(alreadyKilled){
return ; return ;
} }
alreadyKilled = true; alreadyKilled = true;
String host = taskInstance.getHost();
if(host == null){ TaskExecutionContext taskExecutionContext = super.getTaskExecutionContext(taskInstance);
host = Constants.NULL;
} ExecutionContext executionContext = new ExecutionContext(taskExecutionContext, ExecutorType.WORKER);
String queueValue = String.format("%s-%d",
host, taskInstance.getId()); Host host = new Host();
// TODO 这里写 host.setIp(taskInstance.getHost());
taskQueue.sadd(DOLPHINSCHEDULER_TASKS_KILL, queueValue); host.setPort(12346);
executionContext.setHost(host);
nettyKillManager.execute(executionContext);
logger.info("master add kill task :{} id:{} to kill queue", logger.info("master add kill task :{} id:{} to kill queue",
taskInstance.getName(), taskInstance.getId() ); taskInstance.getName(), taskInstance.getId() );
@ -197,7 +209,7 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread {
/** /**
* get remain times * get remain time?s?
* *
* @return remain time * @return remain time
*/ */

5
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/Monitor.java

@ -23,6 +23,11 @@ public interface Monitor {
/** /**
* monitor server and restart * monitor server and restart
*
* @param masterPath masterPath
* @param workerPath workerPath
* @param port port
* @param installPath installPath
*/ */
void monitor(String masterPath, String workerPath, Integer port, String installPath); void monitor(String masterPath, String workerPath, Integer port, String installPath);
} }

6
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/registry/ZookeeperNodeManager.java

@ -74,7 +74,7 @@ public class ZookeeperNodeManager implements InitializingBean {
private ZookeeperRegistryCenter registryCenter; private ZookeeperRegistryCenter registryCenter;
/** /**
* init listener * init listener
* @throws Exception * @throws Exception
*/ */
@Override @Override
@ -234,8 +234,8 @@ public class ZookeeperNodeManager implements InitializingBean {
/** /**
* get worker group nodes * get worker group nodes
* @param workerGroup * @param workerGroup workerGroup
* @return * @return worker nodes
*/ */
public Set<String> getWorkerGroupNodes(String workerGroup){ public Set<String> getWorkerGroupNodes(String workerGroup){
workerGroupLock.lock(); workerGroupLock.lock();

6
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/registry/ZookeeperRegistryCenter.java

@ -129,7 +129,7 @@ public class ZookeeperRegistryCenter implements InitializingBean {
/** /**
* get worker group directly * get worker group directly
* @return * @return worker group nodes
*/ */
public Set<String> getWorkerGroupDirectly() { public Set<String> getWorkerGroupDirectly() {
List<String> workers = getChildrenKeys(getWorkerPath()); List<String> workers = getChildrenKeys(getWorkerPath());
@ -166,8 +166,8 @@ public class ZookeeperRegistryCenter implements InitializingBean {
/** /**
* get worker group path * get worker group path
* @param workerGroup * @param workerGroup workerGroup
* @return * @return worker group path
*/ */
public String getWorkerGroupPath(String workerGroup) { public String getWorkerGroupPath(String workerGroup) {
return WORKER_PATH + "/" + workerGroup; return WORKER_PATH + "/" + workerGroup;

3
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ParamUtils.java

@ -111,7 +111,8 @@ public class ParamUtils {
/** /**
* get parameters map * get parameters map
* @return user defined params map * @param definedParams definedParams
* @return parameters map
*/ */
public static Map<String,Property> getUserDefParamsMap(Map<String,String> definedParams) { public static Map<String,Property> getUserDefParamsMap(Map<String,String> definedParams) {
if (definedParams != null) { if (definedParams != null) {

3
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java

@ -21,9 +21,8 @@ import org.apache.dolphinscheduler.common.utils.CommonUtils;
import org.apache.dolphinscheduler.common.utils.LoggerUtils; import org.apache.dolphinscheduler.common.utils.LoggerUtils;
import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.common.utils.OSUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.service.log.LogClientService; import org.apache.dolphinscheduler.service.log.LogClientService;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;

3
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/cache/TaskExecutionContextCacheManager.java vendored

@ -17,7 +17,8 @@
package org.apache.dolphinscheduler.server.worker.cache; package org.apache.dolphinscheduler.server.worker.cache;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
/** /**
* TaskExecutionContextCacheManager * TaskExecutionContextCacheManager

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/cache/impl/TaskExecutionContextCacheManagerImpl.java vendored

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.worker.cache.impl; package org.apache.dolphinscheduler.server.worker.cache.impl;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.worker.cache.TaskExecutionContextCacheManager; import org.apache.dolphinscheduler.server.worker.cache.TaskExecutionContextCacheManager;
import org.springframework.stereotype.Service; import org.springframework.stereotype.Service;

3
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskExecuteProcessor.java

@ -33,13 +33,12 @@ import org.apache.dolphinscheduler.remote.command.Command;
import org.apache.dolphinscheduler.remote.command.CommandType; import org.apache.dolphinscheduler.remote.command.CommandType;
import org.apache.dolphinscheduler.remote.command.ExecuteTaskAckCommand; import org.apache.dolphinscheduler.remote.command.ExecuteTaskAckCommand;
import org.apache.dolphinscheduler.remote.command.ExecuteTaskRequestCommand; import org.apache.dolphinscheduler.remote.command.ExecuteTaskRequestCommand;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
import org.apache.dolphinscheduler.server.worker.runner.TaskExecuteThread; import org.apache.dolphinscheduler.server.worker.runner.TaskExecuteThread;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;

49
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskKillProcessor.java

@ -17,6 +17,7 @@
package org.apache.dolphinscheduler.server.worker.processor; package org.apache.dolphinscheduler.server.worker.processor;
import com.alibaba.fastjson.JSONObject;
import io.netty.channel.Channel; import io.netty.channel.Channel;
import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
@ -28,9 +29,9 @@ import org.apache.dolphinscheduler.remote.command.Command;
import org.apache.dolphinscheduler.remote.command.CommandType; import org.apache.dolphinscheduler.remote.command.CommandType;
import org.apache.dolphinscheduler.remote.command.KillTaskRequestCommand; import org.apache.dolphinscheduler.remote.command.KillTaskRequestCommand;
import org.apache.dolphinscheduler.remote.command.KillTaskResponseCommand; import org.apache.dolphinscheduler.remote.command.KillTaskResponseCommand;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ProcessUtils; import org.apache.dolphinscheduler.server.utils.ProcessUtils;
import org.apache.dolphinscheduler.server.worker.cache.TaskExecutionContextCacheManager; import org.apache.dolphinscheduler.server.worker.cache.TaskExecutionContextCacheManager;
import org.apache.dolphinscheduler.server.worker.cache.impl.TaskExecutionContextCacheManagerImpl; import org.apache.dolphinscheduler.server.worker.cache.impl.TaskExecutionContextCacheManagerImpl;
@ -75,32 +76,35 @@ public class TaskKillProcessor implements NettyRequestProcessor {
this.workerConfig = SpringApplicationContext.getBean(WorkerConfig.class); this.workerConfig = SpringApplicationContext.getBean(WorkerConfig.class);
this.taskExecutionContextCacheManager = SpringApplicationContext.getBean(TaskExecutionContextCacheManagerImpl.class); this.taskExecutionContextCacheManager = SpringApplicationContext.getBean(TaskExecutionContextCacheManagerImpl.class);
} }
/** /**
* kill task logic * kill task logic
* *
* @param killCommand killCommand * @param context context
* @return execute result
*/ */
private Boolean doKill(KillTaskRequestCommand killCommand){ private Boolean doKill(TaskExecutionContext context){
try { try {
TaskExecutionContext taskExecutionContext = taskExecutionContextCacheManager.getByTaskInstanceId(killCommand.getTaskInstanceId()); TaskExecutionContext taskExecutionContext = taskExecutionContextCacheManager.getByTaskInstanceId(context.getTaskInstanceId());
context.setProcessId(taskExecutionContext.getProcessId());
Integer processId = taskExecutionContext.getProcessId(); Integer processId = taskExecutionContext.getProcessId();
if (processId == null || processId.equals(0)){ if (processId == null || processId.equals(0)){
logger.error("process kill failed, process id :{}, task id:{}", processId, killCommand.getTaskInstanceId()); logger.error("process kill failed, process id :{}, task id:{}", processId, context.getTaskInstanceId());
return false; return false;
} }
killCommand.setProcessId(processId);
String cmd = String.format("sudo kill -9 %s", ProcessUtils.getPidsStr(killCommand.getProcessId())); String cmd = String.format("sudo kill -9 %s", ProcessUtils.getPidsStr(context.getProcessId()));
logger.info("process id:{}, cmd:{}", killCommand.getProcessId(), cmd); logger.info("process id:{}, cmd:{}", context.getProcessId(), cmd);
OSUtils.exeCmd(cmd); OSUtils.exeCmd(cmd);
// find log and kill yarn job // find log and kill yarn job
killYarnJob(killCommand.getHost(), killCommand.getLogPath(), killCommand.getExecutePath(), killCommand.getTenantCode()); killYarnJob(context.getHost(), context.getLogPath(), context.getExecutePath(), context.getTenantCode());
return true; return true;
} catch (Exception e) { } catch (Exception e) {
@ -115,29 +119,37 @@ public class TaskKillProcessor implements NettyRequestProcessor {
KillTaskRequestCommand killTaskRequestCommand = FastJsonSerializer.deserialize(command.getBody(), KillTaskRequestCommand.class); KillTaskRequestCommand killTaskRequestCommand = FastJsonSerializer.deserialize(command.getBody(), KillTaskRequestCommand.class);
logger.info("received command : {}", killTaskRequestCommand); logger.info("received command : {}", killTaskRequestCommand);
Boolean killStatus = doKill(killTaskRequestCommand);
KillTaskResponseCommand killTaskResponseCommand = buildKillTaskResponseCommand(killTaskRequestCommand,killStatus); String contextJson = killTaskRequestCommand.getTaskExecutionContext();
TaskExecutionContext taskExecutionContext = JSONObject.parseObject(contextJson, TaskExecutionContext.class);
Boolean killStatus = doKill(taskExecutionContext);
killTaskCallbackService.addRemoteChannel(taskExecutionContext.getTaskInstanceId(),
new NettyRemoteChannel(channel, command.getOpaque()));
KillTaskResponseCommand killTaskResponseCommand = buildKillTaskResponseCommand(taskExecutionContext,killStatus);
killTaskCallbackService.sendKillResult(killTaskResponseCommand.getTaskInstanceId(),killTaskResponseCommand); killTaskCallbackService.sendKillResult(killTaskResponseCommand.getTaskInstanceId(),killTaskResponseCommand);
} }
/** /**
* build KillTaskResponseCommand * build KillTaskResponseCommand
* *
* @param killTaskRequestCommand killTaskRequestCommand * @param taskExecutionContext taskExecutionContext
* @param killStatus killStatus * @param killStatus killStatus
* @return KillTaskResponseCommand * @return build KillTaskResponseCommand
*/ */
private KillTaskResponseCommand buildKillTaskResponseCommand(KillTaskRequestCommand killTaskRequestCommand, private KillTaskResponseCommand buildKillTaskResponseCommand(TaskExecutionContext taskExecutionContext,
Boolean killStatus) { Boolean killStatus) {
KillTaskResponseCommand killTaskResponseCommand = new KillTaskResponseCommand(); KillTaskResponseCommand killTaskResponseCommand = new KillTaskResponseCommand();
killTaskResponseCommand.setTaskInstanceId(killTaskRequestCommand.getTaskInstanceId()); killTaskResponseCommand.setTaskInstanceId(taskExecutionContext.getTaskInstanceId());
killTaskResponseCommand.setHost(killTaskRequestCommand.getHost()); killTaskResponseCommand.setHost(taskExecutionContext.getHost());
killTaskResponseCommand.setStatus(killStatus ? ExecutionStatus.SUCCESS.getCode() : ExecutionStatus.FAILURE.getCode()); killTaskResponseCommand.setStatus(killStatus ? ExecutionStatus.SUCCESS.getCode() : ExecutionStatus.FAILURE.getCode());
killTaskResponseCommand.setProcessId(killTaskRequestCommand.getProcessId()); killTaskResponseCommand.setProcessId(taskExecutionContext.getProcessId());
killTaskResponseCommand.setAppIds(appIds); killTaskResponseCommand.setAppIds(appIds);
return null; return killTaskResponseCommand;
} }
/** /**
@ -156,6 +168,7 @@ public class TaskKillProcessor implements NettyRequestProcessor {
String log = null; String log = null;
try { try {
logClient = new LogClientService(); logClient = new LogClientService();
logger.info("view log host : {},logPath : {}", host,logPath);
log = logClient.viewLog(host, Constants.RPC_PORT, logPath); log = logClient.viewLog(host, Constants.RPC_PORT, logPath);
} finally { } finally {
if(logClient != null){ if(logClient != null){

4
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistry.java

@ -76,9 +76,11 @@ public class WorkerRegistry {
private String workerGroup; private String workerGroup;
/** /**
* construct * construct
*
* @param zookeeperRegistryCenter zookeeperRegistryCenter * @param zookeeperRegistryCenter zookeeperRegistryCenter
* @param port port * @param port port
* @param heartBeatInterval heartBeatInterval
*/ */
public WorkerRegistry(ZookeeperRegistryCenter zookeeperRegistryCenter, int port, long heartBeatInterval){ public WorkerRegistry(ZookeeperRegistryCenter zookeeperRegistryCenter, int port, long heartBeatInterval){
this(zookeeperRegistryCenter, port, heartBeatInterval, DEFAULT_WORKER_GROUP); this(zookeeperRegistryCenter, port, heartBeatInterval, DEFAULT_WORKER_GROUP);

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskExecuteThread.java

@ -25,7 +25,7 @@ import org.apache.dolphinscheduler.common.task.AbstractParameters;
import org.apache.dolphinscheduler.common.task.TaskTimeoutParameter; import org.apache.dolphinscheduler.common.task.TaskTimeoutParameter;
import org.apache.dolphinscheduler.common.utils.*; import org.apache.dolphinscheduler.common.utils.*;
import org.apache.dolphinscheduler.remote.command.ExecuteTaskResponseCommand; import org.apache.dolphinscheduler.remote.command.ExecuteTaskResponseCommand;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.worker.processor.TaskCallbackService; import org.apache.dolphinscheduler.server.worker.processor.TaskCallbackService;
import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
import org.apache.dolphinscheduler.server.worker.task.TaskManager; import org.apache.dolphinscheduler.server.worker.task.TaskManager;

23
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java

@ -21,17 +21,13 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.common.utils.HadoopUtils; import org.apache.dolphinscheduler.common.utils.HadoopUtils;
import org.apache.dolphinscheduler.common.utils.OSUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils; 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.common.utils.LoggerUtils; import org.apache.dolphinscheduler.common.utils.LoggerUtils;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ProcessUtils; import org.apache.dolphinscheduler.server.utils.ProcessUtils;
import org.apache.dolphinscheduler.server.worker.cache.TaskExecutionContextCacheManager; import org.apache.dolphinscheduler.server.worker.cache.TaskExecutionContextCacheManager;
import org.apache.dolphinscheduler.server.worker.cache.impl.TaskExecutionContextCacheManagerImpl; import org.apache.dolphinscheduler.server.worker.cache.impl.TaskExecutionContextCacheManagerImpl;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -66,11 +62,6 @@ public abstract class AbstractCommandExecutor {
*/ */
protected Consumer<List<String>> logHandler; protected Consumer<List<String>> logHandler;
/**
* timeout
*/
protected int timeout;
/** /**
* logger * logger
*/ */
@ -132,6 +123,7 @@ public abstract class AbstractCommandExecutor {
/** /**
* task specific execution logic * task specific execution logic
*
* @param execCommand execCommand * @param execCommand execCommand
* @return CommandExecuteResult * @return CommandExecuteResult
* @throws Exception * @throws Exception
@ -174,8 +166,6 @@ public abstract class AbstractCommandExecutor {
// waiting for the run to finish // waiting for the run to finish
boolean status = process.waitFor(remainTime, TimeUnit.SECONDS); boolean status = process.waitFor(remainTime, TimeUnit.SECONDS);
// SHELL task state
result.setExitStatusCode(process.exitValue());
logger.info("process has exited, execute path:{}, processId:{} ,exitStatusCode:{}", logger.info("process has exited, execute path:{}, processId:{} ,exitStatusCode:{}",
taskExecutionContext.getExecutePath(), taskExecutionContext.getExecutePath(),
@ -195,6 +185,9 @@ public abstract class AbstractCommandExecutor {
ProcessUtils.kill(taskExecutionContext); ProcessUtils.kill(taskExecutionContext);
result.setExitStatusCode(EXIT_CODE_FAILURE); result.setExitStatusCode(EXIT_CODE_FAILURE);
} }
// SHELL task state
result.setExitStatusCode(process.exitValue());
return result; return result;
} }
@ -378,7 +371,7 @@ public abstract class AbstractCommandExecutor {
List<String> appIds = new ArrayList<>(); List<String> appIds = new ArrayList<>();
/** /**
* analysis logget submited yarn application id * analysis log?get submited yarn application id
*/ */
for (String log : logs) { for (String log : logs) {
String appId = findAppId(log); String appId = findAppId(log);
@ -440,13 +433,13 @@ public abstract class AbstractCommandExecutor {
/** /**
* get remain times * get remain time?s?
* *
* @return remain time * @return remain time
*/ */
private long getRemaintime() { private long getRemaintime() {
long usedTime = (System.currentTimeMillis() - taskExecutionContext.getStartTime().getTime()) / 1000; long usedTime = (System.currentTimeMillis() - taskExecutionContext.getStartTime().getTime()) / 1000;
long remainTime = timeout - usedTime; long remainTime = taskExecutionContext.getTaskTimeout() - usedTime;
if (remainTime < 0) { if (remainTime < 0) {
throw new RuntimeException("task execution time out"); throw new RuntimeException("task execution time out");

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractTask.java

@ -30,7 +30,7 @@ import org.apache.dolphinscheduler.common.task.spark.SparkParameters;
import org.apache.dolphinscheduler.common.task.sql.SqlParameters; import org.apache.dolphinscheduler.common.task.sql.SqlParameters;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.TaskRecordDao; import org.apache.dolphinscheduler.dao.TaskRecordDao;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.utils.ParamUtils;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.slf4j.Logger; import org.slf4j.Logger;

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.worker.task; package org.apache.dolphinscheduler.server.worker.task;
import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ProcessUtils; import org.apache.dolphinscheduler.server.utils.ProcessUtils;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.process.ProcessService;

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/PythonCommandExecutor.java

@ -19,7 +19,7 @@ package org.apache.dolphinscheduler.server.worker.task;
import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.utils.FileUtils; import org.apache.dolphinscheduler.common.utils.FileUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;

3
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/ShellCommandExecutor.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.server.worker.task; package org.apache.dolphinscheduler.server.worker.task;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.slf4j.Logger; import org.slf4j.Logger;
import java.io.File; import java.io.File;
@ -25,7 +25,6 @@ import java.io.IOException;
import java.nio.charset.Charset; import java.nio.charset.Charset;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.Paths; import java.nio.file.Paths;
import java.util.Date;
import java.util.List; import java.util.List;
import java.util.function.Consumer; import java.util.function.Consumer;

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskManager.java

@ -19,7 +19,7 @@ package org.apache.dolphinscheduler.server.worker.task;
import org.apache.dolphinscheduler.common.enums.TaskType; import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.utils.EnumUtils; import org.apache.dolphinscheduler.common.utils.EnumUtils;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.worker.task.datax.DataxTask; import org.apache.dolphinscheduler.server.worker.task.datax.DataxTask;
import org.apache.dolphinscheduler.server.worker.task.flink.FlinkTask; import org.apache.dolphinscheduler.server.worker.task.flink.FlinkTask;
import org.apache.dolphinscheduler.server.worker.task.http.HttpTask; import org.apache.dolphinscheduler.server.worker.task.http.HttpTask;

28
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskProps.java

@ -129,19 +129,21 @@ public class TaskProps {
/** /**
* constructor * constructor
* @param taskParams task params * @param taskParams taskParams
* @param taskDir task dir * @param scheduleTime scheduleTime
* @param scheduleTime schedule time * @param nodeName nodeName
* @param nodeName node name * @param taskType taskType
* @param taskType task type * @param taskInstanceId taskInstanceId
* @param taskInstanceId task instance id * @param envFile envFile
* @param envFile env file * @param tenantCode tenantCode
* @param tenantCode tenant code * @param queue queue
* @param queue queue * @param taskStartTime taskStartTime
* @param taskStartTime task start time * @param definedParams definedParams
* @param definedParams defined params * @param dependence dependence
* @param dependence dependence * @param cmdTypeIfComplement cmdTypeIfComplement
* @param cmdTypeIfComplement cmd type if complement * @param host host
* @param logPath logPath
* @param executePath executePath
*/ */
public TaskProps(String taskParams, public TaskProps(String taskParams,
Date scheduleTime, Date scheduleTime,

56
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java

@ -39,6 +39,7 @@ import java.util.Set;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.DataType;
import org.apache.dolphinscheduler.common.enums.DbType; import org.apache.dolphinscheduler.common.enums.DbType;
import org.apache.dolphinscheduler.common.process.Property; import org.apache.dolphinscheduler.common.process.Property;
import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.AbstractParameters;
@ -50,13 +51,13 @@ import org.apache.dolphinscheduler.dao.datasource.BaseDataSource;
import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory;
import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.DataSource;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.DataxTaskExecutionContext;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.DataxUtils; import org.apache.dolphinscheduler.server.utils.DataxUtils;
import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.utils.ParamUtils;
import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
import org.apache.dolphinscheduler.server.worker.task.CommandExecuteResult; import org.apache.dolphinscheduler.server.worker.task.CommandExecuteResult;
import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor; import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor;
import org.apache.dolphinscheduler.server.worker.task.TaskProps;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.process.ProcessService;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -98,11 +99,6 @@ public class DataxTask extends AbstractTask {
*/ */
private DataxParameters dataXParameters; private DataxParameters dataXParameters;
/**
* task dir
*/
private String taskDir;
/** /**
* shell command executor * shell command executor
*/ */
@ -113,11 +109,6 @@ public class DataxTask extends AbstractTask {
*/ */
private TaskExecutionContext taskExecutionContext; private TaskExecutionContext taskExecutionContext;
/**
* processService
*/
private ProcessService processService;
/** /**
* constructor * constructor
* @param taskExecutionContext taskExecutionContext * @param taskExecutionContext taskExecutionContext
@ -127,13 +118,9 @@ public class DataxTask extends AbstractTask {
super(taskExecutionContext, logger); super(taskExecutionContext, logger);
this.taskExecutionContext = taskExecutionContext; this.taskExecutionContext = taskExecutionContext;
logger.info("task dir : {}", taskDir);
this.shellCommandExecutor = new ShellCommandExecutor(this::logHandle, this.shellCommandExecutor = new ShellCommandExecutor(this::logHandle,
taskExecutionContext,logger); taskExecutionContext,logger);
processService = SpringApplicationContext.getBean(ProcessService.class);
} }
/** /**
@ -155,8 +142,7 @@ public class DataxTask extends AbstractTask {
* @throws Exception * @throws Exception
*/ */
@Override @Override
public void handle() public void handle() throws Exception {
throws Exception {
try { try {
// set the name of the current thread // set the name of the current thread
String threadLoggerInfoName = String.format("TaskLogInfo-%s", taskExecutionContext.getTaskAppId()); String threadLoggerInfoName = String.format("TaskLogInfo-%s", taskExecutionContext.getTaskAppId());
@ -181,7 +167,7 @@ public class DataxTask extends AbstractTask {
/** /**
* cancel DataX process * cancel DataX process
* *
* @param cancelApplication * @param cancelApplication cancelApplication
* @throws Exception * @throws Exception
*/ */
@Override @Override
@ -200,7 +186,9 @@ public class DataxTask extends AbstractTask {
private String buildDataxJsonFile() private String buildDataxJsonFile()
throws Exception { throws Exception {
// generate json // generate json
String fileName = String.format("%s/%s_job.json", taskDir, taskExecutionContext.getTaskAppId()); String fileName = String.format("%s/%s_job.json",
taskExecutionContext.getExecutePath(),
taskExecutionContext.getTaskAppId());
Path path = new File(fileName).toPath(); Path path = new File(fileName).toPath();
if (Files.exists(path)) { if (Files.exists(path)) {
@ -230,13 +218,14 @@ public class DataxTask extends AbstractTask {
*/ */
private List<JSONObject> buildDataxJobContentJson() private List<JSONObject> buildDataxJobContentJson()
throws SQLException { throws SQLException {
DataSource dataSource = processService.findDataSourceById(dataXParameters.getDataSource()); DataxTaskExecutionContext dataxTaskExecutionContext = taskExecutionContext.getDataxTaskExecutionContext();
BaseDataSource dataSourceCfg = DataSourceFactory.getDatasource(dataSource.getType(),
dataSource.getConnectionParams());
DataSource dataTarget = processService.findDataSourceById(dataXParameters.getDataTarget());
BaseDataSource dataTargetCfg = DataSourceFactory.getDatasource(dataTarget.getType(), BaseDataSource dataSourceCfg = DataSourceFactory.getDatasource(DbType.of(dataxTaskExecutionContext.getSourcetype()),
dataTarget.getConnectionParams()); dataxTaskExecutionContext.getSourceConnectionParams());
BaseDataSource dataTargetCfg = DataSourceFactory.getDatasource(DbType.of(dataxTaskExecutionContext.getTargetType()),
dataxTaskExecutionContext.getTargetConnectionParams());
List<JSONObject> readerConnArr = new ArrayList<>(); List<JSONObject> readerConnArr = new ArrayList<>();
JSONObject readerConn = new JSONObject(); JSONObject readerConn = new JSONObject();
@ -250,7 +239,7 @@ public class DataxTask extends AbstractTask {
readerParam.put("connection", readerConnArr); readerParam.put("connection", readerConnArr);
JSONObject reader = new JSONObject(); JSONObject reader = new JSONObject();
reader.put("name", DataxUtils.getReaderPluginName(dataSource.getType())); reader.put("name", DataxUtils.getReaderPluginName(DbType.of(dataxTaskExecutionContext.getSourcetype())));
reader.put("parameter", readerParam); reader.put("parameter", readerParam);
List<JSONObject> writerConnArr = new ArrayList<>(); List<JSONObject> writerConnArr = new ArrayList<>();
@ -263,7 +252,9 @@ public class DataxTask extends AbstractTask {
writerParam.put("username", dataTargetCfg.getUser()); writerParam.put("username", dataTargetCfg.getUser());
writerParam.put("password", dataTargetCfg.getPassword()); writerParam.put("password", dataTargetCfg.getPassword());
writerParam.put("column", writerParam.put("column",
parsingSqlColumnNames(dataSource.getType(), dataTarget.getType(), dataSourceCfg, dataXParameters.getSql())); parsingSqlColumnNames(DbType.of(dataxTaskExecutionContext.getSourcetype()),
DbType.of(dataxTaskExecutionContext.getTargetType()),
dataSourceCfg, dataXParameters.getSql()));
writerParam.put("connection", writerConnArr); writerParam.put("connection", writerConnArr);
if (CollectionUtils.isNotEmpty(dataXParameters.getPreStatements())) { if (CollectionUtils.isNotEmpty(dataXParameters.getPreStatements())) {
@ -275,7 +266,7 @@ public class DataxTask extends AbstractTask {
} }
JSONObject writer = new JSONObject(); JSONObject writer = new JSONObject();
writer.put("name", DataxUtils.getWriterPluginName(dataTarget.getType())); writer.put("name", DataxUtils.getWriterPluginName(DbType.of(dataxTaskExecutionContext.getTargetType())));
writer.put("parameter", writerParam); writer.put("parameter", writerParam);
List<JSONObject> contentList = new ArrayList<>(); List<JSONObject> contentList = new ArrayList<>();
@ -348,7 +339,9 @@ public class DataxTask extends AbstractTask {
private String buildShellCommandFile(String jobConfigFilePath) private String buildShellCommandFile(String jobConfigFilePath)
throws Exception { throws Exception {
// generate scripts // generate scripts
String fileName = String.format("%s/%s_node.sh", taskDir, taskExecutionContext.getTaskAppId()); String fileName = String.format("%s/%s_node.sh",
taskExecutionContext.getExecutePath(),
taskExecutionContext.getTaskAppId());
Path path = new File(fileName).toPath(); Path path = new File(fileName).toPath();
if (Files.exists(path)) { if (Files.exists(path)) {
@ -364,9 +357,6 @@ public class DataxTask extends AbstractTask {
sbr.append(jobConfigFilePath); sbr.append(jobConfigFilePath);
String dataxCommand = sbr.toString(); String dataxCommand = sbr.toString();
// find process instance by task id
ProcessInstance processInstance = processService.findProcessInstanceByTaskId(taskExecutionContext.getTaskInstanceId());
// combining local and global parameters // combining local and global parameters
// replace placeholder // replace placeholder
Map<String, Property> paramsMap = ParamUtils.convert(ParamUtils.getUserDefParamsMap(taskExecutionContext.getDefinedParams()), Map<String, Property> paramsMap = ParamUtils.convert(ParamUtils.getUserDefParamsMap(taskExecutionContext.getDefinedParams()),

4
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/flink/FlinkTask.java

@ -23,12 +23,10 @@ import org.apache.dolphinscheduler.common.task.flink.FlinkParameters;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.FlinkArgsUtils; import org.apache.dolphinscheduler.server.utils.FlinkArgsUtils;
import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.utils.ParamUtils;
import org.apache.dolphinscheduler.server.worker.task.AbstractYarnTask; import org.apache.dolphinscheduler.server.worker.task.AbstractYarnTask;
import org.apache.dolphinscheduler.server.worker.task.TaskProps;
import org.slf4j.Logger; import org.slf4j.Logger;
import java.util.ArrayList; import java.util.ArrayList;

12
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java

@ -30,13 +30,9 @@ import org.apache.dolphinscheduler.common.task.http.HttpParameters;
import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.utils.ParamUtils;
import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
import org.apache.dolphinscheduler.server.worker.task.TaskProps;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.http.HttpEntity; import org.apache.http.HttpEntity;
import org.apache.http.ParseException; import org.apache.http.ParseException;
import org.apache.http.client.config.RequestConfig; import org.apache.http.client.config.RequestConfig;
@ -67,10 +63,7 @@ public class HttpTask extends AbstractTask {
*/ */
private HttpParameters httpParameters; private HttpParameters httpParameters;
/**
* process service
*/
private ProcessService processService;
/** /**
* Convert mill seconds to second unit * Convert mill seconds to second unit
@ -146,7 +139,6 @@ public class HttpTask extends AbstractTask {
*/ */
protected CloseableHttpResponse sendRequest(CloseableHttpClient client) throws IOException { protected CloseableHttpResponse sendRequest(CloseableHttpClient client) throws IOException {
RequestBuilder builder = createRequestBuilder(); RequestBuilder builder = createRequestBuilder();
ProcessInstance processInstance = processService.findProcessInstanceByTaskId(taskExecutionContext.getTaskInstanceId());
// replace placeholder // replace placeholder
Map<String, Property> paramsMap = ParamUtils.convert(ParamUtils.getUserDefParamsMap(taskExecutionContext.getDefinedParams()), Map<String, Property> paramsMap = ParamUtils.convert(ParamUtils.getUserDefParamsMap(taskExecutionContext.getDefinedParams()),

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/mr/MapReduceTask.java

@ -25,7 +25,7 @@ import org.apache.dolphinscheduler.common.task.mr.MapreduceParameters;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.utils.ParamUtils;
import org.apache.dolphinscheduler.server.worker.task.AbstractYarnTask; import org.apache.dolphinscheduler.server.worker.task.AbstractYarnTask;
import org.apache.dolphinscheduler.server.worker.task.TaskProps; import org.apache.dolphinscheduler.server.worker.task.TaskProps;

3
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java

@ -31,10 +31,9 @@ import org.apache.dolphinscheduler.common.utils.ParameterUtils;
import org.apache.dolphinscheduler.dao.datasource.BaseDataSource; import org.apache.dolphinscheduler.dao.datasource.BaseDataSource;
import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory;
import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.DataSource;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.utils.ParamUtils;
import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
import org.apache.dolphinscheduler.server.worker.task.TaskProps;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.process.ProcessService;
import org.slf4j.Logger; import org.slf4j.Logger;

5
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java

@ -24,14 +24,11 @@ import org.apache.dolphinscheduler.common.task.AbstractParameters;
import org.apache.dolphinscheduler.common.task.python.PythonParameters; import org.apache.dolphinscheduler.common.task.python.PythonParameters;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.utils.ParamUtils;
import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
import org.apache.dolphinscheduler.server.worker.task.CommandExecuteResult; import org.apache.dolphinscheduler.server.worker.task.CommandExecuteResult;
import org.apache.dolphinscheduler.server.worker.task.PythonCommandExecutor; import org.apache.dolphinscheduler.server.worker.task.PythonCommandExecutor;
import org.apache.dolphinscheduler.server.worker.task.TaskProps;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.slf4j.Logger; import org.slf4j.Logger;
import java.util.Map; import java.util.Map;

17
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java

@ -24,14 +24,11 @@ import org.apache.dolphinscheduler.common.task.AbstractParameters;
import org.apache.dolphinscheduler.common.task.shell.ShellParameters; import org.apache.dolphinscheduler.common.task.shell.ShellParameters;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.utils.ParamUtils;
import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
import org.apache.dolphinscheduler.server.worker.task.CommandExecuteResult; import org.apache.dolphinscheduler.server.worker.task.CommandExecuteResult;
import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor; import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor;
import org.apache.dolphinscheduler.server.worker.task.TaskProps;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.slf4j.Logger; import org.slf4j.Logger;
import java.io.File; import java.io.File;
@ -54,11 +51,6 @@ public class ShellTask extends AbstractTask {
*/ */
private ShellParameters shellParameters; private ShellParameters shellParameters;
/**
* task dir
*/
private String taskDir;
/** /**
* shell command executor * shell command executor
*/ */
@ -122,7 +114,10 @@ public class ShellTask extends AbstractTask {
*/ */
private String buildCommand() throws Exception { private String buildCommand() throws Exception {
// generate scripts // generate scripts
String fileName = String.format("%s/%s_node.sh", taskDir, taskExecutionContext.getTaskAppId()); String fileName = String.format("%s/%s_node.sh",
taskExecutionContext.getExecutePath(),
taskExecutionContext.getTaskAppId());
Path path = new File(fileName).toPath(); Path path = new File(fileName).toPath();
if (Files.exists(path)) { if (Files.exists(path)) {
@ -148,7 +143,7 @@ public class ShellTask extends AbstractTask {
shellParameters.setRawScript(script); shellParameters.setRawScript(script);
logger.info("raw script : {}", shellParameters.getRawScript()); logger.info("raw script : {}", shellParameters.getRawScript());
logger.info("task dir : {}", taskDir); logger.info("task execute path : {}", taskExecutionContext.getExecutePath());
Set<PosixFilePermission> perms = PosixFilePermissions.fromString(Constants.RWXR_XR_X); Set<PosixFilePermission> perms = PosixFilePermissions.fromString(Constants.RWXR_XR_X);
FileAttribute<Set<PosixFilePermission>> attr = PosixFilePermissions.asFileAttribute(perms); FileAttribute<Set<PosixFilePermission>> attr = PosixFilePermissions.asFileAttribute(perms);

3
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/spark/SparkTask.java

@ -24,11 +24,10 @@ import org.apache.dolphinscheduler.common.task.spark.SparkParameters;
import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.utils.ParamUtils;
import org.apache.dolphinscheduler.server.utils.SparkArgsUtils; import org.apache.dolphinscheduler.server.utils.SparkArgsUtils;
import org.apache.dolphinscheduler.server.worker.task.AbstractYarnTask; import org.apache.dolphinscheduler.server.worker.task.AbstractYarnTask;
import org.apache.dolphinscheduler.server.worker.task.TaskProps;
import org.slf4j.Logger; import org.slf4j.Logger;
import java.util.ArrayList; import java.util.ArrayList;

65
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java

@ -19,7 +19,6 @@ package org.apache.dolphinscheduler.server.worker.task.sql;
import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONArray;
import com.alibaba.fastjson.JSONObject; import com.alibaba.fastjson.JSONObject;
import com.alibaba.fastjson.serializer.SerializerFeature; import com.alibaba.fastjson.serializer.SerializerFeature;
import org.apache.commons.lang.ArrayUtils;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.apache.dolphinscheduler.alert.utils.MailUtils; import org.apache.dolphinscheduler.alert.utils.MailUtils;
import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.Constants;
@ -33,18 +32,15 @@ import org.apache.dolphinscheduler.common.utils.*;
import org.apache.dolphinscheduler.dao.AlertDao; import org.apache.dolphinscheduler.dao.AlertDao;
import org.apache.dolphinscheduler.dao.datasource.BaseDataSource; import org.apache.dolphinscheduler.dao.datasource.BaseDataSource;
import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory;
import org.apache.dolphinscheduler.dao.entity.DataSource;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.UdfFunc; import org.apache.dolphinscheduler.dao.entity.UdfFunc;
import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.dolphinscheduler.remote.entity.TaskExecutionContext; import org.apache.dolphinscheduler.server.entity.SQLTaskExecutionContext;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.utils.ParamUtils;
import org.apache.dolphinscheduler.server.utils.UDFUtils; import org.apache.dolphinscheduler.server.utils.UDFUtils;
import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.AbstractTask;
import org.apache.dolphinscheduler.server.worker.task.TaskProps;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.permission.PermissionCheck;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.slf4j.Logger; import org.slf4j.Logger;
import java.sql.*; import java.sql.*;
@ -64,22 +60,10 @@ public class SqlTask extends AbstractTask {
* sql parameters * sql parameters
*/ */
private SqlParameters sqlParameters; private SqlParameters sqlParameters;
/**
* process service
*/
private ProcessService processService;
/** /**
* alert dao * alert dao
*/ */
private AlertDao alertDao; private AlertDao alertDao;
/**
* datasource
*/
private DataSource dataSource;
/** /**
* base datasource * base datasource
*/ */
@ -102,7 +86,7 @@ public class SqlTask extends AbstractTask {
if (!sqlParameters.checkParameters()) { if (!sqlParameters.checkParameters()) {
throw new RuntimeException("sql task params is not valid"); throw new RuntimeException("sql task params is not valid");
} }
this.processService = SpringApplicationContext.getBean(ProcessService.class);
this.alertDao = SpringApplicationContext.getBean(AlertDao.class); this.alertDao = SpringApplicationContext.getBean(AlertDao.class);
} }
@ -111,6 +95,7 @@ public class SqlTask extends AbstractTask {
// set the name of the current thread // set the name of the current thread
String threadLoggerInfoName = String.format(Constants.TASK_LOG_INFO_FORMAT, taskExecutionContext.getTaskAppId()); String threadLoggerInfoName = String.format(Constants.TASK_LOG_INFO_FORMAT, taskExecutionContext.getTaskAppId());
Thread.currentThread().setName(threadLoggerInfoName); Thread.currentThread().setName(threadLoggerInfoName);
logger.info("Full sql parameters: {}", sqlParameters); logger.info("Full sql parameters: {}", sqlParameters);
logger.info("sql type : {}, datasource : {}, sql : {} , localParams : {},udfs : {},showType : {},connParams : {}", logger.info("sql type : {}, datasource : {}, sql : {} , localParams : {},udfs : {},showType : {},connParams : {}",
sqlParameters.getType(), sqlParameters.getType(),
@ -121,37 +106,15 @@ public class SqlTask extends AbstractTask {
sqlParameters.getShowType(), sqlParameters.getShowType(),
sqlParameters.getConnParams()); sqlParameters.getConnParams());
// not set data source
if (sqlParameters.getDatasource() == 0){
logger.error("datasource id not exists");
exitStatusCode = -1;
return;
}
dataSource= processService.findDataSourceById(sqlParameters.getDatasource());
// data source is null
if (dataSource == null){
logger.error("datasource not exists");
exitStatusCode = -1;
return;
}
logger.info("datasource name : {} , type : {} , desc : {} , user_id : {} , parameter : {}",
dataSource.getName(),
dataSource.getType(),
dataSource.getNote(),
dataSource.getUserId(),
dataSource.getConnectionParams());
Connection con = null; Connection con = null;
List<String> createFuncs = null; List<String> createFuncs = null;
try { try {
// load class // load class
DataSourceFactory.loadClass(dataSource.getType()); DataSourceFactory.loadClass(DbType.valueOf(sqlParameters.getType()));
// get datasource // get datasource
baseDataSource = DataSourceFactory.getDatasource(dataSource.getType(), baseDataSource = DataSourceFactory.getDatasource(DbType.valueOf(sqlParameters.getType()),
dataSource.getConnectionParams()); sqlParameters.getConnParams());
// ready to execute SQL and parameter entity Map // ready to execute SQL and parameter entity Map
SqlBinds mainSqlBinds = getSqlAndSqlParamsMap(sqlParameters.getSql()); SqlBinds mainSqlBinds = getSqlAndSqlParamsMap(sqlParameters.getSql());
@ -175,9 +138,8 @@ public class SqlTask extends AbstractTask {
for(int i=0;i<ids.length;i++){ for(int i=0;i<ids.length;i++){
idsArray[i]=Integer.parseInt(ids[i]); idsArray[i]=Integer.parseInt(ids[i]);
} }
SQLTaskExecutionContext sqlTaskExecutionContext = taskExecutionContext.getSqlTaskExecutionContext();
List<UdfFunc> udfFuncList = processService.queryUdfFunListByids(idsArray); createFuncs = UDFUtils.createFuncs(sqlTaskExecutionContext.getUdfFuncList(), taskExecutionContext.getTenantCode(), logger);
createFuncs = UDFUtils.createFuncs(udfFuncList, taskExecutionContext.getTenantCode(), logger);
} }
// execute sql task // execute sql task
@ -262,7 +224,7 @@ public class SqlTask extends AbstractTask {
CommonUtils.loadKerberosConf(); CommonUtils.loadKerberosConf();
// if hive , load connection params if exists // if hive , load connection params if exists
if (HIVE == dataSource.getType()) { if (HIVE == DbType.valueOf(sqlParameters.getType())) {
Properties paramProp = new Properties(); Properties paramProp = new Properties();
paramProp.setProperty(USER, baseDataSource.getUser()); paramProp.setProperty(USER, baseDataSource.getUser());
paramProp.setProperty(PASSWORD, baseDataSource.getPassword()); paramProp.setProperty(PASSWORD, baseDataSource.getPassword());
@ -387,10 +349,7 @@ public class SqlTask extends AbstractTask {
*/ */
public void sendAttachment(String title,String content){ public void sendAttachment(String title,String content){
// process instance List<User> users = alertDao.queryUserByAlertGroupId(taskExecutionContext.getSqlTaskExecutionContext().getWarningGroupId());
ProcessInstance instance = processService.findProcessInstanceByTaskId(taskExecutionContext.getTaskInstanceId());
List<User> users = alertDao.queryUserByAlertGroupId(instance.getWarningGroupId());
// receiving group list // receiving group list
List<String> receviersList = new ArrayList<String>(); List<String> receviersList = new ArrayList<String>();

6
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java

@ -25,8 +25,8 @@ import org.apache.dolphinscheduler.dao.AlertDao;
import org.apache.dolphinscheduler.dao.DaoFactory; import org.apache.dolphinscheduler.dao.DaoFactory;
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.remote.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.builder.TaskExecutionContextBuilder; import org.apache.dolphinscheduler.server.builder.TaskExecutionContextBuilder;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.utils.ProcessUtils; import org.apache.dolphinscheduler.server.utils.ProcessUtils;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFramework;
@ -236,6 +236,8 @@ public class ZKMasterClient extends AbstractZKClient {
/** /**
* monitor master * monitor master
* @param event event
* @param path path
*/ */
public void handleMasterEvent(TreeCacheEvent event, String path){ public void handleMasterEvent(TreeCacheEvent event, String path){
switch (event.getType()) { switch (event.getType()) {
@ -256,6 +258,8 @@ public class ZKMasterClient extends AbstractZKClient {
/** /**
* monitor worker * monitor worker
* @param event event
* @param path path
*/ */
public void handleWorkerEvent(TreeCacheEvent event, String path){ public void handleWorkerEvent(TreeCacheEvent event, String path){
switch (event.getType()) { switch (event.getType()) {

2
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKWorkerClient.java

@ -72,6 +72,8 @@ public class ZKWorkerClient extends AbstractZKClient {
/** /**
* monitor worker * monitor worker
* @param event event
* @param path path
*/ */
public void handleWorkerEvent(TreeCacheEvent event, String path){ public void handleWorkerEvent(TreeCacheEvent event, String path){
switch (event.getType()) { switch (event.getType()) {

4
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java

@ -1411,8 +1411,12 @@ public class ProcessService {
*/ */
public void changeTaskState(ExecutionStatus state, public void changeTaskState(ExecutionStatus state,
Date endTime, Date endTime,
int processId,
String appIds,
int taskInstId) { int taskInstId) {
TaskInstance taskInstance = taskInstanceMapper.selectById(taskInstId); TaskInstance taskInstance = taskInstanceMapper.selectById(taskInstId);
taskInstance.setPid(processId);
taskInstance.setAppLink(appIds);
taskInstance.setState(state); taskInstance.setState(state);
taskInstance.setEndTime(endTime); taskInstance.setEndTime(endTime);
saveTaskInstance(taskInstance); saveTaskInstance(taskInstance);

Loading…
Cancel
Save