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/08/23 10:18:17 UTC

[GitHub] [dolphinscheduler] ruanwenjun opened a new pull request, #11614: [Improvement-#11613] Add spi priority factory

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

   <!--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
   
   close #11613
   
   ## 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] ruanwenjun commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPI.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.spi.plugin;
+
+public interface PrioritySPI extends Comparable<Integer> {
+
+    SPIIdentify getIdentify();
+
+    /**
+     * Get the conflict strategy of the spi plugin, this will be used when there are existed two plugin has the same identify name.
+     *
+     * <p>
+     * The default strategy will use the high priority plugin.
+     */
+    default SPIConflictStrategy getConflictStrategy() {
+        return SPIConflictStrategy.LOAD_HIGH_PRIORITY;
+    }
+
+    @Override
+    default int compareTo(Integer o) {
+        return Integer.compare(getIdentify().getPriority(), o);
+    }
+
+    enum SPIConflictStrategy {

Review Comment:
   Yes, agree with you



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

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

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


[GitHub] [dolphinscheduler] ruanwenjun commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPI.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.spi.plugin;
+
+public interface PrioritySPI extends Comparable<Integer> {
+
+    SPIIdentify getIdentify();
+
+    /**
+     * Get the conflict strategy of the spi plugin, this will be used when there are existed two plugin has the same identify name.
+     *
+     * <p>
+     * The default strategy will use the high priority plugin.
+     */
+    default SPIConflictStrategy getConflictStrategy() {
+        return SPIConflictStrategy.LOAD_HIGH_PRIORITY;
+    }
+
+    @Override
+    default int compareTo(Integer o) {
+        return Integer.compare(getIdentify().getPriority(), o);
+    }
+
+    enum SPIConflictStrategy {

Review Comment:
   Thanks, I add the doc about plugin priority.



##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPI.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.spi.plugin;
+
+public interface PrioritySPI extends Comparable<Integer> {
+
+    SPIIdentify getIdentify();
+
+    /**
+     * Get the conflict strategy of the spi plugin, this will be used when there are existed two plugin has the same identify name.
+     *
+     * <p>
+     * The default strategy will use the high priority plugin.
+     */
+    default SPIConflictStrategy getConflictStrategy() {
+        return SPIConflictStrategy.LOAD_HIGH_PRIORITY;
+    }
+
+    @Override
+    default int compareTo(Integer o) {
+        return Integer.compare(getIdentify().getPriority(), o);
+    }
+
+    enum SPIConflictStrategy {

Review Comment:
   Thanks, I add the doc about plugin priority, please take a look.



-- 
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] kezhenxu94 commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPIFactory.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.spi.plugin;
+
+import lombok.extern.slf4j.Slf4j;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+@Slf4j
+public class PrioritySPIFactory<T extends PrioritySPI> {
+
+    private final Map<String, T> map = new HashMap<>();
+
+    public PrioritySPIFactory(Class<T> spiClass) {
+        for (T t : ServiceLoader.load(spiClass)) {
+            if (map.containsKey(t.getIdentify().getName())) {
+                resolveConflict(t);
+            } else {
+                map.put(t.getIdentify().getName(), t);
+            }
+        }
+    }
+
+    public Map<String, T> getSPIMap() {
+        return Collections.unmodifiableMap(map);
+    }
+
+    private void resolveConflict(T newSPI) {
+        SPIIdentify identify = newSPI.getIdentify();
+        T oldSPI = map.get(identify.getName());
+
+        if (newSPI.compareTo(oldSPI.getIdentify().getPriority()) == 0) {
+            throw new IllegalArgumentException(String.format("These two spi plugins has conflict identify name with the same priority: %s, %s",
+                    oldSPI.getIdentify(), newSPI.getIdentify()));
+        } else if (newSPI.compareTo(oldSPI.getIdentify().getPriority()) > 0) {
+            log.info("The {} plugin has high priority, will override {}", newSPI.getIdentify(), oldSPI);
+            map.put(identify.getName(), newSPI);
+        }

Review Comment:
   What about adding an `else` block and log "the low priority plugin is skipped", so when `newSPI.compareTo(oldSPI.getIdentify().getPriority()) < 0` users have clue why their plugin is not loaded?



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

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

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


[GitHub] [dolphinscheduler] ruanwenjun commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/task/TaskPluginManager.java:
##########
@@ -53,19 +49,19 @@ public void loadPlugin() {
             logger.warn("The task plugin has already been loaded");
             return;
         }
-        ServiceLoader.load(TaskChannelFactory.class).forEach(factory -> {
-            final String name = factory.getName();
+        PrioritySPIFactory<TaskChannelFactory> prioritySPIFactory = new PrioritySPIFactory<>(TaskChannelFactory.class);
+        for (Map.Entry<String, TaskChannelFactory> entry : prioritySPIFactory.getSPIMap().entrySet()) {
+            String factoryName = entry.getKey();
+            TaskChannelFactory factory = entry.getValue();
 
-            logger.info("Registering task plugin: {}", name);
+            logger.info("Registry task plugin: {} - {}", factoryName, factory.getClass());

Review Comment:
   We need to use `Registering` rather than `Registry`.



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

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

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


[GitHub] [dolphinscheduler] ruanwenjun commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPI.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.spi.plugin;
+
+public interface PrioritySPI extends Comparable<Integer> {
+
+    SPIIdentify getIdentify();
+
+    /**
+     * Get the conflict strategy of the spi plugin, this will be used when there are existed two plugin has the same identify name.
+     *
+     * <p>
+     * The default strategy will use the high priority plugin.
+     */
+    default SPIConflictStrategy getConflictStrategy() {
+        return SPIConflictStrategy.LOAD_HIGH_PRIORITY;
+    }
+
+    @Override
+    default int compareTo(Integer o) {
+        return Integer.compare(getIdentify().getPriority(), o);
+    }
+
+    enum SPIConflictStrategy {

Review Comment:
   I find we have missing the doc about SPI, I just add doc in the code, we need to add the introduction about our SPI, this should be another 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] caishunfeng commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/test/java/org/apache/dolphinscheduler/plugin/task/api/async/AsyncTaskExecutionContextTest.java:
##########
@@ -0,0 +1,21 @@
+package org.apache.dolphinscheduler.plugin.task.api.async;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.junit.Test;
+
+import java.util.Date;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.Delayed;
+
+public class AsyncTaskExecutionContextTest {
+
+    @Test
+    public void testNoTimeout() {

Review Comment:
   It seems not the same thing :eyes: 



##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPIFactory.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.spi.plugin;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+public class PrioritySPIFactory<T extends PrioritySPI> {
+
+    private final Map<String, T> map = new HashMap<>();
+
+    public PrioritySPIFactory(Class<T> spiClass) {
+        for (T t : ServiceLoader.load(spiClass)) {
+            if (map.containsKey(t.getIdentify().getName())) {
+                resolveConflict(t);
+            } else {
+                map.put(t.getIdentify().getName(), t);
+            }
+        }
+    }
+
+    public Map<String, T> getSpiMap() {

Review Comment:
   ```suggestion
       public Map<String, T> getSPIMap() {
   ```



##########
dolphinscheduler-spi/src/test/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPIFactoryTest.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.spi.plugin;
+
+import com.google.auto.service.AutoService;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Map;
+
+public class PrioritySPIFactoryTest {

Review Comment:
   :+1: 



##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPI.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.spi.plugin;
+
+public interface PrioritySPI extends Comparable<Integer> {
+
+    SPIIdentify getIdentify();
+
+    /**
+     * Get the conflict strategy of the spi plugin, this will be used when there are existed two plugin has the same identify name.
+     *
+     * <p>
+     * The default strategy will use the high priority plugin.
+     */
+    default SPIConflictStrategy getConflictStrategy() {
+        return SPIConflictStrategy.LOAD_HIGH_PRIORITY;
+    }
+
+    @Override
+    default int compareTo(Integer o) {
+        return Integer.compare(getIdentify().getPriority(), o);
+    }
+
+    enum SPIConflictStrategy {

Review Comment:
   Need to add doc about the SPI load rule.



##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPI.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.spi.plugin;
+
+public interface PrioritySPI extends Comparable<Integer> {
+
+    SPIIdentify getIdentify();
+
+    /**
+     * Get the conflict strategy of the spi plugin, this will be used when there are existed two plugin has the same identify name.
+     *
+     * <p>
+     * The default strategy will use the high priority plugin.
+     */
+    default SPIConflictStrategy getConflictStrategy() {
+        return SPIConflictStrategy.LOAD_HIGH_PRIORITY;
+    }
+
+    @Override
+    default int compareTo(Integer o) {
+        return Integer.compare(getIdentify().getPriority(), o);
+    }
+
+    enum SPIConflictStrategy {

Review Comment:
   I think we just support the `LOAD_HIGH_PRIORITY` strategy is enough, if two plugin has same priority, throw exception. Which can make a more clear load rule and make users understand earsier.



-- 
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 #11614: [Improvement-#11613] Add spi priority factory

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


-- 
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] kezhenxu94 commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPIFactory.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.spi.plugin;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+public class PrioritySPIFactory<T extends PrioritySPI> {
+
+    private final Logger logger = LoggerFactory.getLogger(PrioritySPIFactory.class);
+
+    private final Map<String, T> map = new HashMap<>();
+
+    public PrioritySPIFactory(Class<T> spiClass) {
+        for (T t : ServiceLoader.load(spiClass)) {
+            if (map.containsKey(t.getIdentify().getName())) {
+                resolveConflict(t);
+            } else {
+                map.put(t.getIdentify().getName(), t);
+            }
+        }
+    }
+
+    public Map<String, T> getSPIMap() {
+        return map;

Review Comment:
   Should return an immutable map
   
   ```suggestion
           return Collections.unmodifiableMap(map);
   ```
   



##########
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/task/TaskPluginManager.java:
##########
@@ -53,19 +49,19 @@ public void loadPlugin() {
             logger.warn("The task plugin has already been loaded");
             return;
         }
-        ServiceLoader.load(TaskChannelFactory.class).forEach(factory -> {
-            final String name = factory.getName();
+        PrioritySPIFactory<TaskChannelFactory> prioritySPIFactory = new PrioritySPIFactory<>(TaskChannelFactory.class);
+        for (Map.Entry<String, TaskChannelFactory> entry : prioritySPIFactory.getSPIMap().entrySet()) {
+            String factoryName = entry.getKey();
+            TaskChannelFactory factory = entry.getValue();
 
-            logger.info("Registering task plugin: {}", name);
+            logger.info("Registry task plugin: {} - {}", factoryName, factory.getClass());

Review Comment:
   Registry is a noun but here we want a verb, no?



##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/SPIIdentify.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.spi.plugin;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+@Data
+@AllArgsConstructor
+public class SPIIdentify {
+
+    private static final int DEFAULT_PRIORITY = 0;
+
+    private String name;
+
+    private int priority = DEFAULT_PRIORITY;
+
+    public static SPIIdentify create(String name, int priority) {
+        return new SPIIdentify(name, priority);
+    }
+
+    public static SPIIdentify create(String name) {
+        return create(name, DEFAULT_PRIORITY);
+    }
+
+}

Review Comment:
   ```suggestion
   @Data
   @lombok.Builder
   @AllArgsConstructor
   public class SPIIdentify {
   
       private static final int DEFAULT_PRIORITY = 0;
   
       private String name;
   
       @lombok.Builder.Default 
       private int priority = DEFAULT_PRIORITY;
   }
   ```
   



##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPIFactory.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.spi.plugin;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+public class PrioritySPIFactory<T extends PrioritySPI> {
+
+    private final Logger logger = LoggerFactory.getLogger(PrioritySPIFactory.class);

Review Comment:
   ```suggestion
   @lombok.Slf4j
   public class PrioritySPIFactory<T extends PrioritySPI> {
   ```
   



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

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

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


[GitHub] [dolphinscheduler] ruanwenjun commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPIFactory.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.spi.plugin;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+public class PrioritySPIFactory<T extends PrioritySPI> {
+
+    private final Logger logger = LoggerFactory.getLogger(PrioritySPIFactory.class);

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] ruanwenjun commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPIFactory.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.spi.plugin;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+public class PrioritySPIFactory<T extends PrioritySPI> {

Review Comment:
   What?



-- 
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] MichaelDeSteven commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPIFactory.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.spi.plugin;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+public class PrioritySPIFactory<T extends PrioritySPI> {
+
+    private final Map<String, T> map = new HashMap<>();
+
+    public PrioritySPIFactory(Class<T> spiClass) {
+        for (T t : ServiceLoader.load(spiClass)) {
+            if (map.containsKey(t.getIdentify().getName())) {
+                resolveConflict(t);
+            } else {
+                map.put(t.getIdentify().getName(), t);
+            }
+        }
+    }
+
+    public Map<String, T> getSPIMap() {
+        return map;
+    }
+
+    private void resolveConflict(T newSPI) {
+        SPIIdentify identify = newSPI.getIdentify();
+        T oldSPI = map.get(identify.getName());
+
+        if (newSPI.compareTo(oldSPI.getIdentify().getPriority()) == 0) {
+            throw new IllegalArgumentException(String.format("These two spi plugins has conflict identify name with the same priority: %s, %s",
+                    oldSPI.getIdentify(), newSPI.getIdentify()));
+        } else if (newSPI.compareTo(oldSPI.getIdentify().getPriority()) > 0) {
+            map.put(identify.getName(), newSPI);

Review Comment:
   Maybe logging the replace class info is better? 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] WangJPLeo commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPIFactory.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.spi.plugin;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+public class PrioritySPIFactory<T extends PrioritySPI> {

Review Comment:
   Priority public components, worshipping.



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

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

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


[GitHub] [dolphinscheduler] ruanwenjun commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/SPIIdentify.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.spi.plugin;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+@Data
+@AllArgsConstructor
+public class SPIIdentify {
+
+    private static final int DEFAULT_PRIORITY = 0;
+
+    private String name;
+
+    private int priority = DEFAULT_PRIORITY;
+
+    public static SPIIdentify create(String name, int priority) {
+        return new SPIIdentify(name, priority);
+    }
+
+    public static SPIIdentify create(String name) {
+        return create(name, DEFAULT_PRIORITY);
+    }
+
+}

Review Comment:
   Done



##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPIFactory.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.spi.plugin;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+public class PrioritySPIFactory<T extends PrioritySPI> {
+
+    private final Logger logger = LoggerFactory.getLogger(PrioritySPIFactory.class);
+
+    private final Map<String, T> map = new HashMap<>();
+
+    public PrioritySPIFactory(Class<T> spiClass) {
+        for (T t : ServiceLoader.load(spiClass)) {
+            if (map.containsKey(t.getIdentify().getName())) {
+                resolveConflict(t);
+            } else {
+                map.put(t.getIdentify().getName(), t);
+            }
+        }
+    }
+
+    public Map<String, T> getSPIMap() {
+        return map;

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] caishunfeng commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPI.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.spi.plugin;
+
+public interface PrioritySPI extends Comparable<Integer> {
+
+    SPIIdentify getIdentify();
+
+    /**
+     * Get the conflict strategy of the spi plugin, this will be used when there are existed two plugin has the same identify name.
+     *
+     * <p>
+     * The default strategy will use the high priority plugin.
+     */
+    default SPIConflictStrategy getConflictStrategy() {
+        return SPIConflictStrategy.LOAD_HIGH_PRIORITY;
+    }
+
+    @Override
+    default int compareTo(Integer o) {
+        return Integer.compare(getIdentify().getPriority(), o);
+    }
+
+    enum SPIConflictStrategy {

Review Comment:
   The SPI doc had exist,see https://dolphinscheduler.apache.org/zh-cn/docs/latest/user_doc/contribute/backend/spi/task.html



-- 
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 #11614: [Improvement-#11613] Add spi priority factory

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

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


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

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

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


[GitHub] [dolphinscheduler] ruanwenjun commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPIFactory.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.spi.plugin;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+public class PrioritySPIFactory<T extends PrioritySPI> {
+
+    private final Map<String, T> map = new HashMap<>();
+
+    public PrioritySPIFactory(Class<T> spiClass) {
+        for (T t : ServiceLoader.load(spiClass)) {
+            if (map.containsKey(t.getIdentify().getName())) {
+                resolveConflict(t);
+            } else {
+                map.put(t.getIdentify().getName(), t);
+            }
+        }
+    }
+
+    public Map<String, T> getSPIMap() {
+        return map;
+    }
+
+    private void resolveConflict(T newSPI) {
+        SPIIdentify identify = newSPI.getIdentify();
+        T oldSPI = map.get(identify.getName());
+
+        if (newSPI.compareTo(oldSPI.getIdentify().getPriority()) == 0) {
+            throw new IllegalArgumentException(String.format("These two spi plugins has conflict identify name with the same priority: %s, %s",
+                    oldSPI.getIdentify(), newSPI.getIdentify()));
+        } else if (newSPI.compareTo(oldSPI.getIdentify().getPriority()) > 0) {
+            map.put(identify.getName(), newSPI);

Review Comment:
   Good catch, agree with you, have add log here, please take a look.



-- 
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 #11614: [Improvement-#11613] Add spi priority factory

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

   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=11614)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11614&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=11614&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11614&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=11614&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=11614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11614&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=11614&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=11614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache-dolphinscheduler&pullRequest=11614&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=11614&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=11614&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache-dolphinscheduler&pullRequest=11614&resolved=false&types=CODE_SMELL)
   
   [![66.2%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '66.2%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11614&metric=new_coverage&view=list) [66.2% Coverage](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache-dolphinscheduler&pullRequest=11614&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 #11614: [Improvement-#11613] Add spi priority factory

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

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


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

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

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


[GitHub] [dolphinscheduler] ruanwenjun commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-task-plugin/dolphinscheduler-task-api/src/test/java/org/apache/dolphinscheduler/plugin/task/api/async/AsyncTaskExecutionContextTest.java:
##########
@@ -0,0 +1,21 @@
+package org.apache.dolphinscheduler.plugin.task.api.async;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
+import org.junit.Test;
+
+import java.util.Date;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.Delayed;
+
+public class AsyncTaskExecutionContextTest {
+
+    @Test
+    public void testNoTimeout() {

Review Comment:
   Has removed this.



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

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

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


[GitHub] [dolphinscheduler] ruanwenjun commented on a diff in pull request #11614: [Improvement-#11613] Add spi priority factory

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


##########
dolphinscheduler-spi/src/main/java/org/apache/dolphinscheduler/spi/plugin/PrioritySPIFactory.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.spi.plugin;
+
+import lombok.extern.slf4j.Slf4j;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+@Slf4j
+public class PrioritySPIFactory<T extends PrioritySPI> {
+
+    private final Map<String, T> map = new HashMap<>();
+
+    public PrioritySPIFactory(Class<T> spiClass) {
+        for (T t : ServiceLoader.load(spiClass)) {
+            if (map.containsKey(t.getIdentify().getName())) {
+                resolveConflict(t);
+            } else {
+                map.put(t.getIdentify().getName(), t);
+            }
+        }
+    }
+
+    public Map<String, T> getSPIMap() {
+        return Collections.unmodifiableMap(map);
+    }
+
+    private void resolveConflict(T newSPI) {
+        SPIIdentify identify = newSPI.getIdentify();
+        T oldSPI = map.get(identify.getName());
+
+        if (newSPI.compareTo(oldSPI.getIdentify().getPriority()) == 0) {
+            throw new IllegalArgumentException(String.format("These two spi plugins has conflict identify name with the same priority: %s, %s",
+                    oldSPI.getIdentify(), newSPI.getIdentify()));
+        } else if (newSPI.compareTo(oldSPI.getIdentify().getPriority()) > 0) {
+            log.info("The {} plugin has high priority, will override {}", newSPI.getIdentify(), oldSPI);
+            map.put(identify.getName(), newSPI);
+        }

Review Comment:
   Agree with you.



-- 
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 #11614: [Improvement-#11613] Add spi priority factory

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

   # [Codecov](https://codecov.io/gh/apache/dolphinscheduler/pull/11614?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 [#11614](https://codecov.io/gh/apache/dolphinscheduler/pull/11614?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2db216b) into [dev](https://codecov.io/gh/apache/dolphinscheduler/commit/abfef1a929b0b28f2e5bce4f52a571fe33a6fe86?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (abfef1a) will **increase** coverage by `0.06%`.
   > The diff coverage is `51.20%`.
   
   > :exclamation: Current head 2db216b differs from pull request most recent head 0e219b0. Consider uploading reports for the commit 0e219b0 to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##                dev   #11614      +/-   ##
   ============================================
   + Coverage     39.78%   39.84%   +0.06%     
   - Complexity     4697     4708      +11     
   ============================================
     Files           991      997       +6     
     Lines         37703    37743      +40     
     Branches       4194     4200       +6     
   ============================================
   + Hits          14999    15039      +40     
     Misses        21138    21138              
     Partials       1566     1566              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/dolphinscheduler/pull/11614?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...datasource/api/plugin/DataSourcePluginManager.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11614/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-ZG9scGhpbnNjaGVkdWxlci1kYXRhc291cmNlLXBsdWdpbi9kb2xwaGluc2NoZWR1bGVyLWRhdGFzb3VyY2UtYXBpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9kb2xwaGluc2NoZWR1bGVyL3BsdWdpbi9kYXRhc291cmNlL2FwaS9wbHVnaW4vRGF0YVNvdXJjZVBsdWdpbk1hbmFnZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...e/dolphinscheduler/server/master/MasterServer.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11614/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-ZG9scGhpbnNjaGVkdWxlci1tYXN0ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvc2VydmVyL21hc3Rlci9NYXN0ZXJTZXJ2ZXIuamF2YQ==) | `0.00% <ø> (ø)` | |
   | [...lphinscheduler/service/task/TaskPluginManager.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11614/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-ZG9scGhpbnNjaGVkdWxlci1zZXJ2aWNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9kb2xwaGluc2NoZWR1bGVyL3NlcnZpY2UvdGFzay9UYXNrUGx1Z2luTWFuYWdlci5qYXZh) | `3.33% <0.00%> (ø)` | |
   | [...duler/spi/datasource/DataSourceChannelFactory.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11614/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-ZG9scGhpbnNjaGVkdWxlci1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvc3BpL2RhdGFzb3VyY2UvRGF0YVNvdXJjZUNoYW5uZWxGYWN0b3J5LmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...nscheduler/plugin/task/api/TaskChannelFactory.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11614/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-ZG9scGhpbnNjaGVkdWxlci10YXNrLXBsdWdpbi9kb2xwaGluc2NoZWR1bGVyLXRhc2stYXBpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9kb2xwaGluc2NoZWR1bGVyL3BsdWdpbi90YXNrL2FwaS9UYXNrQ2hhbm5lbEZhY3RvcnkuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...r/api/service/impl/ProcessInstanceServiceImpl.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11614/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL3NlcnZpY2UvaW1wbC9Qcm9jZXNzSW5zdGFuY2VTZXJ2aWNlSW1wbC5qYXZh) | `60.85% <23.07%> (-0.90%)` | :arrow_down: |
   | [...api/service/impl/ProcessDefinitionServiceImpl.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11614/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-ZG9scGhpbnNjaGVkdWxlci1hcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvYXBpL3NlcnZpY2UvaW1wbC9Qcm9jZXNzRGVmaW5pdGlvblNlcnZpY2VJbXBsLmphdmE=) | `32.32% <29.41%> (+0.05%)` | :arrow_up: |
   | [...er/server/master/registry/MasterHeartBeatTask.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11614/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-ZG9scGhpbnNjaGVkdWxlci1tYXN0ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvc2VydmVyL21hc3Rlci9yZWdpc3RyeS9NYXN0ZXJIZWFydEJlYXRUYXNrLmphdmE=) | `44.44% <44.44%> (ø)` | |
   | [...pache/dolphinscheduler/spi/plugin/SPIIdentify.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11614/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-ZG9scGhpbnNjaGVkdWxlci1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvc3BpL3BsdWdpbi9TUElJZGVudGlmeS5qYXZh) | `50.00% <50.00%> (ø)` | |
   | [...erver/master/runner/task/TaskProcessorFactory.java](https://codecov.io/gh/apache/dolphinscheduler/pull/11614/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-ZG9scGhpbnNjaGVkdWxlci1tYXN0ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2RvbHBoaW5zY2hlZHVsZXIvc2VydmVyL21hc3Rlci9ydW5uZXIvdGFzay9UYXNrUHJvY2Vzc29yRmFjdG9yeS5qYXZh) | `57.89% <85.71%> (+7.89%)` | :arrow_up: |
   | ... and [25 more](https://codecov.io/gh/apache/dolphinscheduler/pull/11614/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