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/12/14 12:25:23 UTC

[GitHub] [dolphinscheduler] jieguangzhou opened a new pull request, #13194: Supports task instance cache operation

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

   <!--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
   
   <!--(For example: This pull request adds checkstyle plugin).-->
   
   ## Brief change log
   
   <!--*(for example:)*
   - *Add maven-checkstyle-plugin to root pom.xml*
   -->
   
   ## Verify this pull request
   
   <!--*(Please pick either of the following options)*-->
   
   This pull request is code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   <!--*(example:)*
   - *Added dolphinscheduler-dao tests for end-to-end.*
   - *Added CronUtilsTest to verify the change.*
   - *Manually verified the change by testing locally.* -->
   
   (or)
   
   If your pull request contain incompatible change, you should also add it to `docs/docs/en/guide/upgrede/incompatible.md`
   


-- 
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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/IsCache.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.common.enums;
+
+import com.baomidou.mybatisplus.annotation.EnumValue;
+
+public enum IsCache {

Review Comment:
   You can use the `Flag` enum. No need to recreate a similar one.



##########
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskEventType.java:
##########
@@ -22,5 +22,7 @@ public enum TaskEventType {
     DELAY,
     RUNNING,
     RESULT,
-    WORKER_REJECT
+    WORKER_REJECT,
+
+    CACHE,

Review Comment:
   ```suggestion
       WORKER_REJECT,
       CACHE,
   ```



##########
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java:
##########
@@ -319,4 +327,38 @@ public TaskInstance queryTaskInstanceById(User loginUser, long projectCode, Long
         }
         return taskInstance;
     }
+
+    @Override
+    public TaskInstanceRemoveCacheResponse removeTaskInstanceCache(User loginUser, long projectCode,
+                                                                   Integer taskInstanceId) {
+        Result result = new Result();
+
+        Project project = projectMapper.queryByCode(projectCode);
+        projectService.checkProjectAndAuthThrowException(loginUser, project, INSTANCE_UPDATE);
+
+        TaskInstance taskInstance = taskInstanceMapper.selectById(taskInstanceId);
+        if (taskInstance == null) {
+            logger.error("Task definition can not be found, projectCode:{}, taskInstanceId:{}.", projectCode,
+                    taskInstanceId);
+            putMsg(result, Status.TASK_INSTANCE_NOT_FOUND);
+            return new TaskInstanceRemoveCacheResponse(result);
+        }
+        String tagCacheKey = taskInstance.getCacheKey();
+        String cacheKey = TaskCacheUtils.revertCacheKey(tagCacheKey);
+        List<Integer> cacheTaskInstanceIds = new ArrayList<>();
+        while (true) {
+            TaskInstance cacheTaskInstance = taskInstanceDao.findTaskInstanceByCacheKey(cacheKey);

Review Comment:
   Maybe it's not a loop action. 



##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    final static String MERGE_TAG = "-";
+
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext context) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPool(taskInstance, context));
+        String data = StringUtils.join(keyElements, "_");
+        String cacheKey = md5(data);
+        System.out.println("cacheKey: " + cacheKey + ", data: " + data);

Review Comment:
   should remove it.



##########
dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql:
##########
@@ -894,7 +898,8 @@ CREATE TABLE `t_ds_task_instance` (
   `test_flag`  tinyint(4) DEFAULT null COMMENT 'test flag:0 normal, 1 test run',
   PRIMARY KEY (`id`),
   KEY `process_instance_id` (`process_instance_id`) USING BTREE,
-  KEY `idx_code_version` (`task_code`, `task_definition_version`) USING BTREE
+  KEY `idx_code_version` (`task_code`, `task_definition_version`) USING BTREE,
+  KEY `cache_key` (`cache_key`) USING BTREE,

Review Comment:
   I think the unique index is better. WDYT?



##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java:
##########
@@ -187,6 +189,19 @@ public class TaskInstance implements Serializable {
      */
     private Flag flag;
 
+    /**
+     * task is cache: yes/no
+     */
+    private IsCache isCache;
+
+    /**
+     * task is cache: yes/no
+     */
+    @TableField(updateStrategy = FieldStrategy.IGNORED)
+    private String cacheKey;
+
+    @TableField(exist = false)
+    private String tmpCacheKey;

Review Comment:
   Please remove this field if it doesn't belong to this entity.



##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java:
##########
@@ -187,6 +189,19 @@ public class TaskInstance implements Serializable {
      */
     private Flag flag;
 
+    /**
+     * task is cache: yes/no
+     */
+    private IsCache isCache;

Review Comment:
   Same here



##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java:
##########
@@ -187,6 +189,19 @@ public class TaskInstance implements Serializable {
      */
     private Flag flag;
 
+    /**
+     * task is cache: yes/no
+     */
+    private IsCache isCache;
+
+    /**
+     * task is cache: yes/no
+     */

Review Comment:
   Please use the right comments.



##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java:
##########
@@ -187,6 +189,19 @@ public class TaskInstance implements Serializable {
      */
     private Flag flag;
 
+    /**
+     * task is cache: yes/no
+     */
+    private IsCache isCache;
+
+    /**
+     * task is cache: yes/no
+     */
+    @TableField(updateStrategy = FieldStrategy.IGNORED)

Review Comment:
   Why add the ignored update strategy for it?



##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinition.java:
##########
@@ -115,6 +116,11 @@ public class TaskDefinition {
      */
     private Flag flag;
 
+    /**
+     * task is cache: yes/no
+     */
+    private IsCache isCache;

Review Comment:
   ```suggestion
       private Flag isCache;
   ```



##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    final static String MERGE_TAG = "-";
+
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext context) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPool(taskInstance, context));
+        String data = StringUtils.join(keyElements, "_");
+        String cacheKey = md5(data);
+        System.out.println("cacheKey: " + cacheKey + ", data: " + data);
+        return cacheKey;
+    }
+
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    public static String revertCacheKey(String tagCacheKey) {
+        if (tagCacheKey == null) {
+            return "";
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            return tagCacheKey.split(MERGE_TAG)[1];
+        } else {
+            return tagCacheKey;
+        }
+    }
+
+    public static String md5(String data) {
+        try {
+            MessageDigest md = MessageDigest.getInstance("MD5");
+            byte[] md5 = md.digest(data.getBytes(StandardCharsets.UTF_8));
+
+            StringBuilder sb = new StringBuilder();
+            for (byte b : md5) {
+                sb.append(String.format("%02x", b));
+            }
+            return sb.toString();
+        } catch (NoSuchAlgorithmException e) {
+            e.printStackTrace();
+        }

Review Comment:
   ```suggestion
           } catch (NoSuchAlgorithmException ignore) {
               
           }
   ```



##########
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java:
##########
@@ -276,4 +287,30 @@ private boolean isTaskNeedToCheck(TaskPriority taskPriority) {
         }
         return false;
     }
+
+    private boolean checkIsCacheExecution(TaskInstance taskInstance, TaskExecutionContext context) {
+        if (taskInstance.getIsCache().equals(IsCache.YES)) {
+            String cacheKey = TaskCacheUtils.generateCacheKey(taskInstance, context);
+            TaskInstance cacheTaskInstance = taskInstanceDao.findTaskInstanceByCacheKey(cacheKey);
+            if (cacheTaskInstance != null) {
+                logger.info("Task {} is cache, no need to dispatch, task instance id: {}",
+                        taskInstance.getName(), taskInstance.getId());
+                addCacheEvent(taskInstance, cacheTaskInstance);
+                taskInstance.setTmpCacheKey(TaskCacheUtils.generateTagCacheKey(cacheTaskInstance.getId(), cacheKey));
+                return true;
+            } else {
+                taskInstance.setTmpCacheKey(cacheKey);
+            }
+        }
+        return false;

Review Comment:
   ```suggestion
           if (taskInstance.getIsCache().equals(IsCache.NO)) {
               return false;
           }
             String cacheKey = TaskCacheUtils.generateCacheKey(taskInstance, context);
             TaskInstance cacheTaskInstance = taskInstanceDao.findTaskInstanceByCacheKey(cacheKey);
             if (cacheTaskInstance != null) {
                 logger.info("Task {} is cache, no need to dispatch, task instance id: {}",
                         taskInstance.getName(), taskInstance.getId());
                 addCacheEvent(taskInstance, cacheTaskInstance);
                 taskInstance.setTmpCacheKey(TaskCacheUtils.generateTagCacheKey(cacheTaskInstance.getId(), cacheKey));
                 return true;
             } 
           taskInstance.setTmpCacheKey(cacheKey);
           return false;
   ```



-- 
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] jieguangzhou commented on pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   @caishunfeng @ruanwenjun please PTAL, thanks


-- 
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] Radeity commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    private TaskCacheUtils() {
+        throw new IllegalStateException("Utility class");
+    }
+
+    public static final String MERGE_TAG = "-";
+
+    /**
+     * generate cache key for task instance
+     * the follow message will be used to generate cache key
+     * 2. task version
+     * 3. task is cache
+     * 4. input VarPool, from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * @param taskExecutionContext taskExecutionContext
+     * @return cache key
+     */
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext taskExecutionContext) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPoolData(taskInstance, taskExecutionContext));
+        String data = StringUtils.join(keyElements, "_");
+        return DigestUtils.sha256Hex(data);
+    }
+
+    /**
+     * generate cache key for task instance which is cache execute
+     * this key will record which cache task instance will be copied, and cache key will be used
+     * tagCacheKey = sourceTaskId + "-" + cacheKey
+     * @param sourceTaskId source task id
+     * @param cacheKey cache key
+     * @return tagCacheKey
+     */
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    /**
+     * revert cache key tag to source task id and cache key
+     * @param tagCacheKey cache key
+     * @return Pair<Integer, String>, first is source task id, second is cache key
+     */
+    public static Pair<Integer, String> revertCacheKey(String tagCacheKey) {
+        Pair<Integer, String> taskIdAndCacheKey;
+        if (tagCacheKey == null) {
+            taskIdAndCacheKey = Pair.of(-1, "");
+            return taskIdAndCacheKey;
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            String[] split = tagCacheKey.split(MERGE_TAG);
+            if (split.length == 2) {
+                taskIdAndCacheKey = Pair.of(Integer.parseInt(split[0]), split[1]);
+            } else {
+                taskIdAndCacheKey = Pair.of(-1, "");
+            }
+            return taskIdAndCacheKey;
+        } else {
+            return Pair.of(-1, tagCacheKey);
+        }
+    }
+
+    /**
+     * get hash data of task input var pool
+     * there are two parts of task input var pool: from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * taskExecutionContext taskExecutionContext
+     */
+    public static String getTaskInputVarPoolData(TaskInstance taskInstance, TaskExecutionContext context) {
+        JsonNode taskParams = JSONUtils.parseObject(taskInstance.getTaskParams());
+
+        // The set of input values considered from localParams in the taskParams
+        Set<String> propertyInSet = JSONUtils.toList(taskParams.get("localParams").toString(), Property.class).stream()
+                .filter(property -> property.getDirect().equals(Direct.IN))
+                .map(Property::getProp).collect(Collectors.toSet());
+
+        // The set of input values considered from `${var}` form task definition
+        propertyInSet.addAll(getScriptVarInSet(taskInstance));

Review Comment:
   > Dose environment config means the actual definition content about environment_code ?
   
   Yes, it is. I think it's better consider it when generate cache key in this PR for two reasons:
   - `EnvironmentConfig` can be directly fetched from either `TaskInstance` or `TaskExecutionContext`.
   - `EnvironmentConfig` is like `taskCode` which DS takes charge of their management. Rather, data from the worker or from the resource center may change, user themselves should take into consideration.
   
   WDYT?



-- 
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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.6% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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] jieguangzhou commented on pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   @Amy0104 @songjianet please help to review the front-end 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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql:
##########
@@ -894,7 +898,8 @@ CREATE TABLE `t_ds_task_instance` (
   `test_flag`  tinyint(4) DEFAULT null COMMENT 'test flag:0 normal, 1 test run',
   PRIMARY KEY (`id`),
   KEY `process_instance_id` (`process_instance_id`) USING BTREE,
-  KEY `idx_code_version` (`task_code`, `task_definition_version`) USING BTREE
+  KEY `idx_code_version` (`task_code`, `task_definition_version`) USING BTREE,
+  KEY `cache_key` (`cache_key`) USING BTREE,

Review Comment:
   Sometimes there are one-to-many relationships. And I have set 'limit 1' in SQL query .



-- 
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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   # [Codecov](https://codecov.io/gh/apache/dolphinscheduler/pull/13194?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 [#13194](https://codecov.io/gh/apache/dolphinscheduler/pull/13194?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3609255) into [dev](https://codecov.io/gh/apache/dolphinscheduler/commit/43e6ac32ddc468c57633162a5cd442a70f418c18?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (43e6ac3) will **decrease** coverage by `0.16%`.
   > The diff coverage is `2.45%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##                dev   #13194      +/-   ##
   ============================================
   - Coverage     39.37%   39.20%   -0.17%     
   + Complexity     4278     4276       -2     
   ============================================
     Files          1066     1070       +4     
     Lines         40479    40595     +116     
     Branches       4657     4652       -5     
   ============================================
   - Hits          15937    15914      -23     
   - Misses        22755    22893     +138     
   - Partials       1787     1788       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/dolphinscheduler/pull/13194?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...heduler/api/controller/TaskInstanceController.java](https://codecov.io/gh/apache/dolphinscheduler/pull/13194/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-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL2NvbnRyb2xsZXIvVGFza0luc3RhbmNlQ29udHJvbGxlci5qYXZh) | `45.45% <0.00%> (-4.55%)` | :arrow_down: |
   | [.../taskInstance/TaskInstanceRemoveCacheResponse.java](https://codecov.io/gh/apache/dolphinscheduler/pull/13194/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-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL2R0by90YXNrSW5zdGFuY2UvVGFza0luc3RhbmNlUmVtb3ZlQ2FjaGVSZXNwb25zZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...uler/api/service/impl/TaskInstanceServiceImpl.java](https://codecov.io/gh/apache/dolphinscheduler/pull/13194/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-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL3NlcnZpY2UvaW1wbC9UYXNrSW5zdGFuY2VTZXJ2aWNlSW1wbC5qYXZh) | `47.58% <0.00%> (-8.52%)` | :arrow_down: |
   | [.../apache/dolphinscheduler/common/enums/IsCache.java](https://codecov.io/gh/apache/dolphinscheduler/pull/13194/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-ZG9scGhpbnNjaGVkdWxlci1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvY29tbW9uL2VudW1zL0lzQ2FjaGUuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...e/dolphinscheduler/common/enums/TaskEventType.java](https://codecov.io/gh/apache/dolphinscheduler/pull/13194/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-ZG9scGhpbnNjaGVkdWxlci1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvY29tbW9uL2VudW1zL1Rhc2tFdmVudFR5cGUuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...he/dolphinscheduler/dao/entity/TaskDefinition.java](https://codecov.io/gh/apache/dolphinscheduler/pull/13194/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-ZG9scGhpbnNjaGVkdWxlci1kYW8vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvZGFvL2VudGl0eS9UYXNrRGVmaW5pdGlvbi5qYXZh) | `31.57% <ø> (ø)` | |
   | [...ache/dolphinscheduler/dao/entity/TaskInstance.java](https://codecov.io/gh/apache/dolphinscheduler/pull/13194/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-ZG9scGhpbnNjaGVkdWxlci1kYW8vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvZGFvL2VudGl0eS9UYXNrSW5zdGFuY2UuamF2YQ==) | `45.45% <ø> (ø)` | |
   | [...duler/dao/repository/impl/TaskInstanceDaoImpl.java](https://codecov.io/gh/apache/dolphinscheduler/pull/13194/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-ZG9scGhpbnNjaGVkdWxlci1kYW8vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvZGFvL3JlcG9zaXRvcnkvaW1wbC9UYXNrSW5zdGFuY2VEYW9JbXBsLmphdmE=) | `3.17% <0.00%> (-0.16%)` | :arrow_down: |
   | [...che/dolphinscheduler/dao/utils/TaskCacheUtils.java](https://codecov.io/gh/apache/dolphinscheduler/pull/13194/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-ZG9scGhpbnNjaGVkdWxlci1kYW8vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvZGFvL3V0aWxzL1Rhc2tDYWNoZVV0aWxzLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...ver/master/consumer/TaskPriorityQueueConsumer.java](https://codecov.io/gh/apache/dolphinscheduler/pull/13194/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-ZG9scGhpbnNjaGVkdWxlci1tYXN0ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvc2VydmVyL21hc3Rlci9jb25zdW1lci9UYXNrUHJpb3JpdHlRdWV1ZUNvbnN1bWVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | ... and [37 more](https://codecov.io/gh/apache/dolphinscheduler/pull/13194/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) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?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] Radeity commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    private TaskCacheUtils() {
+        throw new IllegalStateException("Utility class");
+    }
+
+    public static final String MERGE_TAG = "-";
+
+    /**
+     * generate cache key for task instance
+     * the follow message will be used to generate cache key
+     * 2. task version
+     * 3. task is cache
+     * 4. input VarPool, from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * @param taskExecutionContext taskExecutionContext
+     * @return cache key
+     */
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext taskExecutionContext) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPoolData(taskInstance, taskExecutionContext));
+        String data = StringUtils.join(keyElements, "_");
+        return DigestUtils.sha256Hex(data);
+    }
+
+    /**
+     * generate cache key for task instance which is cache execute
+     * this key will record which cache task instance will be copied, and cache key will be used
+     * tagCacheKey = sourceTaskId + "-" + cacheKey
+     * @param sourceTaskId source task id
+     * @param cacheKey cache key
+     * @return tagCacheKey
+     */
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    /**
+     * revert cache key tag to source task id and cache key
+     * @param tagCacheKey cache key
+     * @return Pair<Integer, String>, first is source task id, second is cache key
+     */
+    public static Pair<Integer, String> revertCacheKey(String tagCacheKey) {
+        Pair<Integer, String> taskIdAndCacheKey;
+        if (tagCacheKey == null) {
+            taskIdAndCacheKey = Pair.of(-1, "");
+            return taskIdAndCacheKey;
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            String[] split = tagCacheKey.split(MERGE_TAG);
+            if (split.length == 2) {
+                taskIdAndCacheKey = Pair.of(Integer.parseInt(split[0]), split[1]);
+            } else {
+                taskIdAndCacheKey = Pair.of(-1, "");
+            }
+            return taskIdAndCacheKey;
+        } else {
+            return Pair.of(-1, tagCacheKey);
+        }
+    }
+
+    /**
+     * get hash data of task input var pool
+     * there are two parts of task input var pool: from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * taskExecutionContext taskExecutionContext
+     */
+    public static String getTaskInputVarPoolData(TaskInstance taskInstance, TaskExecutionContext context) {
+        JsonNode taskParams = JSONUtils.parseObject(taskInstance.getTaskParams());
+
+        // The set of input values considered from localParams in the taskParams
+        Set<String> propertyInSet = JSONUtils.toList(taskParams.get("localParams").toString(), Property.class).stream()
+                .filter(property -> property.getDirect().equals(Direct.IN))
+                .map(Property::getProp).collect(Collectors.toSet());
+
+        // The set of input values considered from `${var}` form task definition
+        propertyInSet.addAll(getScriptVarInSet(taskInstance));

Review Comment:
   I've noticed that you generate cache key with `taskDefinitionVersion`, `taskDefinition` only has field `environment_code`, however, environment config change will not change `environment_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] sonarcloud[bot] commented on pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![59.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50-16px.png '59.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [59.6% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.6% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [10 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![2.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '2.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [2.1% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![3.7%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5-16px.png '3.7%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [3.7% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.6% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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] caishunfeng merged pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


-- 
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] zhongjiajie commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql:
##########
@@ -894,7 +898,8 @@ CREATE TABLE `t_ds_task_instance` (
   `test_flag`  tinyint(4) DEFAULT null COMMENT 'test flag:0 normal, 1 test run',
   PRIMARY KEY (`id`),
   KEY `process_instance_id` (`process_instance_id`) USING BTREE,
-  KEY `idx_code_version` (`task_code`, `task_definition_version`) USING BTREE
+  KEY `idx_code_version` (`task_code`, `task_definition_version`) USING BTREE,
+  KEY `cache_key` (`cache_key`) USING BTREE

Review Comment:
   ```suggestion
     KEY `idx_cache_key` (`cache_key`) USING BTREE
   ```



##########
dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql:
##########
@@ -796,6 +800,7 @@ CREATE TABLE t_ds_task_instance (
 ) ;
 
 create index idx_task_instance_code_version on t_ds_task_instance (task_code, task_definition_version);
+create index cache_key on t_ds_task_instance (cache_key);

Review Comment:
   ```suggestion
   create index idx_cache_key on t_ds_task_instance (cache_key);
   ```



##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/3.2.0_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -119,3 +119,14 @@ d//
 delimiter ;
 select uc_dolphin_T_t_ds_task_instance_R_test_flag();
 DROP FUNCTION uc_dolphin_T_t_ds_task_instance_R_test_flag();
+
+ALTER TABLE t_ds_task_definition DROP COLUMN IF EXISTS is_cache;

Review Comment:
   can we make L123 and L128 together to make more sense, to told user we want to recreate the column.
   
   ```sql
   ALTER TABLE t_ds_task_definition DROP COLUMN IF EXISTS is_cache;
   ALTER TABLE t_ds_task_definition ADD COLUMN IF NOT EXISTS is_cache int DEFAULT '0';
   ```



-- 
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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    private TaskCacheUtils() {
+        throw new IllegalStateException("Utility class");
+    }
+
+    public static final String MERGE_TAG = "-";
+
+    /**
+     * generate cache key for task instance
+     * the follow message will be used to generate cache key
+     * 2. task version
+     * 3. task is cache
+     * 4. input VarPool, from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * @param taskExecutionContext taskExecutionContext
+     * @return cache key
+     */
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext taskExecutionContext) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPoolData(taskInstance, taskExecutionContext));
+        String data = StringUtils.join(keyElements, "_");
+        return DigestUtils.sha256Hex(data);
+    }
+
+    /**
+     * generate cache key for task instance which is cache execute
+     * this key will record which cache task instance will be copied, and cache key will be used
+     * tagCacheKey = sourceTaskId + "-" + cacheKey
+     * @param sourceTaskId source task id
+     * @param cacheKey cache key
+     * @return tagCacheKey
+     */
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    /**
+     * revert cache key tag to source task id and cache key
+     * @param tagCacheKey cache key
+     * @return Pair<Integer, String>, first is source task id, second is cache key
+     */
+    public static Pair<Integer, String> revertCacheKey(String tagCacheKey) {
+        Pair<Integer, String> taskIdAndCacheKey;
+        if (tagCacheKey == null) {
+            taskIdAndCacheKey = Pair.of(-1, "");
+            return taskIdAndCacheKey;
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            String[] split = tagCacheKey.split(MERGE_TAG);
+            if (split.length == 2) {
+                taskIdAndCacheKey = Pair.of(Integer.parseInt(split[0]), split[1]);
+            } else {
+                taskIdAndCacheKey = Pair.of(-1, "");
+            }
+            return taskIdAndCacheKey;
+        } else {
+            return Pair.of(-1, tagCacheKey);
+        }
+    }
+
+    /**
+     * get hash data of task input var pool
+     * there are two parts of task input var pool: from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * taskExecutionContext taskExecutionContext
+     */
+    public static String getTaskInputVarPoolData(TaskInstance taskInstance, TaskExecutionContext context) {
+        JsonNode taskParams = JSONUtils.parseObject(taskInstance.getTaskParams());
+
+        // The set of input values considered from localParams in the taskParams
+        Set<String> propertyInSet = JSONUtils.toList(taskParams.get("localParams").toString(), Property.class).stream()
+                .filter(property -> property.getDirect().equals(Direct.IN))
+                .map(Property::getProp).collect(Collectors.toSet());
+
+        // The set of input values considered from `${var}` form task definition
+        propertyInSet.addAll(getScriptVarInSet(taskInstance));

Review Comment:
   Dose `environment config` means the actual definition content about `environment_code `?
   If yes, we have not considered it in this pr.
   By the way, If some related resources are used by the task but change in the other places but don't change `taskDefinitionVersion` and `taskDefinition`, we will not catch them now. Such as the data from the worker or from the resource center.



-- 
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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java:
##########
@@ -319,4 +329,30 @@ public TaskInstance queryTaskInstanceById(User loginUser, long projectCode, Long
         }
         return taskInstance;
     }
