You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dolphinscheduler.apache.org by GitBox <gi...@apache.org> on 2022/06/30 10:46:41 UTC

[GitHub] [dolphinscheduler] WangJPLeo opened a new pull request, #10704: [Feature] Global parameter and local parameter calculation external expansion.

WangJPLeo opened a new pull request, #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704

   <!--Thanks very much for contributing to Apache DolphinScheduler. Please review https://dolphinscheduler.apache.org/en-us/community/development/pull-request.html before opening a pull request.-->
   
   
   ## Purpose of the pull request
   
   1. Add an external extension class to the service.
   2. The global parameters and local parameters are calculated and prepared in the master service.
   3. The local parameters in the worker service will be used directly.
   
   ## Brief change log
   
   close #10703 
   
   ## Verify this pull request
   
   Manually verified the change by testing locally.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] caishunfeng commented on a diff in pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
caishunfeng commented on code in PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#discussion_r910933942


##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DolphinSchedulerCuringGlobalParams.class);
+
+    @Autowired
+    private TimePlaceholderResolverExpandService timePlaceholderResolverExpandService;
+
+    @Override
+    public String convertParameterPlaceholders(String val, Map<String, String> allParamMap) {
+        return ParameterUtils.convertParameterPlaceholders(val, allParamMap);
+    }
+
+    @Override
+    public boolean timeFunctionNeedExpand(String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionNeedExpand(placeholderName);
+    }
+
+    @Override
+    public String timeFunctionExtension(Integer processInstanceId, String timezone, String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionExtension(processInstanceId, timezone, placeholderName);
+    }
+
+    @Override
+    public String curingGlobalParams(Integer processInstanceId, Map<String, String> globalParamMap, List<Property> globalParamList, CommandType commandType, Date scheduleTime, String timezone) {
+        if (globalParamList == null || globalParamList.isEmpty()) {
+            return null;
+        }
+        Map<String, String> globalMap = new HashMap<>();
+        if (globalParamMap != null) {
+            globalMap.putAll(globalParamMap);
+        }
+        Map<String, String> allParamMap = new HashMap<>();
+        //If it is a complement, a complement time needs to be passed in, according to the task type
+        Map<String, String> timeParams = BusinessTimeUtils.
+                getBusinessTime(commandType, scheduleTime, timezone);
+
+        if (timeParams != null) {
+            allParamMap.putAll(timeParams);
+        }
+        allParamMap.putAll(globalMap);
+        Set<Map.Entry<String, String>> entries = allParamMap.entrySet();
+        Map<String, String> resolveMap = new HashMap<>();
+        for (Map.Entry<String, String> entry : entries) {
+            String val = entry.getValue();
+            if (val.startsWith("$")) {
+                String str = "";
+                // whether external scaling calculation is required
+                if (timeFunctionNeedExpand(val)) {
+                    str = timeFunctionExtension(processInstanceId, timezone, val);
+                } else {
+                    str = convertParameterPlaceholders(val, allParamMap);
+                }
+                resolveMap.put(entry.getKey(), str);
+            }
+        }
+        globalMap.putAll(resolveMap);
+        for (Property property : globalParamList) {
+            String val = globalMap.get(property.getProp());
+            if (val != null) {
+                property.setValue(val);
+            }
+        }
+        return JSONUtils.toJsonString(globalParamList);
+    }
+
+    @Override
+    public Map<String, Property> paramParsingPreparation(TaskExecutionContext taskExecutionContext, AbstractParameters parameters, ProcessInstance processInstance) {
+        Preconditions.checkNotNull(taskExecutionContext);
+        Preconditions.checkNotNull(parameters);
+        setGlobalParamsMap(taskExecutionContext);
+        Map<String, Property> globalParams = ParamUtils.getUserDefParamsMap(taskExecutionContext.getDefinedParams());
+        Map<String,String> globalParamsMap = taskExecutionContext.getDefinedParams();
+        org.apache.dolphinscheduler.spi.enums.CommandType commandType = org.apache.dolphinscheduler.spi.enums.CommandType.of(taskExecutionContext.getCmdTypeIfComplement());

Review Comment:
   ```suggestion
           CommandType commandType = CommandType.of(taskExecutionContext.getCmdTypeIfComplement());
   ```



##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {

Review Comment:
   ```suggestion
   public class CuringGlobalParams implements CuringParamsService {
   ```



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/parser/ParamUtils.java:
##########
@@ -43,77 +33,81 @@ public class ParamUtils {
     /**
      * parameter conversion
      * Warning:
-     *  When you first invoke the function of convert, the variables of localParams and varPool in the ShellParameters will be modified.
+     *  1.When you first invoke the function of convert, the variables of localParams and varPool in the ShellParameters will be modified.
      *  But in the whole system the variables of localParams and varPool have been used in other functions. I'm not sure if this current
      *  situation is wrong. So I cannot modify the original logic.
      *
+     *  2.Change time: 2022-06-30
+     *  The purpose is for external expansion of local parameters.
+     *  now the method is replaced by the paramParsingPreparation() method of DolphinSchedulerCuringGlobalParams.
+     *
      * @param taskExecutionContext the context of this task instance
      * @param parameters the parameters
      * @return global params
      *
      */
-    public static Map<String, Property> convert(TaskExecutionContext taskExecutionContext, AbstractParameters parameters) {
-        Preconditions.checkNotNull(taskExecutionContext);
-        Preconditions.checkNotNull(parameters);
-        Map<String, Property> globalParams = getUserDefParamsMap(taskExecutionContext.getDefinedParams());
-        Map<String,String> globalParamsMap = taskExecutionContext.getDefinedParams();
-        CommandType commandType = CommandType.of(taskExecutionContext.getCmdTypeIfComplement());
-        Date scheduleTime = taskExecutionContext.getScheduleTime();
-
-        // combining local and global parameters
-        Map<String, Property> localParams = parameters.getInputLocalParametersMap();
-
-        //stream pass params
-        Map<String, Property> varParams = parameters.getVarPoolMap();
-
-        if (globalParams.size() == 0 && localParams.size() == 0 && varParams.size() == 0) {
-            return null;
-        }
-        // if it is a complement,
-        // you need to pass in the task instance id to locate the time
-        // of the process instance complement
-        Map<String,String> params = BusinessTimeUtils
-                .getBusinessTime(commandType,
-                        scheduleTime);
-
-        if (globalParamsMap != null) {
-
-            params.putAll(globalParamsMap);
-        }
-
-        if (StringUtils.isNotBlank(taskExecutionContext.getExecutePath())) {
-            params.put(PARAMETER_TASK_EXECUTE_PATH, taskExecutionContext.getExecutePath());
-        }
-        params.put(PARAMETER_TASK_INSTANCE_ID, Integer.toString(taskExecutionContext.getTaskInstanceId()));
-
-        if (varParams.size() != 0) {
-            globalParams.putAll(varParams);
-        }
-        if (localParams.size() != 0) {
-            globalParams.putAll(localParams);
-        }
-
-        Iterator<Map.Entry<String, Property>> iter = globalParams.entrySet().iterator();
-        while (iter.hasNext()) {
-            Map.Entry<String, Property> en = iter.next();
-            Property property = en.getValue();
-
-            if (StringUtils.isNotEmpty(property.getValue())
-                    && property.getValue().startsWith("$")) {
-                /**
-                 *  local parameter refers to global parameter with the same name
-                 *  note: the global parameters of the process instance here are solidified parameters,
-                 *  and there are no variables in them.
-                 */
-                String val = property.getValue();
-
-                val  = ParameterUtils.convertParameterPlaceholders(val, params);
-                property.setValue(val);
-            }
-        }
-
-        return globalParams;
-    }
+//    public static Map<String, Property> convert(TaskExecutionContext taskExecutionContext, AbstractParameters parameters) {

Review Comment:
   remove if no use.



##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DolphinSchedulerCuringGlobalParams.class);
+
+    @Autowired
+    private TimePlaceholderResolverExpandService timePlaceholderResolverExpandService;
+
+    @Override
+    public String convertParameterPlaceholders(String val, Map<String, String> allParamMap) {
+        return ParameterUtils.convertParameterPlaceholders(val, allParamMap);
+    }
+
+    @Override
+    public boolean timeFunctionNeedExpand(String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionNeedExpand(placeholderName);
+    }
+
+    @Override
+    public String timeFunctionExtension(Integer processInstanceId, String timezone, String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionExtension(processInstanceId, timezone, placeholderName);
+    }
+
+    @Override
+    public String curingGlobalParams(Integer processInstanceId, Map<String, String> globalParamMap, List<Property> globalParamList, CommandType commandType, Date scheduleTime, String timezone) {
+        if (globalParamList == null || globalParamList.isEmpty()) {
+            return null;
+        }
+        Map<String, String> globalMap = new HashMap<>();
+        if (globalParamMap != null) {
+            globalMap.putAll(globalParamMap);
+        }
+        Map<String, String> allParamMap = new HashMap<>();
+        //If it is a complement, a complement time needs to be passed in, according to the task type
+        Map<String, String> timeParams = BusinessTimeUtils.
+                getBusinessTime(commandType, scheduleTime, timezone);
+
+        if (timeParams != null) {
+            allParamMap.putAll(timeParams);
+        }
+        allParamMap.putAll(globalMap);
+        Set<Map.Entry<String, String>> entries = allParamMap.entrySet();
+        Map<String, String> resolveMap = new HashMap<>();
+        for (Map.Entry<String, String> entry : entries) {
+            String val = entry.getValue();
+            if (val.startsWith("$")) {
+                String str = "";
+                // whether external scaling calculation is required
+                if (timeFunctionNeedExpand(val)) {
+                    str = timeFunctionExtension(processInstanceId, timezone, val);
+                } else {
+                    str = convertParameterPlaceholders(val, allParamMap);
+                }
+                resolveMap.put(entry.getKey(), str);
+            }
+        }
+        globalMap.putAll(resolveMap);
+        for (Property property : globalParamList) {
+            String val = globalMap.get(property.getProp());
+            if (val != null) {
+                property.setValue(val);
+            }
+        }
+        return JSONUtils.toJsonString(globalParamList);
+    }
+
+    @Override
+    public Map<String, Property> paramParsingPreparation(TaskExecutionContext taskExecutionContext, AbstractParameters parameters, ProcessInstance processInstance) {
+        Preconditions.checkNotNull(taskExecutionContext);
+        Preconditions.checkNotNull(parameters);
+        setGlobalParamsMap(taskExecutionContext);
+        Map<String, Property> globalParams = ParamUtils.getUserDefParamsMap(taskExecutionContext.getDefinedParams());
+        Map<String,String> globalParamsMap = taskExecutionContext.getDefinedParams();
+        org.apache.dolphinscheduler.spi.enums.CommandType commandType = org.apache.dolphinscheduler.spi.enums.CommandType.of(taskExecutionContext.getCmdTypeIfComplement());
+        Date scheduleTime = taskExecutionContext.getScheduleTime();
+
+        // combining local and global parameters
+        Map<String, Property> localParams = parameters.getInputLocalParametersMap();
+
+        //stream pass params
+        Map<String, Property> varParams = parameters.getVarPoolMap();
+
+        if (globalParams.size() == 0 && localParams.size() == 0 && varParams.size() == 0) {
+            return null;
+        }
+        // if it is a complement,
+        // you need to pass in the task instance id to locate the time
+        // of the process instance complement
+        Map<String,String> params = org.apache.dolphinscheduler.plugin.task.api.parser.BusinessTimeUtils
+                .getBusinessTime(commandType,
+                        scheduleTime);
+
+        if (globalParamsMap != null) {
+            params.putAll(globalParamsMap);
+        }
+
+        if (StringUtils.isNotBlank(taskExecutionContext.getExecutePath())) {
+            params.put(PARAMETER_TASK_EXECUTE_PATH, taskExecutionContext.getExecutePath());
+        }
+        params.put(PARAMETER_TASK_INSTANCE_ID, Integer.toString(taskExecutionContext.getTaskInstanceId()));
+
+        if (varParams.size() != 0) {
+            globalParams.putAll(varParams);
+        }
+        if (localParams.size() != 0) {
+            globalParams.putAll(localParams);
+        }
+
+        Iterator<Map.Entry<String, Property>> iter = globalParams.entrySet().iterator();
+        while (iter.hasNext()) {
+            Map.Entry<String, Property> en = iter.next();
+            Property property = en.getValue();
+
+            if (StringUtils.isNotEmpty(property.getValue()) && property.getValue().startsWith("$")) {
+                /**
+                 *  local parameter refers to global parameter with the same name
+                 *  note: the global parameters of the process instance here are solidified parameters,
+                 *  and there are no variables in them.
+                 */
+                String val = property.getValue();
+                // whether external scaling calculation is required
+                if (timeFunctionNeedExpand(val)) {
+                    Map<String, String> cmdParam = JSONUtils.toMap(processInstance.getCommandParam());
+                    val = timeFunctionExtension(taskExecutionContext.getProcessInstanceId(), cmdParam.get(Constants.SCHEDULE_TIMEZONE), val);
+                } else {
+                    val  = org.apache.dolphinscheduler.plugin.task.api.parser.ParameterUtils.convertParameterPlaceholders(val, params);

Review Comment:
   ```suggestion
                       val  = ParameterUtils.convertParameterPlaceholders(val, params);
   ```



##########
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/parser/ParameterUtils.java:
##########
@@ -155,54 +151,59 @@ public static void setInParameter(int index, PreparedStatement stmt, DataType da
     /**
      * curing user define parameters
      *
+     * Warning:
+     *      Change time: 2022-06-30
+     *      The purpose is for external expansion of local parameters.
+     *      now the method is replaced by the curingGlobalParams() method of DolphinSchedulerCuringGlobalParams.
+     *
      * @param globalParamMap  global param map
      * @param globalParamList global param list
      * @param commandType     command type
      * @param scheduleTime    schedule time
      * @return curing user define parameters
      */
-    public static String curingGlobalParams(Map<String, String> globalParamMap, List<Property> globalParamList,
-                                            CommandType commandType, Date scheduleTime) {
-
-        if (globalParamList == null || globalParamList.isEmpty()) {
-            return null;
-        }
-
-        Map<String, String> globalMap = new HashMap<>();
-        if (globalParamMap != null) {
-            globalMap.putAll(globalParamMap);
-        }
-        Map<String, String> allParamMap = new HashMap<>();
-        //If it is a complement, a complement time needs to be passed in, according to the task type
-        Map<String, String> timeParams = BusinessTimeUtils
-            .getBusinessTime(commandType, scheduleTime);
-
-        if (timeParams != null) {
-            allParamMap.putAll(timeParams);
-        }
-
-        allParamMap.putAll(globalMap);
-
-        Set<Map.Entry<String, String>> entries = allParamMap.entrySet();
-
-        Map<String, String> resolveMap = new HashMap<>();
-        for (Map.Entry<String, String> entry : entries) {
-            String val = entry.getValue();
-            if (val.startsWith("$")) {
-                String str = ParameterUtils.convertParameterPlaceholders(val, allParamMap);
-                resolveMap.put(entry.getKey(), str);
-            }
-        }
-        globalMap.putAll(resolveMap);
-
-        for (Property property : globalParamList) {
-            String val = globalMap.get(property.getProp());
-            if (val != null) {
-                property.setValue(val);
-            }
-        }
-        return JSONUtils.toJsonString(globalParamList);
-    }
+//    public static String curingGlobalParams(Map<String, String> globalParamMap, List<Property> globalParamList,

Review Comment:
   same here.



##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DolphinSchedulerCuringGlobalParams.class);
+
+    @Autowired
+    private TimePlaceholderResolverExpandService timePlaceholderResolverExpandService;
+
+    @Override
+    public String convertParameterPlaceholders(String val, Map<String, String> allParamMap) {
+        return ParameterUtils.convertParameterPlaceholders(val, allParamMap);
+    }
+
+    @Override
+    public boolean timeFunctionNeedExpand(String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionNeedExpand(placeholderName);
+    }
+
+    @Override
+    public String timeFunctionExtension(Integer processInstanceId, String timezone, String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionExtension(processInstanceId, timezone, placeholderName);
+    }
+
+    @Override
+    public String curingGlobalParams(Integer processInstanceId, Map<String, String> globalParamMap, List<Property> globalParamList, CommandType commandType, Date scheduleTime, String timezone) {
+        if (globalParamList == null || globalParamList.isEmpty()) {
+            return null;
+        }
+        Map<String, String> globalMap = new HashMap<>();
+        if (globalParamMap != null) {
+            globalMap.putAll(globalParamMap);
+        }
+        Map<String, String> allParamMap = new HashMap<>();
+        //If it is a complement, a complement time needs to be passed in, according to the task type
+        Map<String, String> timeParams = BusinessTimeUtils.
+                getBusinessTime(commandType, scheduleTime, timezone);
+
+        if (timeParams != null) {
+            allParamMap.putAll(timeParams);
+        }
+        allParamMap.putAll(globalMap);
+        Set<Map.Entry<String, String>> entries = allParamMap.entrySet();
+        Map<String, String> resolveMap = new HashMap<>();
+        for (Map.Entry<String, String> entry : entries) {
+            String val = entry.getValue();
+            if (val.startsWith("$")) {
+                String str = "";
+                // whether external scaling calculation is required
+                if (timeFunctionNeedExpand(val)) {
+                    str = timeFunctionExtension(processInstanceId, timezone, val);
+                } else {
+                    str = convertParameterPlaceholders(val, allParamMap);
+                }
+                resolveMap.put(entry.getKey(), str);
+            }
+        }
+        globalMap.putAll(resolveMap);
+        for (Property property : globalParamList) {
+            String val = globalMap.get(property.getProp());
+            if (val != null) {
+                property.setValue(val);
+            }
+        }
+        return JSONUtils.toJsonString(globalParamList);
+    }
+
+    @Override
+    public Map<String, Property> paramParsingPreparation(TaskExecutionContext taskExecutionContext, AbstractParameters parameters, ProcessInstance processInstance) {
+        Preconditions.checkNotNull(taskExecutionContext);
+        Preconditions.checkNotNull(parameters);
+        setGlobalParamsMap(taskExecutionContext);
+        Map<String, Property> globalParams = ParamUtils.getUserDefParamsMap(taskExecutionContext.getDefinedParams());
+        Map<String,String> globalParamsMap = taskExecutionContext.getDefinedParams();
+        org.apache.dolphinscheduler.spi.enums.CommandType commandType = org.apache.dolphinscheduler.spi.enums.CommandType.of(taskExecutionContext.getCmdTypeIfComplement());
+        Date scheduleTime = taskExecutionContext.getScheduleTime();
+
+        // combining local and global parameters
+        Map<String, Property> localParams = parameters.getInputLocalParametersMap();
+
+        //stream pass params
+        Map<String, Property> varParams = parameters.getVarPoolMap();
+
+        if (globalParams.size() == 0 && localParams.size() == 0 && varParams.size() == 0) {
+            return null;
+        }
+        // if it is a complement,
+        // you need to pass in the task instance id to locate the time
+        // of the process instance complement
+        Map<String,String> params = org.apache.dolphinscheduler.plugin.task.api.parser.BusinessTimeUtils
+                .getBusinessTime(commandType,
+                        scheduleTime);
+
+        if (globalParamsMap != null) {
+            params.putAll(globalParamsMap);
+        }
+
+        if (StringUtils.isNotBlank(taskExecutionContext.getExecutePath())) {
+            params.put(PARAMETER_TASK_EXECUTE_PATH, taskExecutionContext.getExecutePath());
+        }
+        params.put(PARAMETER_TASK_INSTANCE_ID, Integer.toString(taskExecutionContext.getTaskInstanceId()));
+
+        if (varParams.size() != 0) {
+            globalParams.putAll(varParams);
+        }
+        if (localParams.size() != 0) {
+            globalParams.putAll(localParams);
+        }
+
+        Iterator<Map.Entry<String, Property>> iter = globalParams.entrySet().iterator();
+        while (iter.hasNext()) {
+            Map.Entry<String, Property> en = iter.next();
+            Property property = en.getValue();
+
+            if (StringUtils.isNotEmpty(property.getValue()) && property.getValue().startsWith("$")) {

Review Comment:
   same here



##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DolphinSchedulerCuringGlobalParams.class);
+
+    @Autowired
+    private TimePlaceholderResolverExpandService timePlaceholderResolverExpandService;
+
+    @Override
+    public String convertParameterPlaceholders(String val, Map<String, String> allParamMap) {
+        return ParameterUtils.convertParameterPlaceholders(val, allParamMap);
+    }
+
+    @Override
+    public boolean timeFunctionNeedExpand(String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionNeedExpand(placeholderName);
+    }
+
+    @Override
+    public String timeFunctionExtension(Integer processInstanceId, String timezone, String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionExtension(processInstanceId, timezone, placeholderName);
+    }
+
+    @Override
+    public String curingGlobalParams(Integer processInstanceId, Map<String, String> globalParamMap, List<Property> globalParamList, CommandType commandType, Date scheduleTime, String timezone) {
+        if (globalParamList == null || globalParamList.isEmpty()) {
+            return null;
+        }
+        Map<String, String> globalMap = new HashMap<>();
+        if (globalParamMap != null) {
+            globalMap.putAll(globalParamMap);
+        }
+        Map<String, String> allParamMap = new HashMap<>();
+        //If it is a complement, a complement time needs to be passed in, according to the task type
+        Map<String, String> timeParams = BusinessTimeUtils.
+                getBusinessTime(commandType, scheduleTime, timezone);
+
+        if (timeParams != null) {
+            allParamMap.putAll(timeParams);
+        }
+        allParamMap.putAll(globalMap);
+        Set<Map.Entry<String, String>> entries = allParamMap.entrySet();
+        Map<String, String> resolveMap = new HashMap<>();
+        for (Map.Entry<String, String> entry : entries) {
+            String val = entry.getValue();
+            if (val.startsWith("$")) {
+                String str = "";
+                // whether external scaling calculation is required
+                if (timeFunctionNeedExpand(val)) {
+                    str = timeFunctionExtension(processInstanceId, timezone, val);
+                } else {
+                    str = convertParameterPlaceholders(val, allParamMap);
+                }
+                resolveMap.put(entry.getKey(), str);
+            }
+        }
+        globalMap.putAll(resolveMap);
+        for (Property property : globalParamList) {
+            String val = globalMap.get(property.getProp());
+            if (val != null) {
+                property.setValue(val);
+            }
+        }
+        return JSONUtils.toJsonString(globalParamList);
+    }
+
+    @Override
+    public Map<String, Property> paramParsingPreparation(TaskExecutionContext taskExecutionContext, AbstractParameters parameters, ProcessInstance processInstance) {
+        Preconditions.checkNotNull(taskExecutionContext);
+        Preconditions.checkNotNull(parameters);
+        setGlobalParamsMap(taskExecutionContext);
+        Map<String, Property> globalParams = ParamUtils.getUserDefParamsMap(taskExecutionContext.getDefinedParams());
+        Map<String,String> globalParamsMap = taskExecutionContext.getDefinedParams();
+        org.apache.dolphinscheduler.spi.enums.CommandType commandType = org.apache.dolphinscheduler.spi.enums.CommandType.of(taskExecutionContext.getCmdTypeIfComplement());
+        Date scheduleTime = taskExecutionContext.getScheduleTime();
+
+        // combining local and global parameters
+        Map<String, Property> localParams = parameters.getInputLocalParametersMap();
+
+        //stream pass params
+        Map<String, Property> varParams = parameters.getVarPoolMap();
+
+        if (globalParams.size() == 0 && localParams.size() == 0 && varParams.size() == 0) {
+            return null;
+        }
+        // if it is a complement,
+        // you need to pass in the task instance id to locate the time
+        // of the process instance complement
+        Map<String,String> params = org.apache.dolphinscheduler.plugin.task.api.parser.BusinessTimeUtils
+                .getBusinessTime(commandType,
+                        scheduleTime);
+
+        if (globalParamsMap != null) {
+            params.putAll(globalParamsMap);
+        }
+
+        if (StringUtils.isNotBlank(taskExecutionContext.getExecutePath())) {
+            params.put(PARAMETER_TASK_EXECUTE_PATH, taskExecutionContext.getExecutePath());
+        }
+        params.put(PARAMETER_TASK_INSTANCE_ID, Integer.toString(taskExecutionContext.getTaskInstanceId()));
+
+        if (varParams.size() != 0) {
+            globalParams.putAll(varParams);
+        }
+        if (localParams.size() != 0) {
+            globalParams.putAll(localParams);
+        }
+
+        Iterator<Map.Entry<String, Property>> iter = globalParams.entrySet().iterator();
+        while (iter.hasNext()) {
+            Map.Entry<String, Property> en = iter.next();
+            Property property = en.getValue();
+
+            if (StringUtils.isNotEmpty(property.getValue()) && property.getValue().startsWith("$")) {
+                /**
+                 *  local parameter refers to global parameter with the same name
+                 *  note: the global parameters of the process instance here are solidified parameters,
+                 *  and there are no variables in them.
+                 */
+                String val = property.getValue();
+                // whether external scaling calculation is required
+                if (timeFunctionNeedExpand(val)) {
+                    Map<String, String> cmdParam = JSONUtils.toMap(processInstance.getCommandParam());
+                    val = timeFunctionExtension(taskExecutionContext.getProcessInstanceId(), cmdParam.get(Constants.SCHEDULE_TIMEZONE), val);
+                } else {
+                    val  = org.apache.dolphinscheduler.plugin.task.api.parser.ParameterUtils.convertParameterPlaceholders(val, params);
+                }
+                property.setValue(val);
+            }
+        }
+        if (org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils.isEmpty(globalParams)) {

Review Comment:
   ```suggestion
           if (MapUtils.isEmpty(globalParams)) {
   ```



##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DolphinSchedulerCuringGlobalParams.class);
+
+    @Autowired
+    private TimePlaceholderResolverExpandService timePlaceholderResolverExpandService;
+
+    @Override
+    public String convertParameterPlaceholders(String val, Map<String, String> allParamMap) {
+        return ParameterUtils.convertParameterPlaceholders(val, allParamMap);
+    }
+
+    @Override
+    public boolean timeFunctionNeedExpand(String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionNeedExpand(placeholderName);
+    }
+
+    @Override
+    public String timeFunctionExtension(Integer processInstanceId, String timezone, String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionExtension(processInstanceId, timezone, placeholderName);
+    }
+
+    @Override
+    public String curingGlobalParams(Integer processInstanceId, Map<String, String> globalParamMap, List<Property> globalParamList, CommandType commandType, Date scheduleTime, String timezone) {
+        if (globalParamList == null || globalParamList.isEmpty()) {
+            return null;
+        }
+        Map<String, String> globalMap = new HashMap<>();
+        if (globalParamMap != null) {
+            globalMap.putAll(globalParamMap);
+        }
+        Map<String, String> allParamMap = new HashMap<>();
+        //If it is a complement, a complement time needs to be passed in, according to the task type
+        Map<String, String> timeParams = BusinessTimeUtils.
+                getBusinessTime(commandType, scheduleTime, timezone);
+
+        if (timeParams != null) {
+            allParamMap.putAll(timeParams);
+        }
+        allParamMap.putAll(globalMap);
+        Set<Map.Entry<String, String>> entries = allParamMap.entrySet();
+        Map<String, String> resolveMap = new HashMap<>();
+        for (Map.Entry<String, String> entry : entries) {
+            String val = entry.getValue();
+            if (val.startsWith("$")) {
+                String str = "";
+                // whether external scaling calculation is required
+                if (timeFunctionNeedExpand(val)) {
+                    str = timeFunctionExtension(processInstanceId, timezone, val);
+                } else {
+                    str = convertParameterPlaceholders(val, allParamMap);
+                }
+                resolveMap.put(entry.getKey(), str);
+            }
+        }
+        globalMap.putAll(resolveMap);
+        for (Property property : globalParamList) {
+            String val = globalMap.get(property.getProp());
+            if (val != null) {
+                property.setValue(val);
+            }
+        }
+        return JSONUtils.toJsonString(globalParamList);
+    }
+
+    @Override
+    public Map<String, Property> paramParsingPreparation(TaskExecutionContext taskExecutionContext, AbstractParameters parameters, ProcessInstance processInstance) {

Review Comment:
   please add some comments.



##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DolphinSchedulerCuringGlobalParams.class);
+
+    @Autowired
+    private TimePlaceholderResolverExpandService timePlaceholderResolverExpandService;
+
+    @Override
+    public String convertParameterPlaceholders(String val, Map<String, String> allParamMap) {
+        return ParameterUtils.convertParameterPlaceholders(val, allParamMap);
+    }
+
+    @Override
+    public boolean timeFunctionNeedExpand(String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionNeedExpand(placeholderName);
+    }
+
+    @Override
+    public String timeFunctionExtension(Integer processInstanceId, String timezone, String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionExtension(processInstanceId, timezone, placeholderName);
+    }
+
+    @Override
+    public String curingGlobalParams(Integer processInstanceId, Map<String, String> globalParamMap, List<Property> globalParamList, CommandType commandType, Date scheduleTime, String timezone) {
+        if (globalParamList == null || globalParamList.isEmpty()) {
+            return null;
+        }
+        Map<String, String> globalMap = new HashMap<>();
+        if (globalParamMap != null) {
+            globalMap.putAll(globalParamMap);
+        }
+        Map<String, String> allParamMap = new HashMap<>();
+        //If it is a complement, a complement time needs to be passed in, according to the task type
+        Map<String, String> timeParams = BusinessTimeUtils.
+                getBusinessTime(commandType, scheduleTime, timezone);
+
+        if (timeParams != null) {
+            allParamMap.putAll(timeParams);
+        }
+        allParamMap.putAll(globalMap);
+        Set<Map.Entry<String, String>> entries = allParamMap.entrySet();
+        Map<String, String> resolveMap = new HashMap<>();
+        for (Map.Entry<String, String> entry : entries) {
+            String val = entry.getValue();
+            if (val.startsWith("$")) {

Review Comment:
   use constant



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#issuecomment-1171189866

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10704)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [30 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL)
   
   [![38.3%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/25-16px.png '38.3%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list) [38.3% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] WangJPLeo commented on a diff in pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
WangJPLeo commented on code in PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#discussion_r910964933


##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DolphinSchedulerCuringGlobalParams.class);
+
+    @Autowired
+    private TimePlaceholderResolverExpandService timePlaceholderResolverExpandService;
+
+    @Override
+    public String convertParameterPlaceholders(String val, Map<String, String> allParamMap) {
+        return ParameterUtils.convertParameterPlaceholders(val, allParamMap);
+    }
+
+    @Override
+    public boolean timeFunctionNeedExpand(String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionNeedExpand(placeholderName);
+    }
+
+    @Override
+    public String timeFunctionExtension(Integer processInstanceId, String timezone, String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionExtension(processInstanceId, timezone, placeholderName);
+    }
+
+    @Override
+    public String curingGlobalParams(Integer processInstanceId, Map<String, String> globalParamMap, List<Property> globalParamList, CommandType commandType, Date scheduleTime, String timezone) {
+        if (globalParamList == null || globalParamList.isEmpty()) {
+            return null;
+        }
+        Map<String, String> globalMap = new HashMap<>();
+        if (globalParamMap != null) {
+            globalMap.putAll(globalParamMap);
+        }
+        Map<String, String> allParamMap = new HashMap<>();
+        //If it is a complement, a complement time needs to be passed in, according to the task type
+        Map<String, String> timeParams = BusinessTimeUtils.
+                getBusinessTime(commandType, scheduleTime, timezone);
+
+        if (timeParams != null) {
+            allParamMap.putAll(timeParams);
+        }
+        allParamMap.putAll(globalMap);
+        Set<Map.Entry<String, String>> entries = allParamMap.entrySet();
+        Map<String, String> resolveMap = new HashMap<>();
+        for (Map.Entry<String, String> entry : entries) {
+            String val = entry.getValue();
+            if (val.startsWith("$")) {
+                String str = "";
+                // whether external scaling calculation is required
+                if (timeFunctionNeedExpand(val)) {
+                    str = timeFunctionExtension(processInstanceId, timezone, val);
+                } else {
+                    str = convertParameterPlaceholders(val, allParamMap);
+                }
+                resolveMap.put(entry.getKey(), str);
+            }
+        }
+        globalMap.putAll(resolveMap);
+        for (Property property : globalParamList) {
+            String val = globalMap.get(property.getProp());
+            if (val != null) {
+                property.setValue(val);
+            }
+        }
+        return JSONUtils.toJsonString(globalParamList);
+    }
+
+    @Override
+    public Map<String, Property> paramParsingPreparation(TaskExecutionContext taskExecutionContext, AbstractParameters parameters, ProcessInstance processInstance) {
+        Preconditions.checkNotNull(taskExecutionContext);
+        Preconditions.checkNotNull(parameters);
+        setGlobalParamsMap(taskExecutionContext);
+        Map<String, Property> globalParams = ParamUtils.getUserDefParamsMap(taskExecutionContext.getDefinedParams());
+        Map<String,String> globalParamsMap = taskExecutionContext.getDefinedParams();
+        org.apache.dolphinscheduler.spi.enums.CommandType commandType = org.apache.dolphinscheduler.spi.enums.CommandType.of(taskExecutionContext.getCmdTypeIfComplement());

Review Comment:
   Proven to be replaceable.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#issuecomment-1171258921

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10704)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [38 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL)
   
   [![34.3%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/25-16px.png '34.3%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list) [34.3% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#issuecomment-1171282801

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10704)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [34 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL)
   
   [![33.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/25-16px.png '33.9%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list) [33.9% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] ruanwenjun commented on a diff in pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
ruanwenjun commented on code in PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#discussion_r910964971


##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DolphinSchedulerCuringGlobalParams.class);
+
+    @Autowired
+    private TimePlaceholderResolverExpandService timePlaceholderResolverExpandService;
+
+    @Override
+    public String convertParameterPlaceholders(String val, Map<String, String> allParamMap) {
+        return ParameterUtils.convertParameterPlaceholders(val, allParamMap);
+    }
+
+    @Override
+    public boolean timeFunctionNeedExpand(String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionNeedExpand(placeholderName);
+    }
+
+    @Override
+    public String timeFunctionExtension(Integer processInstanceId, String timezone, String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionExtension(processInstanceId, timezone, placeholderName);
+    }
+
+    @Override
+    public String curingGlobalParams(Integer processInstanceId, Map<String, String> globalParamMap, List<Property> globalParamList, CommandType commandType, Date scheduleTime, String timezone) {
+        if (globalParamList == null || globalParamList.isEmpty()) {
+            return null;
+        }
+        Map<String, String> globalMap = new HashMap<>();
+        if (globalParamMap != null) {
+            globalMap.putAll(globalParamMap);
+        }
+        Map<String, String> allParamMap = new HashMap<>();
+        //If it is a complement, a complement time needs to be passed in, according to the task type
+        Map<String, String> timeParams = BusinessTimeUtils.
+                getBusinessTime(commandType, scheduleTime, timezone);
+
+        if (timeParams != null) {
+            allParamMap.putAll(timeParams);
+        }
+        allParamMap.putAll(globalMap);
+        Set<Map.Entry<String, String>> entries = allParamMap.entrySet();
+        Map<String, String> resolveMap = new HashMap<>();
+        for (Map.Entry<String, String> entry : entries) {
+            String val = entry.getValue();
+            if (val.startsWith("$")) {
+                String str = "";
+                // whether external scaling calculation is required
+                if (timeFunctionNeedExpand(val)) {
+                    str = timeFunctionExtension(processInstanceId, timezone, val);
+                } else {
+                    str = convertParameterPlaceholders(val, allParamMap);
+                }
+                resolveMap.put(entry.getKey(), str);
+            }
+        }
+        globalMap.putAll(resolveMap);
+        for (Property property : globalParamList) {
+            String val = globalMap.get(property.getProp());
+            if (val != null) {
+                property.setValue(val);
+            }
+        }
+        return JSONUtils.toJsonString(globalParamList);
+    }
+
+    @Override
+    public Map<String, Property> paramParsingPreparation(TaskExecutionContext taskExecutionContext, AbstractParameters parameters, ProcessInstance processInstance) {

Review Comment:
   I hope we can add comments and example in the interface, unless the interface is easy to know.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#issuecomment-1171258833

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10704)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [38 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL)
   
   [![34.3%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/25-16px.png '34.3%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list) [34.3% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#issuecomment-1171282770

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10704)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [34 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL)
   
   [![33.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/25-16px.png '33.9%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list) [33.9% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#issuecomment-1171151672

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10704)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [32 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL)
   
   [![38.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/25-16px.png '38.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list) [38.1% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] codecov-commenter commented on pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#issuecomment-1171144494

   # [Codecov](https://codecov.io/gh/apache/dolphinscheduler/pull/10704?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10704](https://codecov.io/gh/apache/dolphinscheduler/pull/10704?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5742d0f) into [dev](https://codecov.io/gh/apache/dolphinscheduler/commit/a90e5746fc3d6103c00ad39715f58ed54eb1700f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a90e574) will **decrease** coverage by `0.06%`.
   > The diff coverage is `37.62%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##                dev   #10704      +/-   ##
   ============================================
   - Coverage     40.89%   40.83%   -0.07%     
   + Complexity     4830     4826       -4     
   ============================================
     Files           897      897              
     Lines         36157    36129      -28     
     Branches       4006     3987      -19     
   ============================================
   - Hits          14788    14754      -34     
   - Misses        19898    19909      +11     
   + Partials       1471     1466       -5     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/dolphinscheduler/pull/10704?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...r/api/service/impl/ProcessInstanceServiceImpl.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10704/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL3NlcnZpY2UvaW1wbC9Qcm9jZXNzSW5zdGFuY2VTZXJ2aWNlSW1wbC5qYXZh) | `60.10% <ø> (ø)` | |
   | [...er/server/builder/TaskExecutionContextBuilder.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10704/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1tYXN0ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvc2VydmVyL2J1aWxkZXIvVGFza0V4ZWN1dGlvbkNvbnRleHRCdWlsZGVyLmphdmE=) | `66.00% <0.00%> (-4.22%)` | :arrow_down: |
   | [...r/server/master/runner/MasterSchedulerService.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10704/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1tYXN0ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvc2VydmVyL21hc3Rlci9ydW5uZXIvTWFzdGVyU2NoZWR1bGVyU2VydmljZS5qYXZh) | `0.00% <ø> (ø)` | |
   | [...pand/TimePlaceholderResolverExpandServiceImpl.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10704/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1zZXJ2aWNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9kb2xwaGluc2NoZWR1bGVyL3NlcnZpY2UvZXhwYW5kL1RpbWVQbGFjZWhvbGRlclJlc29sdmVyRXhwYW5kU2VydmljZUltcGwuamF2YQ==) | `100.00% <ø> (ø)` | |
   | [...nscheduler/service/process/ProcessServiceImpl.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10704/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1zZXJ2aWNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9kb2xwaGluc2NoZWR1bGVyL3NlcnZpY2UvcHJvY2Vzcy9Qcm9jZXNzU2VydmljZUltcGwuamF2YQ==) | `30.28% <ø> (ø)` | |
   | [...inscheduler/plugin/task/api/parser/ParamUtils.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10704/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci10YXNrLXBsdWdpbi9kb2xwaGluc2NoZWR1bGVyLXRhc2stYXBpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9kb2xwaGluc2NoZWR1bGVyL3BsdWdpbi90YXNrL2FwaS9wYXJzZXIvUGFyYW1VdGlscy5qYXZh) | `0.00% <ø> (ø)` | |
   | [...heduler/plugin/task/api/parser/ParameterUtils.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10704/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci10YXNrLXBsdWdpbi9kb2xwaGluc2NoZWR1bGVyLXRhc2stYXBpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9kb2xwaGluc2NoZWR1bGVyL3BsdWdpbi90YXNrL2FwaS9wYXJzZXIvUGFyYW1ldGVyVXRpbHMuamF2YQ==) | `0.00% <ø> (ø)` | |
   | [...lphinscheduler/plugin/task/dq/DataQualityTask.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10704/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci10YXNrLXBsdWdpbi9kb2xwaGluc2NoZWR1bGVyLXRhc2stZGF0YXF1YWxpdHkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvcGx1Z2luL3Rhc2svZHEvRGF0YVF1YWxpdHlUYXNrLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../dolphinscheduler/plugin/task/datax/DataxTask.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10704/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci10YXNrLXBsdWdpbi9kb2xwaGluc2NoZWR1bGVyLXRhc2stZGF0YXgvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvcGx1Z2luL3Rhc2svZGF0YXgvRGF0YXhUYXNrLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../dolphinscheduler/plugin/task/flink/FlinkTask.java](https://codecov.io/gh/apache/dolphinscheduler/pull/10704/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci10YXNrLXBsdWdpbi9kb2xwaGluc2NoZWR1bGVyLXRhc2stZmxpbmsvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvcGx1Z2luL3Rhc2svZmxpbmsvRmxpbmtUYXNrLmphdmE=) | `66.25% <0.00%> (+1.58%)` | :arrow_up: |
   | ... and [26 more](https://codecov.io/gh/apache/dolphinscheduler/pull/10704/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/dolphinscheduler/pull/10704?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/dolphinscheduler/pull/10704?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [a90e574...5742d0f](https://codecov.io/gh/apache/dolphinscheduler/pull/10704?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] lenboo commented on a diff in pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
lenboo commented on code in PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#discussion_r910945103


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/main/java/org/apache/dolphinscheduler/plugin/task/api/parser/ParamUtils.java:
##########
@@ -43,77 +33,81 @@ public class ParamUtils {
     /**
      * parameter conversion
      * Warning:
-     *  When you first invoke the function of convert, the variables of localParams and varPool in the ShellParameters will be modified.
+     *  1.When you first invoke the function of convert, the variables of localParams and varPool in the ShellParameters will be modified.
      *  But in the whole system the variables of localParams and varPool have been used in other functions. I'm not sure if this current
      *  situation is wrong. So I cannot modify the original logic.
      *
+     *  2.Change time: 2022-06-30
+     *  The purpose is for external expansion of local parameters.
+     *  now the method is replaced by the paramParsingPreparation() method of DolphinSchedulerCuringGlobalParams.
+     *
      * @param taskExecutionContext the context of this task instance
      * @param parameters the parameters
      * @return global params
      *
      */
-    public static Map<String, Property> convert(TaskExecutionContext taskExecutionContext, AbstractParameters parameters) {
-        Preconditions.checkNotNull(taskExecutionContext);
-        Preconditions.checkNotNull(parameters);
-        Map<String, Property> globalParams = getUserDefParamsMap(taskExecutionContext.getDefinedParams());
-        Map<String,String> globalParamsMap = taskExecutionContext.getDefinedParams();
-        CommandType commandType = CommandType.of(taskExecutionContext.getCmdTypeIfComplement());
-        Date scheduleTime = taskExecutionContext.getScheduleTime();
-
-        // combining local and global parameters
-        Map<String, Property> localParams = parameters.getInputLocalParametersMap();
-
-        //stream pass params
-        Map<String, Property> varParams = parameters.getVarPoolMap();
-
-        if (globalParams.size() == 0 && localParams.size() == 0 && varParams.size() == 0) {
-            return null;
-        }
-        // if it is a complement,
-        // you need to pass in the task instance id to locate the time
-        // of the process instance complement
-        Map<String,String> params = BusinessTimeUtils
-                .getBusinessTime(commandType,
-                        scheduleTime);
-
-        if (globalParamsMap != null) {
-
-            params.putAll(globalParamsMap);
-        }
-
-        if (StringUtils.isNotBlank(taskExecutionContext.getExecutePath())) {
-            params.put(PARAMETER_TASK_EXECUTE_PATH, taskExecutionContext.getExecutePath());
-        }
-        params.put(PARAMETER_TASK_INSTANCE_ID, Integer.toString(taskExecutionContext.getTaskInstanceId()));
-
-        if (varParams.size() != 0) {
-            globalParams.putAll(varParams);
-        }
-        if (localParams.size() != 0) {
-            globalParams.putAll(localParams);
-        }
-
-        Iterator<Map.Entry<String, Property>> iter = globalParams.entrySet().iterator();
-        while (iter.hasNext()) {
-            Map.Entry<String, Property> en = iter.next();
-            Property property = en.getValue();
-
-            if (StringUtils.isNotEmpty(property.getValue())
-                    && property.getValue().startsWith("$")) {
-                /**
-                 *  local parameter refers to global parameter with the same name
-                 *  note: the global parameters of the process instance here are solidified parameters,
-                 *  and there are no variables in them.
-                 */
-                String val = property.getValue();
-
-                val  = ParameterUtils.convertParameterPlaceholders(val, params);
-                property.setValue(val);
-            }
-        }
-
-        return globalParams;
-    }
+//    public static Map<String, Property> convert(TaskExecutionContext taskExecutionContext, AbstractParameters parameters) {

Review Comment:
   You can delete unused code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] ruanwenjun commented on a diff in pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
ruanwenjun commented on code in PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#discussion_r910979371


##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DolphinSchedulerCuringGlobalParams.class);
+
+    @Autowired
+    private TimePlaceholderResolverExpandService timePlaceholderResolverExpandService;
+
+    @Override
+    public String convertParameterPlaceholders(String val, Map<String, String> allParamMap) {
+        return ParameterUtils.convertParameterPlaceholders(val, allParamMap);
+    }
+
+    @Override
+    public boolean timeFunctionNeedExpand(String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionNeedExpand(placeholderName);
+    }
+
+    @Override
+    public String timeFunctionExtension(Integer processInstanceId, String timezone, String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionExtension(processInstanceId, timezone, placeholderName);
+    }
+
+    @Override
+    public String curingGlobalParams(Integer processInstanceId, Map<String, String> globalParamMap, List<Property> globalParamList, CommandType commandType, Date scheduleTime, String timezone) {
+        if (globalParamList == null || globalParamList.isEmpty()) {
+            return null;
+        }
+        Map<String, String> globalMap = new HashMap<>();
+        if (globalParamMap != null) {
+            globalMap.putAll(globalParamMap);
+        }
+        Map<String, String> allParamMap = new HashMap<>();
+        //If it is a complement, a complement time needs to be passed in, according to the task type
+        Map<String, String> timeParams = BusinessTimeUtils.
+                getBusinessTime(commandType, scheduleTime, timezone);
+
+        if (timeParams != null) {
+            allParamMap.putAll(timeParams);
+        }
+        allParamMap.putAll(globalMap);
+        Set<Map.Entry<String, String>> entries = allParamMap.entrySet();
+        Map<String, String> resolveMap = new HashMap<>();
+        for (Map.Entry<String, String> entry : entries) {
+            String val = entry.getValue();
+            if (val.startsWith("$")) {
+                String str = "";
+                // whether external scaling calculation is required
+                if (timeFunctionNeedExpand(val)) {
+                    str = timeFunctionExtension(processInstanceId, timezone, val);
+                } else {
+                    str = convertParameterPlaceholders(val, allParamMap);
+                }
+                resolveMap.put(entry.getKey(), str);
+            }
+        }
+        globalMap.putAll(resolveMap);
+        for (Property property : globalParamList) {
+            String val = globalMap.get(property.getProp());
+            if (val != null) {
+                property.setValue(val);
+            }
+        }
+        return JSONUtils.toJsonString(globalParamList);
+    }
+
+    @Override
+    public Map<String, Property> paramParsingPreparation(TaskExecutionContext taskExecutionContext, AbstractParameters parameters, ProcessInstance processInstance) {
+        Preconditions.checkNotNull(taskExecutionContext);
+        Preconditions.checkNotNull(parameters);
+        setGlobalParamsMap(taskExecutionContext);

Review Comment:
   So this method will affect `taskExecutionContext`, I think this should be note in the comment.



##########
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java:
##########
@@ -255,7 +254,7 @@ public WorkflowExecuteRunnable(ProcessInstance processInstance
             , ProcessAlertManager processAlertManager
             , MasterConfig masterConfig
             , StateWheelExecuteThread stateWheelExecuteThread
-            , CuringGlobalParamsService curingGlobalParamsService) {
+            , CuringParamsService curingGlobalParamsService) {

Review Comment:
   ```suggestion
               , CuringParamsService curingParamsService) {
   ```



##########
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/builder/TaskExecutionContextBuilder.java:
##########
@@ -131,6 +136,13 @@ public TaskExecutionContextBuilder buildK8sTaskRelatedInfo(K8sTaskExecutionConte
         taskExecutionContext.setK8sTaskExecutionContext(k8sTaskExecutionContext);
         return this;
     }
+
+    public TaskExecutionContextBuilder buildParamInfo(CuringParamsService curingParamsService, AbstractParameters baseParam, ProcessInstance processInstance) {
+        Map<String, Property> propertyMap = curingParamsService.paramParsingPreparation(taskExecutionContext, baseParam, processInstance);
+        taskExecutionContext.setPrepareParamsMap(propertyMap);
+        return this;
+    }

Review Comment:
   It's better to calculate the param in the upper layer.
   ```suggestion
       public TaskExecutionContextBuilder buildParamInfo(Map<String, Property> propertyMap) {
           taskExecutionContext.setPrepareParamsMap(propertyMap);
           return this;
       }
   ```



##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DolphinSchedulerCuringGlobalParams.class);
+
+    @Autowired
+    private TimePlaceholderResolverExpandService timePlaceholderResolverExpandService;
+
+    @Override
+    public String convertParameterPlaceholders(String val, Map<String, String> allParamMap) {
+        return ParameterUtils.convertParameterPlaceholders(val, allParamMap);
+    }
+
+    @Override
+    public boolean timeFunctionNeedExpand(String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionNeedExpand(placeholderName);
+    }
+
+    @Override
+    public String timeFunctionExtension(Integer processInstanceId, String timezone, String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionExtension(processInstanceId, timezone, placeholderName);
+    }
+
+    @Override
+    public String curingGlobalParams(Integer processInstanceId, Map<String, String> globalParamMap, List<Property> globalParamList, CommandType commandType, Date scheduleTime, String timezone) {
+        if (globalParamList == null || globalParamList.isEmpty()) {
+            return null;
+        }
+        Map<String, String> globalMap = new HashMap<>();
+        if (globalParamMap != null) {
+            globalMap.putAll(globalParamMap);
+        }
+        Map<String, String> allParamMap = new HashMap<>();
+        //If it is a complement, a complement time needs to be passed in, according to the task type
+        Map<String, String> timeParams = BusinessTimeUtils.
+                getBusinessTime(commandType, scheduleTime, timezone);
+
+        if (timeParams != null) {
+            allParamMap.putAll(timeParams);
+        }
+        allParamMap.putAll(globalMap);
+        Set<Map.Entry<String, String>> entries = allParamMap.entrySet();
+        Map<String, String> resolveMap = new HashMap<>();
+        for (Map.Entry<String, String> entry : entries) {
+            String val = entry.getValue();
+            if (val.startsWith("$")) {
+                String str = "";
+                // whether external scaling calculation is required
+                if (timeFunctionNeedExpand(val)) {
+                    str = timeFunctionExtension(processInstanceId, timezone, val);
+                } else {
+                    str = convertParameterPlaceholders(val, allParamMap);
+                }
+                resolveMap.put(entry.getKey(), str);
+            }
+        }
+        globalMap.putAll(resolveMap);
+        for (Property property : globalParamList) {
+            String val = globalMap.get(property.getProp());
+            if (val != null) {
+                property.setValue(val);
+            }
+        }
+        return JSONUtils.toJsonString(globalParamList);
+    }
+
+    @Override
+    public Map<String, Property> paramParsingPreparation(TaskExecutionContext taskExecutionContext, AbstractParameters parameters, ProcessInstance processInstance) {
+        Preconditions.checkNotNull(taskExecutionContext);
+        Preconditions.checkNotNull(parameters);
+        setGlobalParamsMap(taskExecutionContext);
+        Map<String, Property> globalParams = ParamUtils.getUserDefParamsMap(taskExecutionContext.getDefinedParams());
+        Map<String,String> globalParamsMap = taskExecutionContext.getDefinedParams();
+        org.apache.dolphinscheduler.spi.enums.CommandType commandType = org.apache.dolphinscheduler.spi.enums.CommandType.of(taskExecutionContext.getCmdTypeIfComplement());
+        Date scheduleTime = taskExecutionContext.getScheduleTime();
+
+        // combining local and global parameters
+        Map<String, Property> localParams = parameters.getInputLocalParametersMap();
+
+        //stream pass params
+        Map<String, Property> varParams = parameters.getVarPoolMap();
+
+        if (globalParams.size() == 0 && localParams.size() == 0 && varParams.size() == 0) {

Review Comment:
   Use `isEmpty()` is better than `.size() == 0`



##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DolphinSchedulerCuringGlobalParams.class);
+
+    @Autowired
+    private TimePlaceholderResolverExpandService timePlaceholderResolverExpandService;
+
+    @Override
+    public String convertParameterPlaceholders(String val, Map<String, String> allParamMap) {
+        return ParameterUtils.convertParameterPlaceholders(val, allParamMap);
+    }
+
+    @Override
+    public boolean timeFunctionNeedExpand(String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionNeedExpand(placeholderName);
+    }
+
+    @Override
+    public String timeFunctionExtension(Integer processInstanceId, String timezone, String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionExtension(processInstanceId, timezone, placeholderName);
+    }
+
+    @Override
+    public String curingGlobalParams(Integer processInstanceId, Map<String, String> globalParamMap, List<Property> globalParamList, CommandType commandType, Date scheduleTime, String timezone) {
+        if (globalParamList == null || globalParamList.isEmpty()) {
+            return null;
+        }
+        Map<String, String> globalMap = new HashMap<>();
+        if (globalParamMap != null) {
+            globalMap.putAll(globalParamMap);
+        }
+        Map<String, String> allParamMap = new HashMap<>();
+        //If it is a complement, a complement time needs to be passed in, according to the task type
+        Map<String, String> timeParams = BusinessTimeUtils.
+                getBusinessTime(commandType, scheduleTime, timezone);
+
+        if (timeParams != null) {
+            allParamMap.putAll(timeParams);
+        }
+        allParamMap.putAll(globalMap);
+        Set<Map.Entry<String, String>> entries = allParamMap.entrySet();
+        Map<String, String> resolveMap = new HashMap<>();
+        for (Map.Entry<String, String> entry : entries) {
+            String val = entry.getValue();
+            if (val.startsWith("$")) {
+                String str = "";
+                // whether external scaling calculation is required
+                if (timeFunctionNeedExpand(val)) {
+                    str = timeFunctionExtension(processInstanceId, timezone, val);
+                } else {
+                    str = convertParameterPlaceholders(val, allParamMap);
+                }
+                resolveMap.put(entry.getKey(), str);
+            }
+        }
+        globalMap.putAll(resolveMap);
+        for (Property property : globalParamList) {
+            String val = globalMap.get(property.getProp());
+            if (val != null) {
+                property.setValue(val);
+            }
+        }
+        return JSONUtils.toJsonString(globalParamList);
+    }
+
+    @Override
+    public Map<String, Property> paramParsingPreparation(TaskExecutionContext taskExecutionContext, AbstractParameters parameters, ProcessInstance processInstance) {
+        Preconditions.checkNotNull(taskExecutionContext);
+        Preconditions.checkNotNull(parameters);

Review Comment:
   It's better to use `@NonNull` in the method to replace Preconditions.checkNotNull.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] lenboo commented on pull request #10704: [Optimization] Calculate global parameter and local parameter at master.

Posted by GitBox <gi...@apache.org>.
lenboo commented on PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#issuecomment-1171310395

   +1


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] sonarcloud[bot] commented on pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#issuecomment-1171147787

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=10704)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL) [32 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=10704&resolved=false&types=CODE_SMELL)
   
   [![38.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/25-16px.png '38.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list) [38.1% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=10704&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] WangJPLeo commented on a diff in pull request #10704: [Optimization] Global parameter and local parameter calculation external expansion.

Posted by GitBox <gi...@apache.org>.
WangJPLeo commented on code in PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704#discussion_r910952054


##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {

Review Comment:
   ok.



##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/expand/DolphinSchedulerCuringGlobalParams.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.service.expand;
+
+import com.google.common.base.Preconditions;
+import org.apache.dolphinscheduler.common.Constants;
+import org.apache.dolphinscheduler.common.enums.CommandType;
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.utils.ParameterUtils;
+import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+import org.apache.dolphinscheduler.plugin.task.api.parameters.AbstractParameters;
+import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
+import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
+import org.apache.dolphinscheduler.spi.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_EXECUTE_PATH;
+import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.PARAMETER_TASK_INSTANCE_ID;
+
+@Component
+public class DolphinSchedulerCuringGlobalParams implements CuringParamsService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DolphinSchedulerCuringGlobalParams.class);
+
+    @Autowired
+    private TimePlaceholderResolverExpandService timePlaceholderResolverExpandService;
+
+    @Override
+    public String convertParameterPlaceholders(String val, Map<String, String> allParamMap) {
+        return ParameterUtils.convertParameterPlaceholders(val, allParamMap);
+    }
+
+    @Override
+    public boolean timeFunctionNeedExpand(String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionNeedExpand(placeholderName);
+    }
+
+    @Override
+    public String timeFunctionExtension(Integer processInstanceId, String timezone, String placeholderName) {
+        return timePlaceholderResolverExpandService.timeFunctionExtension(processInstanceId, timezone, placeholderName);
+    }
+
+    @Override
+    public String curingGlobalParams(Integer processInstanceId, Map<String, String> globalParamMap, List<Property> globalParamList, CommandType commandType, Date scheduleTime, String timezone) {
+        if (globalParamList == null || globalParamList.isEmpty()) {
+            return null;
+        }
+        Map<String, String> globalMap = new HashMap<>();
+        if (globalParamMap != null) {
+            globalMap.putAll(globalParamMap);
+        }
+        Map<String, String> allParamMap = new HashMap<>();
+        //If it is a complement, a complement time needs to be passed in, according to the task type
+        Map<String, String> timeParams = BusinessTimeUtils.
+                getBusinessTime(commandType, scheduleTime, timezone);
+
+        if (timeParams != null) {
+            allParamMap.putAll(timeParams);
+        }
+        allParamMap.putAll(globalMap);
+        Set<Map.Entry<String, String>> entries = allParamMap.entrySet();
+        Map<String, String> resolveMap = new HashMap<>();
+        for (Map.Entry<String, String> entry : entries) {
+            String val = entry.getValue();
+            if (val.startsWith("$")) {

Review Comment:
   ok.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [dolphinscheduler] lenboo merged pull request #10704: [Optimization] Calculate global parameter and local parameter at master.

Posted by GitBox <gi...@apache.org>.
lenboo merged PR #10704:
URL: https://github.com/apache/dolphinscheduler/pull/10704


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@dolphinscheduler.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org