This is an automated email from the ASF dual-hosted git repository.

gallardot pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git


The following commit(s) were added to refs/heads/dev by this push:
     new 86c066ab9b Removed unused params in TaskExecutionContext (#17195)
86c066ab9b is described below

commit 86c066ab9b83efef8ee4b6987d118a423e44d0ce
Author: Wenjun Ruan <[email protected]>
AuthorDate: Sat May 17 10:33:53 2025 +0800

    Removed unused params in TaskExecutionContext (#17195)
---
 .../task/runnable/TaskExecutionContextBuilder.java      | 17 +----------------
 .../master/runner/TaskExecutionContextFactory.java      |  6 +-----
 .../plugin/task/api/AbstractCommandExecutor.java        |  4 ----
 .../plugin/task/api/AbstractYarnTask.java               |  9 ++-------
 .../plugin/task/api/TaskExecutionContext.java           | 15 +--------------
 .../plugin/task/flink/FlinkStreamTask.java              |  6 ------
 .../dolphinscheduler/plugin/task/flink/FlinkTask.java   |  6 ------
 .../dolphinscheduler/plugin/task/mr/MapReduceTask.java  |  5 -----
 .../dolphinscheduler/plugin/task/python/PythonTask.java |  1 +
 .../dolphinscheduler/plugin/task/spark/SparkTask.java   |  5 -----
 .../dolphinscheduler/plugin/task/sqoop/SqoopTask.java   |  8 --------
 .../worker/utils/TaskExecutionContextUtilsTest.java     |  1 -
 12 files changed, 6 insertions(+), 77 deletions(-)

diff --git 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionContextBuilder.java
 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionContextBuilder.java
index edbfb9af4d..883dd77702 100644
--- 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionContextBuilder.java
+++ 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/engine/task/runnable/TaskExecutionContextBuilder.java
@@ -102,11 +102,9 @@ public class TaskExecutionContextBuilder {
         
taskExecutionContext.setScheduleTime(DateUtils.dateToTimeStamp(workflowInstance.getScheduleTime()));
         
taskExecutionContext.setGlobalParams(workflowInstance.getGlobalParams());
         taskExecutionContext.setExecutorId(workflowInstance.getExecutorId());
-        
taskExecutionContext.setCmdTypeIfComplement(workflowInstance.getCmdTypeIfComplement().getCode());
         taskExecutionContext.setTenantCode(workflowInstance.getTenantCode());
         
taskExecutionContext.setWorkflowDefinitionCode(workflowInstance.getWorkflowDefinitionCode());
         
taskExecutionContext.setWorkflowDefinitionVersion(workflowInstance.getWorkflowDefinitionVersion());
-        taskExecutionContext.setProjectCode(workflowInstance.getProjectCode());
         return this;
     }
 
@@ -128,27 +126,14 @@ public class TaskExecutionContextBuilder {
     }
 
     /**
-     * build global and local params
+     * The runtime params, include local params from task, global params from 
workflow, startup params from command, varpool params from pre-task, built-in 
params from system  
      *
-     * @param propertyMap
-     * @return
      */
     public TaskExecutionContextBuilder buildPrepareParams(final Map<String, 
Property> propertyMap) {
         taskExecutionContext.setPrepareParamsMap(propertyMap);
         return this;
     }
 
-    /**
-     * build business params
-     *
-     * @param businessParamsMap
-     * @return
-     */
-    public TaskExecutionContextBuilder buildBusinessParams(final Map<String, 
Property> businessParamsMap) {
-        taskExecutionContext.setParamsMap(businessParamsMap);
-        return this;
-    }
-
     public TaskExecutionContextBuilder buildWorkflowInstanceHost(final String 
masterHost) {
         taskExecutionContext.setWorkflowInstanceHost(masterHost);
         return this;
diff --git 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionContextFactory.java
 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionContextFactory.java
index 327cc6de99..7788b96b3f 100644
--- 
a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionContextFactory.java
+++ 
b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/TaskExecutionContextFactory.java
@@ -85,7 +85,7 @@ public class TaskExecutionContextFactory {
                 .buildTaskDefinitionRelatedInfo(request.getTaskDefinition())
                 .buildProcessInstanceRelatedInfo(request.getWorkflowInstance())
                 .buildResourceParameters(getResourceParameters(taskInstance))
-                .buildBusinessParams(getBusinessParams(workflowInstance))
+                // todo: use TaskRuntimeParameters to replace Map<String, 
Property> in TaskExecutionContext
                 .buildPrepareParams(getPrepareParams(taskInstance, 
workflowInstance, workflowDefinition, project))
                 
.buildK8sTaskRelatedInfo(getK8sTaskExecutionContext(taskInstance))
                 .create();
@@ -152,10 +152,6 @@ public class TaskExecutionContextFactory {
         return k8sTaskExecutionContext;
     }
 
-    private Map<String, Property> getBusinessParams(final WorkflowInstance 
workflowInstance) {
-        return curingParamsService.preBuildBusinessParams(workflowInstance);
-    }
-
     private Map<String, Property> getPrepareParams(final TaskInstance 
taskInstance,
                                                    final WorkflowInstance 
workflowInstance,
                                                    final WorkflowDefinition 
workflowDefinition,
diff --git 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java
 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java
index 8582804b26..67a5c4ccac 100644
--- 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java
+++ 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractCommandExecutor.java
@@ -93,10 +93,6 @@ public abstract class AbstractCommandExecutor {
         this.logBuffer = new LinkedBlockingQueue<>();
         this.logBuffer.add(EMPTY_STRING);
 
-        if (this.taskRequest != null) {
-            // set logBufferEnable=true if the task uses logHandler and 
logBuffer to buffer log messages
-            this.taskRequest.setLogBufferEnable(true);
-        }
     }
 
     // todo: We need to build the IShellActuator in outer class, since 
different task may have specific logic to build
diff --git 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractYarnTask.java
 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractYarnTask.java
index 2f538a3945..d4b98afd67 100644
--- 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractYarnTask.java
+++ 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/AbstractYarnTask.java
@@ -25,9 +25,9 @@ import 
org.apache.dolphinscheduler.plugin.task.api.model.TaskResponse;
 import 
org.apache.dolphinscheduler.plugin.task.api.shell.IShellInterceptorBuilder;
 import 
org.apache.dolphinscheduler.plugin.task.api.shell.ShellInterceptorBuilderFactory;
 import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.ParameterUtils;
 
 import java.util.List;
-import java.util.Map;
 
 import lombok.extern.slf4j.Slf4j;
 
@@ -46,7 +46,7 @@ public abstract class AbstractYarnTask extends 
AbstractRemoteTask {
     public void handle(TaskCallBack taskCallBack) throws TaskException {
         try {
             IShellInterceptorBuilder shellActuatorBuilder = 
ShellInterceptorBuilderFactory.newBuilder()
-                    .properties(getProperties())
+                    
.properties(ParameterUtils.convert(taskRequest.getPrepareParamsMap()))
                     // todo: do we need to move the replace to subclass?
                     .appendScript(getScript().replaceAll("\\r\\n", 
System.lineSeparator()));
             // SHELL task exit code
@@ -109,9 +109,4 @@ public abstract class AbstractYarnTask extends 
AbstractRemoteTask {
      * Get the script used to bootstrap the task
      */
     protected abstract String getScript();
-
-    /**
-     * Get the properties of the task used to replace the placeholders in the 
script.
-     */
-    protected abstract Map<String, String> getProperties();
 }
diff --git 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java
 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java
index c7ac6e7ef9..c13a6e4d6f 100644
--- 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java
+++ 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/TaskExecutionContext.java
@@ -80,26 +80,17 @@ public class TaskExecutionContext implements Serializable {
 
     private int executorId;
 
-    private int cmdTypeIfComplement;
-
     private String tenantCode;
 
     private int workflowDefinitionId;
 
-    private int projectId;
-
-    private long projectCode;
-
     private String taskParams;
 
     private String environmentConfig;
 
     /**
-     * definedParams
-     * // todo: we need to rename definedParams, prepareParamsMap, paramsMap, 
this is confusing
+     * Include local params, global params and system built-in params
      */
-    private Map<String, String> definedParams;
-
     private Map<String, Property> prepareParamsMap;
 
     // Please use task instanceId
@@ -126,16 +117,12 @@ public class TaskExecutionContext implements Serializable 
{
 
     private int dryRun;
 
-    private Map<String, Property> paramsMap;
-
     private Integer cpuQuota;
 
     private Integer memoryMax;
 
     private int testFlag;
 
-    private boolean logBufferEnable;
-
     private int dispatchFailTimes;
 
     private boolean failover;
diff --git 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamTask.java
 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamTask.java
index 9ec10889bb..3c2c1e046b 100644
--- 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamTask.java
+++ 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink-stream/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkStreamTask.java
@@ -28,7 +28,6 @@ import org.apache.commons.collections4.CollectionUtils;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 import java.util.stream.Collectors;
 
 import lombok.extern.slf4j.Slf4j;
@@ -70,11 +69,6 @@ public class FlinkStreamTask extends FlinkTask implements 
StreamTask {
         return args.stream().collect(Collectors.joining(" "));
     }
 
-    @Override
-    protected Map<String, String> getProperties() {
-        return taskExecutionContext.getDefinedParams();
-    }
-
     @Override
     public AbstractParameters getParameters() {
         return flinkParameters;
diff --git 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java
 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java
index 4f2963116c..4b3d9f98e1 100644
--- 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java
+++ 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-flink/src/main/java/org/apache/dolphinscheduler/plugin/task/flink/FlinkTask.java
@@ -24,7 +24,6 @@ import 
org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
 import 
org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
 
 import java.util.List;
-import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
@@ -79,11 +78,6 @@ public class FlinkTask extends AbstractYarnTask {
         return args.stream().collect(Collectors.joining(" "));
     }
 
-    @Override
-    protected Map<String, String> getProperties() {
-        return taskExecutionContext.getDefinedParams();
-    }
-
     @Override
     public AbstractParameters getParameters() {
         return flinkParameters;
diff --git 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-mr/src/main/java/org/apache/dolphinscheduler/plugin/task/mr/MapReduceTask.java
 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-mr/src/main/java/org/apache/dolphinscheduler/plugin/task/mr/MapReduceTask.java
index 0979dc73cd..9d50691924 100644
--- 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-mr/src/main/java/org/apache/dolphinscheduler/plugin/task/mr/MapReduceTask.java
+++ 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-mr/src/main/java/org/apache/dolphinscheduler/plugin/task/mr/MapReduceTask.java
@@ -91,11 +91,6 @@ public class MapReduceTask extends AbstractYarnTask {
         return args.stream().collect(Collectors.joining(" "));
     }
 
-    @Override
-    protected Map<String, String> getProperties() {
-        return taskExecutionContext.getDefinedParams();
-    }
-
     @Override
     public AbstractParameters getParameters() {
         return mapreduceParameters;
diff --git 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-python/src/main/java/org/apache/dolphinscheduler/plugin/task/python/PythonTask.java
 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-python/src/main/java/org/apache/dolphinscheduler/plugin/task/python/PythonTask.java
index 539ecd7240..1b5636a306 100644
--- 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-python/src/main/java/org/apache/dolphinscheduler/plugin/task/python/PythonTask.java
+++ 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-python/src/main/java/org/apache/dolphinscheduler/plugin/task/python/PythonTask.java
@@ -98,6 +98,7 @@ public class PythonTask extends AbstractTask {
             setProcessId(taskResponse.getProcessId());
             setTaskOutputParams(shellCommandExecutor.getTaskOutputParams());
             
pythonParameters.dealOutParam(shellCommandExecutor.getTaskOutputParams());
+            
taskRequest.setVarPool(JSONUtils.toJsonString(pythonParameters.getVarPool()));
         } catch (Exception e) {
             log.error("python task failure", e);
             setExitStatusCode(TaskConstants.EXIT_CODE_FAILURE);
diff --git 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-spark/src/main/java/org/apache/dolphinscheduler/plugin/task/spark/SparkTask.java
 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-spark/src/main/java/org/apache/dolphinscheduler/plugin/task/spark/SparkTask.java
index 3c5fc17698..5771cca4f3 100644
--- 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-spark/src/main/java/org/apache/dolphinscheduler/plugin/task/spark/SparkTask.java
+++ 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-spark/src/main/java/org/apache/dolphinscheduler/plugin/task/spark/SparkTask.java
@@ -112,11 +112,6 @@ public class SparkTask extends AbstractYarnTask {
         return args.stream().collect(Collectors.joining(" "));
     }
 
-    @Override
-    protected Map<String, String> getProperties() {
-        return 
ParameterUtils.convert(taskExecutionContext.getPrepareParamsMap());
-    }
-
     /**
      * build spark options
      *
diff --git 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-sqoop/src/main/java/org/apache/dolphinscheduler/plugin/task/sqoop/SqoopTask.java
 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-sqoop/src/main/java/org/apache/dolphinscheduler/plugin/task/sqoop/SqoopTask.java
index a77e0919e8..2db0068839 100644
--- 
a/dolphinscheduler-task-plugin/dolphinscheduler-task-sqoop/src/main/java/org/apache/dolphinscheduler/plugin/task/sqoop/SqoopTask.java
+++ 
b/dolphinscheduler-task-plugin/dolphinscheduler-task-sqoop/src/main/java/org/apache/dolphinscheduler/plugin/task/sqoop/SqoopTask.java
@@ -23,12 +23,9 @@ import 
org.apache.dolphinscheduler.plugin.task.api.TaskException;
 import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
 import org.apache.dolphinscheduler.plugin.task.api.log.SensitiveDataConverter;
 import 
org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
-import org.apache.dolphinscheduler.plugin.task.api.utils.ParameterUtils;
 import 
org.apache.dolphinscheduler.plugin.task.sqoop.generator.SqoopJobGenerator;
 import org.apache.dolphinscheduler.plugin.task.sqoop.parameter.SqoopParameters;
 
-import java.util.Map;
-
 import lombok.extern.slf4j.Slf4j;
 
 /**
@@ -75,11 +72,6 @@ public class SqoopTask extends AbstractYarnTask {
 
     }
 
-    @Override
-    protected Map<String, String> getProperties() {
-        return 
ParameterUtils.convert(taskExecutionContext.getPrepareParamsMap());
-    }
-
     @Override
     public AbstractParameters getParameters() {
         return sqoopParameters;
diff --git 
a/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/utils/TaskExecutionContextUtilsTest.java
 
b/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/utils/TaskExecutionContextUtilsTest.java
index 05d419696b..3fa08e33b1 100644
--- 
a/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/utils/TaskExecutionContextUtilsTest.java
+++ 
b/dolphinscheduler-worker/src/test/java/org/apache/dolphinscheduler/server/worker/utils/TaskExecutionContextUtilsTest.java
@@ -33,7 +33,6 @@ class TaskExecutionContextUtilsTest {
     void createTaskInstanceWorkingDirectory() throws IOException {
         TaskExecutionContext taskExecutionContext = new TaskExecutionContext();
         taskExecutionContext.setTenantCode("tenantCode");
-        taskExecutionContext.setProjectCode(1);
         taskExecutionContext.setWorkflowDefinitionCode(1L);
         taskExecutionContext.setWorkflowDefinitionVersion(1);
         taskExecutionContext.setWorkflowInstanceId(1);

Reply via email to