+
+    @Override
+    public TaskInstanceRemoveCacheResponse removeTaskInstanceCache(User loginUser, long projectCode,
+                                                                   Integer taskInstanceId) {
+        Result result = new Result();
+
+        Project project = projectMapper.queryByCode(projectCode);
+        projectService.checkProjectAndAuthThrowException(loginUser, project, INSTANCE_UPDATE);
+
+        TaskInstance taskInstance = taskInstanceMapper.selectById(taskInstanceId);
+        if (taskInstance == null) {
+            logger.error("Task definition can not be found, projectCode:{}, taskInstanceId:{}.", projectCode,
+                    taskInstanceId);
+            putMsg(result, Status.TASK_INSTANCE_NOT_FOUND);
+            return new TaskInstanceRemoveCacheResponse(result);
+        }
+        String tagCacheKey = taskInstance.getCacheKey();
+        Pair<Integer, String> taskIdAndCacheKey = TaskCacheUtils.revertCacheKey(tagCacheKey);
+        String cacheKey = taskIdAndCacheKey.getRight();
+        if (!StringUtils.isEmpty(cacheKey)) {

Review Comment:
   ```suggestion
           if (StringUtils.isNotEmpty(cacheKey)) {
   ```



-- 
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] Radeity commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
docs/docs/zh/faq.md:
##########
@@ -720,4 +720,18 @@ A:在 3.0.0-alpha 版本之后,Python gateway server 集成到 api server 
 
 ---
 
+## Q: 缓存执行时怎么判断任务已经存在缓存过的任务,即如何判断一个任务可以使用另外一个任务的运行结果?
+
+A: 对于标识为`缓存执行`的任务, 当任务启动时会生成一个缓存key, 该key由以下字段组合哈希得到:
+
+- 任务定义:任务实例对应的任务定义的id
+- 任务的版本:任务实例对应的任务定义的版本
+- 任务输入的参数:包括上游节点和全局参数传入的参数中,被任务定义的参数列表所引用和任务定义中使用`${}`引用的参数
+
+当缓存标识的任务运行时,会查找数据库中是否用相同缓存key的数据,
+- 若有则复制该任务实例并进行相应数据的更新
+- 若无,则任务照常运行,并在任务完成时将任务实例的数据存入缓存
+
+若不需要缓存时,可以在工作流实例中右键运行清除缓存,则会清楚该版本下当前输入的参数的缓存数据。

Review Comment:
   Typo error here, “清除”.



##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    private TaskCacheUtils() {
+        throw new IllegalStateException("Utility class");
+    }
+
+    public static final String MERGE_TAG = "-";
+
+    /**
+     * generate cache key for task instance
+     * the follow message will be used to generate cache key
+     * 2. task version
+     * 3. task is cache
+     * 4. input VarPool, from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * @param taskExecutionContext taskExecutionContext
+     * @return cache key
+     */
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext taskExecutionContext) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPoolData(taskInstance, taskExecutionContext));
+        String data = StringUtils.join(keyElements, "_");
+        return DigestUtils.sha256Hex(data);
+    }
+
+    /**
+     * generate cache key for task instance which is cache execute
+     * this key will record which cache task instance will be copied, and cache key will be used
+     * tagCacheKey = sourceTaskId + "-" + cacheKey
+     * @param sourceTaskId source task id
+     * @param cacheKey cache key
+     * @return tagCacheKey
+     */
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    /**
+     * revert cache key tag to source task id and cache key
+     * @param tagCacheKey cache key
+     * @return Pair<Integer, String>, first is source task id, second is cache key
+     */
+    public static Pair<Integer, String> revertCacheKey(String tagCacheKey) {
+        Pair<Integer, String> taskIdAndCacheKey;
+        if (tagCacheKey == null) {
+            taskIdAndCacheKey = Pair.of(-1, "");
+            return taskIdAndCacheKey;
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            String[] split = tagCacheKey.split(MERGE_TAG);
+            if (split.length == 2) {
+                taskIdAndCacheKey = Pair.of(Integer.parseInt(split[0]), split[1]);
+            } else {
+                taskIdAndCacheKey = Pair.of(-1, "");
+            }
+            return taskIdAndCacheKey;
+        } else {
+            return Pair.of(-1, tagCacheKey);
+        }
+    }
+
+    /**
+     * get hash data of task input var pool
+     * there are two parts of task input var pool: from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * taskExecutionContext taskExecutionContext
+     */
+    public static String getTaskInputVarPoolData(TaskInstance taskInstance, TaskExecutionContext context) {
+        JsonNode taskParams = JSONUtils.parseObject(taskInstance.getTaskParams());
+
+        // The set of input values considered from localParams in the taskParams
+        Set<String> propertyInSet = JSONUtils.toList(taskParams.get("localParams").toString(), Property.class).stream()
+                .filter(property -> property.getDirect().equals(Direct.IN))
+                .map(Property::getProp).collect(Collectors.toSet());
+
+        // The set of input values considered from `${var}` form task definition
+        propertyInSet.addAll(getScriptVarInSet(taskInstance));

Review Comment:
   Should we also consider environment variables, such as adding environment config as part of cache key element?



-- 
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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [10 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![2.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '2.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [2.1% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![4.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5-16px.png '4.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [4.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.8%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.8% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    private TaskCacheUtils() {
+        throw new IllegalStateException("Utility class");
+    }
+
+    public static final String MERGE_TAG = "-";
+
+    /**
+     * generate cache key for task instance
+     * the follow message will be used to generate cache key
+     * 2. task version
+     * 3. task is cache
+     * 4. input VarPool, from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * @param taskExecutionContext taskExecutionContext
+     * @return cache key
+     */
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext taskExecutionContext) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPoolData(taskInstance, taskExecutionContext));
+        String data = StringUtils.join(keyElements, "_");
+        return DigestUtils.sha256Hex(data);
+    }
+
+    /**
+     * generate cache key for task instance which is cache execute
+     * this key will record which cache task instance will be copied, and cache key will be used
+     * tagCacheKey = sourceTaskId + "-" + cacheKey
+     * @param sourceTaskId source task id
+     * @param cacheKey cache key
+     * @return tagCacheKey
+     */
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    /**
+     * revert cache key tag to source task id and cache key
+     * @param tagCacheKey cache key
+     * @return Pair<Integer, String>, first is source task id, second is cache key
+     */
+    public static Pair<Integer, String> revertCacheKey(String tagCacheKey) {
+        Pair<Integer, String> taskIdAndCacheKey;
+        if (tagCacheKey == null) {
+            taskIdAndCacheKey = Pair.of(-1, "");
+            return taskIdAndCacheKey;
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            String[] split = tagCacheKey.split(MERGE_TAG);
+            if (split.length == 2) {
+                taskIdAndCacheKey = Pair.of(Integer.parseInt(split[0]), split[1]);
+            } else {
+                taskIdAndCacheKey = Pair.of(-1, "");
+            }
+            return taskIdAndCacheKey;
+        } else {
+            return Pair.of(-1, tagCacheKey);
+        }
+    }
+
+    /**
+     * get hash data of task input var pool
+     * there are two parts of task input var pool: from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * taskExecutionContext taskExecutionContext
+     */
+    public static String getTaskInputVarPoolData(TaskInstance taskInstance, TaskExecutionContext context) {
+        JsonNode taskParams = JSONUtils.parseObject(taskInstance.getTaskParams());
+
+        // The set of input values considered from localParams in the taskParams
+        Set<String> propertyInSet = JSONUtils.toList(taskParams.get("localParams").toString(), Property.class).stream()
+                .filter(property -> property.getDirect().equals(Direct.IN))
+                .map(Property::getProp).collect(Collectors.toSet());
+
+        // The set of input values considered from `${var}` form task definition
+        propertyInSet.addAll(getScriptVarInSet(taskInstance));

Review Comment:
   Dose `environment config` means the actual definition content about `environment_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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java:
##########
@@ -319,4 +327,38 @@ public TaskInstance queryTaskInstanceById(User loginUser, long projectCode, Long
         }
         return taskInstance;
     }
+
+    @Override
+    public TaskInstanceRemoveCacheResponse removeTaskInstanceCache(User loginUser, long projectCode,
+                                                                   Integer taskInstanceId) {
+        Result result = new Result();
+
+        Project project = projectMapper.queryByCode(projectCode);
+        projectService.checkProjectAndAuthThrowException(loginUser, project, INSTANCE_UPDATE);
+
+        TaskInstance taskInstance = taskInstanceMapper.selectById(taskInstanceId);
+        if (taskInstance == null) {
+            logger.error("Task definition can not be found, projectCode:{}, taskInstanceId:{}.", projectCode,
+                    taskInstanceId);
+            putMsg(result, Status.TASK_INSTANCE_NOT_FOUND);
+            return new TaskInstanceRemoveCacheResponse(result);
+        }
+        String tagCacheKey = taskInstance.getCacheKey();
+        String cacheKey = TaskCacheUtils.revertCacheKey(tagCacheKey);
+        List<Integer> cacheTaskInstanceIds = new ArrayList<>();
+        while (true) {
+            TaskInstance cacheTaskInstance = taskInstanceDao.findTaskInstanceByCacheKey(cacheKey);

Review Comment:
   There may be one `cacheKey` for multiple pieces of data. For example, two cache tasks will same cache key run almost simultaneously.



-- 
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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [15 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![48.2%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/40-16px.png '48.2%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [48.2% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.8%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.8% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=SECURITY_HOTSPOT) [2 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [14 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![2.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '2.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [2.1% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![4.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5-16px.png '4.8%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [4.8% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=SECURITY_HOTSPOT) [2 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [14 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![2.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '2.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [2.1% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![4.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5-16px.png '4.8%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [4.8% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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] github-code-scanning[bot] commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

Posted by GitBox <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13194:
URL: https://github.com/apache/dolphinscheduler/pull/13194#discussion_r1049381997


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    private TaskCacheUtils() {
+        throw new IllegalStateException("Utility class");
+    }
+
+    public static final String MERGE_TAG = "-";
+
+    /**
+     * generate cache key for task instance
+     * the follow message will be used to generate cache key
+     * 2. task version
+     * 3. task is cache
+     * 4. input VarPool, from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * @param taskExecutionContext taskExecutionContext
+     * @return cache key
+     */
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext taskExecutionContext) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPoolData(taskInstance, taskExecutionContext));
+        String data = StringUtils.join(keyElements, "_");
+        return DigestUtils.sha1Hex(data);
+    }
+
+    /**
+     * generate cache key for task instance which is cache execute
+     * this key will record which cache task instance will be copied, and cache key will be used
+     * tagCacheKey = sourceTaskId + "-" + cacheKey
+     * @param sourceTaskId source task id
+     * @param cacheKey cache key
+     * @return tagCacheKey
+     */
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    /**
+     * revert cache key tag to source task id and cache key
+     * @param tagCacheKey cache key
+     * @return Pair<Integer, String>, first is source task id, second is cache key
+     */
+    public static Pair<Integer, String> revertCacheKey(String tagCacheKey) {
+        Pair<Integer, String> taskIdAndCacheKey;
+        if (tagCacheKey == null) {
+            taskIdAndCacheKey = Pair.of(-1, "");
+            return taskIdAndCacheKey;
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            String[] split = tagCacheKey.split(MERGE_TAG);
+            if (split.length == 2) {
+                taskIdAndCacheKey = Pair.of(Integer.parseInt(split[0]), split[1]);

Review Comment:
   ## Missing catch of NumberFormatException
   
   Potential uncaught 'java.lang.NumberFormatException'.
   
   [Show more details](https://github.com/apache/dolphinscheduler/security/code-scanning/2399)



-- 
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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
docs/docs/zh/faq.md:
##########
@@ -720,4 +720,18 @@ A:在 3.0.0-alpha 版本之后,Python gateway server 集成到 api server 
 
 ---
 
+## Q: 缓存执行时怎么判断任务已经存在缓存过的任务,即如何判断一个任务可以使用另外一个任务的运行结果?
+
+A: 对于标识为`缓存执行`的任务, 当任务启动时会生成一个缓存key, 该key由以下字段组合哈希得到:
+
+- 任务定义:任务实例对应的任务定义的id
+- 任务的版本:任务实例对应的任务定义的版本
+- 任务输入的参数:包括上游节点和全局参数传入的参数中,被任务定义的参数列表所引用和任务定义中使用`${}`引用的参数
+
+当缓存标识的任务运行时,会查找数据库中是否用相同缓存key的数据,
+- 若有则复制该任务实例并进行相应数据的更新
+- 若无,则任务照常运行,并在任务完成时将任务实例的数据存入缓存
+
+若不需要缓存时,可以在工作流实例中右键运行清除缓存,则会清楚该版本下当前输入的参数的缓存数据。

Review Comment:
   Thanks, I have fixed it



-- 
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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.6% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.5% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.6% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.6% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=SECURITY_HOTSPOT) [2 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [10 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![2.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '2.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [2.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![3.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5-16px.png '3.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [3.6% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=SECURITY_HOTSPOT) [2 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [10 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![2.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '2.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [2.0% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![3.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5-16px.png '3.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [3.6% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    private TaskCacheUtils() {
+        throw new IllegalStateException("Utility class");
+    }
+
+    public static final String MERGE_TAG = "-";
+
+    /**
+     * generate cache key for task instance
+     * the follow message will be used to generate cache key
+     * 2. task version
+     * 3. task is cache
+     * 4. input VarPool, from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * @param taskExecutionContext taskExecutionContext
+     * @return cache key
+     */
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext taskExecutionContext) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPoolData(taskInstance, taskExecutionContext));
+        String data = StringUtils.join(keyElements, "_");
+        return DigestUtils.sha256Hex(data);
+    }
+
+    /**
+     * generate cache key for task instance which is cache execute
+     * this key will record which cache task instance will be copied, and cache key will be used
+     * tagCacheKey = sourceTaskId + "-" + cacheKey
+     * @param sourceTaskId source task id
+     * @param cacheKey cache key
+     * @return tagCacheKey
+     */
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    /**
+     * revert cache key tag to source task id and cache key
+     * @param tagCacheKey cache key
+     * @return Pair<Integer, String>, first is source task id, second is cache key
+     */
+    public static Pair<Integer, String> revertCacheKey(String tagCacheKey) {
+        Pair<Integer, String> taskIdAndCacheKey;
+        if (tagCacheKey == null) {
+            taskIdAndCacheKey = Pair.of(-1, "");
+            return taskIdAndCacheKey;
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            String[] split = tagCacheKey.split(MERGE_TAG);
+            if (split.length == 2) {
+                taskIdAndCacheKey = Pair.of(Integer.parseInt(split[0]), split[1]);
+            } else {
+                taskIdAndCacheKey = Pair.of(-1, "");
+            }
+            return taskIdAndCacheKey;
+        } else {
+            return Pair.of(-1, tagCacheKey);
+        }
+    }
+
+    /**
+     * get hash data of task input var pool
+     * there are two parts of task input var pool: from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * taskExecutionContext taskExecutionContext
+     */
+    public static String getTaskInputVarPoolData(TaskInstance taskInstance, TaskExecutionContext context) {
+        JsonNode taskParams = JSONUtils.parseObject(taskInstance.getTaskParams());
+
+        // The set of input values considered from localParams in the taskParams
+        Set<String> propertyInSet = JSONUtils.toList(taskParams.get("localParams").toString(), Property.class).stream()
+                .filter(property -> property.getDirect().equals(Direct.IN))
+                .map(Property::getProp).collect(Collectors.toSet());
+
+        // The set of input values considered from `${var}` form task definition
+        propertyInSet.addAll(getScriptVarInSet(taskInstance));

Review Comment:
   > Should we also consider environment variables, such as adding environment config as part of the cache key element?
   
   Is the `environment variables` mean  dolphinscheduler_env.sh?



-- 
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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    private TaskCacheUtils() {
+        throw new IllegalStateException("Utility class");
+    }
+
+    public static final String MERGE_TAG = "-";
+
+    /**
+     * generate cache key for task instance
+     * the follow message will be used to generate cache key
+     * 2. task version
+     * 3. task is cache
+     * 4. input VarPool, from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * @param taskExecutionContext taskExecutionContext
+     * @return cache key
+     */
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext taskExecutionContext) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPoolData(taskInstance, taskExecutionContext));
+        String data = StringUtils.join(keyElements, "_");
+        return DigestUtils.sha256Hex(data);
+    }
+
+    /**
+     * generate cache key for task instance which is cache execute
+     * this key will record which cache task instance will be copied, and cache key will be used
+     * tagCacheKey = sourceTaskId + "-" + cacheKey
+     * @param sourceTaskId source task id
+     * @param cacheKey cache key
+     * @return tagCacheKey
+     */
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    /**
+     * revert cache key tag to source task id and cache key
+     * @param tagCacheKey cache key
+     * @return Pair<Integer, String>, first is source task id, second is cache key
+     */
+    public static Pair<Integer, String> revertCacheKey(String tagCacheKey) {
+        Pair<Integer, String> taskIdAndCacheKey;
+        if (tagCacheKey == null) {
+            taskIdAndCacheKey = Pair.of(-1, "");
+            return taskIdAndCacheKey;
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            String[] split = tagCacheKey.split(MERGE_TAG);
+            if (split.length == 2) {
+                taskIdAndCacheKey = Pair.of(Integer.parseInt(split[0]), split[1]);
+            } else {
+                taskIdAndCacheKey = Pair.of(-1, "");
+            }
+            return taskIdAndCacheKey;
+        } else {
+            return Pair.of(-1, tagCacheKey);
+        }
+    }
+
+    /**
+     * get hash data of task input var pool
+     * there are two parts of task input var pool: from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * taskExecutionContext taskExecutionContext
+     */
+    public static String getTaskInputVarPoolData(TaskInstance taskInstance, TaskExecutionContext context) {
+        JsonNode taskParams = JSONUtils.parseObject(taskInstance.getTaskParams());
+
+        // The set of input values considered from localParams in the taskParams
+        Set<String> propertyInSet = JSONUtils.toList(taskParams.get("localParams").toString(), Property.class).stream()
+                .filter(property -> property.getDirect().equals(Direct.IN))
+                .map(Property::getProp).collect(Collectors.toSet());
+
+        // The set of input values considered from `${var}` form task definition
+        propertyInSet.addAll(getScriptVarInSet(taskInstance));

Review Comment:
   Dose `environment config` means the actual definition content about `environment_code `?
   If yes, we have not considered it in this pr.
   By the way, If some related resources are used by the task but change in the other places but don't change `taskDefinitionVersion` and `taskDefinition`, we will not catch them now. Such as the data using a fixed path or from the resource center.



-- 
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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.5% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.5% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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] github-code-scanning[bot] commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

Posted by GitBox <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13194:
URL: https://github.com/apache/dolphinscheduler/pull/13194#discussion_r1049349577


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    private TaskCacheUtils() {
+        throw new IllegalStateException("Utility class");
+    }
+
+    public static final String MERGE_TAG = "-";
+
+    /**
+     * generate cache key for task instance
+     * the follow message will be used to generate cache key
+     * 2. task version
+     * 3. task is cache
+     * 4. input VarPool, from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * @param taskExecutionContext taskExecutionContext
+     * @return cache key
+     */
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext taskExecutionContext) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPoolData(taskInstance, taskExecutionContext));
+        String data = StringUtils.join(keyElements, "_");
+        return DigestUtils.sha1Hex(data);
+    }
+
+    /**
+     * generate cache key for task instance which is cache execute
+     * this key will record which cache task instance will be copied, and cache key will be used
+     * tagCacheKey = sourceTaskId + "-" + cacheKey
+     * @param sourceTaskId source task id
+     * @param cacheKey cache key
+     * @return tagCacheKey
+     */
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    /**
+     * revert cache key tag to source task id and cache key
+     * @param tagCacheKey cache key
+     * @return Pair<Integer, String>, first is source task id, second is cache key
+     */
+    public static Pair<Integer, String> revertCacheKey(String tagCacheKey) {
+        Pair<Integer, String> taskIdAndCacheKey;
+        if (tagCacheKey == null) {
+            taskIdAndCacheKey = Pair.of(-1, "");
+            return taskIdAndCacheKey;
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            String[] split = tagCacheKey.split(MERGE_TAG);
+            taskIdAndCacheKey = Pair.of(Integer.valueOf(split[0]), split[1]);

Review Comment:
   ## Missing catch of NumberFormatException
   
   Potential uncaught 'java.lang.NumberFormatException'.
   
   [Show more details](https://github.com/apache/dolphinscheduler/security/code-scanning/2398)



-- 
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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.6% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    private TaskCacheUtils() {
+        throw new IllegalStateException("Utility class");
+    }
+
+    public static final String MERGE_TAG = "-";
+
+    /**
+     * generate cache key for task instance
+     * the follow message will be used to generate cache key
+     * 2. task version
+     * 3. task is cache
+     * 4. input VarPool, from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * @param taskExecutionContext taskExecutionContext
+     * @return cache key
+     */
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext taskExecutionContext) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPoolData(taskInstance, taskExecutionContext));
+        String data = StringUtils.join(keyElements, "_");
+        return DigestUtils.sha256Hex(data);
+    }
+
+    /**
+     * generate cache key for task instance which is cache execute
+     * this key will record which cache task instance will be copied, and cache key will be used
+     * tagCacheKey = sourceTaskId + "-" + cacheKey
+     * @param sourceTaskId source task id
+     * @param cacheKey cache key
+     * @return tagCacheKey
+     */
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    /**
+     * revert cache key tag to source task id and cache key
+     * @param tagCacheKey cache key
+     * @return Pair<Integer, String>, first is source task id, second is cache key
+     */
+    public static Pair<Integer, String> revertCacheKey(String tagCacheKey) {
+        Pair<Integer, String> taskIdAndCacheKey;
+        if (tagCacheKey == null) {
+            taskIdAndCacheKey = Pair.of(-1, "");
+            return taskIdAndCacheKey;
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            String[] split = tagCacheKey.split(MERGE_TAG);
+            if (split.length == 2) {
+                taskIdAndCacheKey = Pair.of(Integer.parseInt(split[0]), split[1]);
+            } else {
+                taskIdAndCacheKey = Pair.of(-1, "");
+            }
+            return taskIdAndCacheKey;
+        } else {
+            return Pair.of(-1, tagCacheKey);
+        }
+    }
+
+    /**
+     * get hash data of task input var pool
+     * there are two parts of task input var pool: from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * taskExecutionContext taskExecutionContext
+     */
+    public static String getTaskInputVarPoolData(TaskInstance taskInstance, TaskExecutionContext context) {
+        JsonNode taskParams = JSONUtils.parseObject(taskInstance.getTaskParams());
+
+        // The set of input values considered from localParams in the taskParams
+        Set<String> propertyInSet = JSONUtils.toList(taskParams.get("localParams").toString(), Property.class).stream()
+                .filter(property -> property.getDirect().equals(Direct.IN))
+                .map(Property::getProp).collect(Collectors.toSet());
+
+        // The set of input values considered from `${var}` form task definition
+        propertyInSet.addAll(getScriptVarInSet(taskInstance));

Review Comment:
   > Should we also consider environment variables, such as adding environment config as part of the cache key element?
   
   Is the `environment variables` mean  dolphinscheduler_env.sh or environment in the task definition?



-- 
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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.8%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.8% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.5% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    private TaskCacheUtils() {
+        throw new IllegalStateException("Utility class");
+    }
+
+    public static final String MERGE_TAG = "-";
+
+    /**
+     * generate cache key for task instance
+     * the follow message will be used to generate cache key
+     * 2. task version
+     * 3. task is cache
+     * 4. input VarPool, from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * @param taskExecutionContext taskExecutionContext
+     * @return cache key
+     */
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext taskExecutionContext) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPoolData(taskInstance, taskExecutionContext));
+        String data = StringUtils.join(keyElements, "_");
+        return DigestUtils.sha256Hex(data);
+    }
+
+    /**
+     * generate cache key for task instance which is cache execute
+     * this key will record which cache task instance will be copied, and cache key will be used
+     * tagCacheKey = sourceTaskId + "-" + cacheKey
+     * @param sourceTaskId source task id
+     * @param cacheKey cache key
+     * @return tagCacheKey
+     */
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    /**
+     * revert cache key tag to source task id and cache key
+     * @param tagCacheKey cache key
+     * @return Pair<Integer, String>, first is source task id, second is cache key
+     */
+    public static Pair<Integer, String> revertCacheKey(String tagCacheKey) {
+        Pair<Integer, String> taskIdAndCacheKey;
+        if (tagCacheKey == null) {
+            taskIdAndCacheKey = Pair.of(-1, "");
+            return taskIdAndCacheKey;
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            String[] split = tagCacheKey.split(MERGE_TAG);
+            if (split.length == 2) {
+                taskIdAndCacheKey = Pair.of(Integer.parseInt(split[0]), split[1]);
+            } else {
+                taskIdAndCacheKey = Pair.of(-1, "");
+            }
+            return taskIdAndCacheKey;
+        } else {
+            return Pair.of(-1, tagCacheKey);
+        }
+    }
+
+    /**
+     * get hash data of task input var pool
+     * there are two parts of task input var pool: from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * taskExecutionContext taskExecutionContext
+     */
+    public static String getTaskInputVarPoolData(TaskInstance taskInstance, TaskExecutionContext context) {
+        JsonNode taskParams = JSONUtils.parseObject(taskInstance.getTaskParams());
+
+        // The set of input values considered from localParams in the taskParams
+        Set<String> propertyInSet = JSONUtils.toList(taskParams.get("localParams").toString(), Property.class).stream()
+                .filter(property -> property.getDirect().equals(Direct.IN))
+                .map(Property::getProp).collect(Collectors.toSet());
+
+        // The set of input values considered from `${var}` form task definition
+        propertyInSet.addAll(getScriptVarInSet(taskInstance));

Review Comment:
   Dose `environment config` means the actual definition content about `environment_code `?
   If yes, we have not considered it in this pr



-- 
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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/3.2.0_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -119,3 +119,14 @@ d//
 delimiter ;
 select uc_dolphin_T_t_ds_task_instance_R_test_flag();
 DROP FUNCTION uc_dolphin_T_t_ds_task_instance_R_test_flag();
+
+ALTER TABLE t_ds_task_definition DROP COLUMN IF EXISTS is_cache;

Review Comment:
   Done



##########
dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql:
##########
@@ -894,7 +898,8 @@ CREATE TABLE `t_ds_task_instance` (
   `test_flag`  tinyint(4) DEFAULT null COMMENT 'test flag:0 normal, 1 test run',
   PRIMARY KEY (`id`),
   KEY `process_instance_id` (`process_instance_id`) USING BTREE,
-  KEY `idx_code_version` (`task_code`, `task_definition_version`) USING BTREE
+  KEY `idx_code_version` (`task_code`, `task_definition_version`) USING BTREE,
+  KEY `cache_key` (`cache_key`) USING BTREE

Review Comment:
   Done



-- 
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] github-code-scanning[bot] commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

Posted by GitBox <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13194:
URL: https://github.com/apache/dolphinscheduler/pull/13194#discussion_r1048428217


##########
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/TaskInstanceServiceImpl.java:
##########
@@ -319,4 +328,39 @@
         }
         return taskInstance;
     }
+
+    @Override
+    public TaskInstanceRemoveCacheResponse removeTaskInstanceCache(User loginUser, long projectCode,
+                                                                   Integer taskInstanceId) {
+        Result result = new Result();
+
+        Project project = projectMapper.queryByCode(projectCode);
+        projectService.checkProjectAndAuthThrowException(loginUser, project,
+                ApiFuncIdentificationConstant.map.get(INSTANCE_UPDATE));

Review Comment:
   ## Type mismatch on container access
   
   Actual argument type 'String' is incompatible with expected argument type 'ExecuteType'.
   
   [Show more details](https://github.com/apache/dolphinscheduler/security/code-scanning/2397)



##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,110 @@
+package org.apache.dolphinscheduler.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    final static String MERGE_TAG = "-";
+
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext context) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPool(taskInstance, context));
+        String data = StringUtils.join(keyElements, "_");
+        String cacheKey = md5(data);
+        System.out.println("cacheKey: " + cacheKey + ", data: " + data);
+        return cacheKey;
+    }
+
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    public static String revertCacheKey(String tagCacheKey) {
+        if (tagCacheKey == null) {
+            return "";
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            return tagCacheKey.split(MERGE_TAG)[1];
+        } else {
+            return tagCacheKey;
+        }
+    }
+
+    public static String md5(String data) {
+        try {
+            MessageDigest md = MessageDigest.getInstance("MD5");

Review Comment:
   ## Use of a broken or risky cryptographic algorithm
   
   Cryptographic algorithm [MD5](1) is weak and should not be used.
   
   [Show more details](https://github.com/apache/dolphinscheduler/security/code-scanning/2396)



-- 
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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [10 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![2.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '2.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [2.1% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![3.7%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5-16px.png '3.7%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [3.7% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [15 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![48.2%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/40-16px.png '48.2%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [48.2% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.8%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.8% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![59.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/50-16px.png '59.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [59.6% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    private TaskCacheUtils() {
+        throw new IllegalStateException("Utility class");
+    }
+
+    public static final String MERGE_TAG = "-";
+
+    /**
+     * generate cache key for task instance
+     * the follow message will be used to generate cache key
+     * 2. task version
+     * 3. task is cache
+     * 4. input VarPool, from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * @param taskExecutionContext taskExecutionContext
+     * @return cache key
+     */
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext taskExecutionContext) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPoolData(taskInstance, taskExecutionContext));
+        String data = StringUtils.join(keyElements, "_");
+        return DigestUtils.sha256Hex(data);
+    }
+
+    /**
+     * generate cache key for task instance which is cache execute
+     * this key will record which cache task instance will be copied, and cache key will be used
+     * tagCacheKey = sourceTaskId + "-" + cacheKey
+     * @param sourceTaskId source task id
+     * @param cacheKey cache key
+     * @return tagCacheKey
+     */
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    /**
+     * revert cache key tag to source task id and cache key
+     * @param tagCacheKey cache key
+     * @return Pair<Integer, String>, first is source task id, second is cache key
+     */
+    public static Pair<Integer, String> revertCacheKey(String tagCacheKey) {
+        Pair<Integer, String> taskIdAndCacheKey;
+        if (tagCacheKey == null) {
+            taskIdAndCacheKey = Pair.of(-1, "");
+            return taskIdAndCacheKey;
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            String[] split = tagCacheKey.split(MERGE_TAG);
+            if (split.length == 2) {
+                taskIdAndCacheKey = Pair.of(Integer.parseInt(split[0]), split[1]);
+            } else {
+                taskIdAndCacheKey = Pair.of(-1, "");
+            }
+            return taskIdAndCacheKey;
+        } else {
+            return Pair.of(-1, tagCacheKey);
+        }
+    }
+
+    /**
+     * get hash data of task input var pool
+     * there are two parts of task input var pool: from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * taskExecutionContext taskExecutionContext
+     */
+    public static String getTaskInputVarPoolData(TaskInstance taskInstance, TaskExecutionContext context) {
+        JsonNode taskParams = JSONUtils.parseObject(taskInstance.getTaskParams());
+
+        // The set of input values considered from localParams in the taskParams
+        Set<String> propertyInSet = JSONUtils.toList(taskParams.get("localParams").toString(), Property.class).stream()
+                .filter(property -> property.getDirect().equals(Direct.IN))
+                .map(Property::getProp).collect(Collectors.toSet());
+
+        // The set of input values considered from `${var}` form task definition
+        propertyInSet.addAll(getScriptVarInSet(taskInstance));

Review Comment:
   Any task definition-related changes(including environment code, but it doesn't include the dolphinscheduler_env.sh) have a task version to capture the changes



-- 
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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
docs/docs/zh/faq.md:
##########
@@ -720,4 +720,18 @@ A:在 3.0.0-alpha 版本之后,Python gateway server 集成到 api server 
 
 ---
 
+## Q: 缓存执行时怎么判断任务已经存在缓存过的任务,即如何判断一个任务可以使用另外一个任务的运行结果?
+
+A: 对于标识为`缓存执行`的任务, 当任务启动时会生成一个缓存key, 该key由以下字段组合哈希得到:
+
+- 任务定义:任务实例对应的任务定义的id
+- 任务的版本:任务实例对应的任务定义的版本
+- 任务输入的参数:包括上游节点和全局参数传入的参数中,被任务定义的参数列表所引用和任务定义中使用`${}`引用的参数
+
+当缓存标识的任务运行时,会查找数据库中是否用相同缓存key的数据,
+- 若有则复制该任务实例并进行相应数据的更新
+- 若无,则任务照常运行,并在任务完成时将任务实例的数据存入缓存
+
+若不需要缓存时,可以在工作流实例中右键运行清除缓存,则会清楚该版本下当前输入的参数的缓存数据。

Review Comment:
   thank, I have fixed it



-- 
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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/TaskCacheUtils.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.dao.utils;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
+import org.apache.dolphinscheduler.plugin.task.api.model.Property;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class TaskCacheUtils {
+
+    private TaskCacheUtils() {
+        throw new IllegalStateException("Utility class");
+    }
+
+    public static final String MERGE_TAG = "-";
+
+    /**
+     * generate cache key for task instance
+     * the follow message will be used to generate cache key
+     * 2. task version
+     * 3. task is cache
+     * 4. input VarPool, from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * @param taskExecutionContext taskExecutionContext
+     * @return cache key
+     */
+    public static String generateCacheKey(TaskInstance taskInstance, TaskExecutionContext taskExecutionContext) {
+        List<String> keyElements = new ArrayList<>();
+        keyElements.add(String.valueOf(taskInstance.getTaskCode()));
+        keyElements.add(String.valueOf(taskInstance.getTaskDefinitionVersion()));
+        keyElements.add(String.valueOf(taskInstance.getIsCache().getCode()));
+        keyElements.add(getTaskInputVarPoolData(taskInstance, taskExecutionContext));
+        String data = StringUtils.join(keyElements, "_");
+        return DigestUtils.sha256Hex(data);
+    }
+
+    /**
+     * generate cache key for task instance which is cache execute
+     * this key will record which cache task instance will be copied, and cache key will be used
+     * tagCacheKey = sourceTaskId + "-" + cacheKey
+     * @param sourceTaskId source task id
+     * @param cacheKey cache key
+     * @return tagCacheKey
+     */
+    public static String generateTagCacheKey(Integer sourceTaskId, String cacheKey) {
+        return sourceTaskId + MERGE_TAG + cacheKey;
+    }
+
+    /**
+     * revert cache key tag to source task id and cache key
+     * @param tagCacheKey cache key
+     * @return Pair<Integer, String>, first is source task id, second is cache key
+     */
+    public static Pair<Integer, String> revertCacheKey(String tagCacheKey) {
+        Pair<Integer, String> taskIdAndCacheKey;
+        if (tagCacheKey == null) {
+            taskIdAndCacheKey = Pair.of(-1, "");
+            return taskIdAndCacheKey;
+        }
+        if (tagCacheKey.contains(MERGE_TAG)) {
+            String[] split = tagCacheKey.split(MERGE_TAG);
+            if (split.length == 2) {
+                taskIdAndCacheKey = Pair.of(Integer.parseInt(split[0]), split[1]);
+            } else {
+                taskIdAndCacheKey = Pair.of(-1, "");
+            }
+            return taskIdAndCacheKey;
+        } else {
+            return Pair.of(-1, tagCacheKey);
+        }
+    }
+
+    /**
+     * get hash data of task input var pool
+     * there are two parts of task input var pool: from upstream task and workflow global parameters
+     * @param taskInstance task instance
+     * taskExecutionContext taskExecutionContext
+     */
+    public static String getTaskInputVarPoolData(TaskInstance taskInstance, TaskExecutionContext context) {
+        JsonNode taskParams = JSONUtils.parseObject(taskInstance.getTaskParams());
+
+        // The set of input values considered from localParams in the taskParams
+        Set<String> propertyInSet = JSONUtils.toList(taskParams.get("localParams").toString(), Property.class).stream()
+                .filter(property -> property.getDirect().equals(Direct.IN))
+                .map(Property::getProp).collect(Collectors.toSet());
+
+        // The set of input values considered from `${var}` form task definition
+        propertyInSet.addAll(getScriptVarInSet(taskInstance));

Review Comment:
   Yes, I have added `EnvironmentConfig` field to generate the cache key.



-- 
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] jieguangzhou commented on a diff in pull request #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java:
##########
@@ -187,6 +189,19 @@ public class TaskInstance implements Serializable {
      */
     private Flag flag;
 
+    /**
+     * task is cache: yes/no
+     */
+    private IsCache isCache;
+
+    /**
+     * task is cache: yes/no
+     */
+    @TableField(updateStrategy = FieldStrategy.IGNORED)

Review Comment:
   The default value of cache_key is `null`, if we remove the cache_key, we have to set it to `null`. So we have to ignore the update strategy, if not, the `null` will not be updated, that means cache data will not be removed.



-- 
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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   SonarCloud Quality Gate failed.&nbsp; &nbsp; [![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')](https://sonarcloud.io/dashboard?id=apache-dolphinscheduler&pullRequest=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [10 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![2.1%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '2.1%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [2.1% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![4.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/5-16px.png '4.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [4.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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 #13194: [Feature][Master] Add task caching mechanism to improve the running speed of repetitive tasks

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

   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=13194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&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=13194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=13194&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=13194&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=13194&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=13194&resolved=false&types=CODE_SMELL)
   
   [![63.6%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '63.6%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list) [63.6% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_coverage&view=list)  
   [![2.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.5%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&metric=new_duplicated_lines_density&view=list) [2.5% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=13194&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