You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by te...@apache.org on 2020/10/15 10:49:45 UTC

[shardingsphere-elasticjob] branch master updated: Refactor JobItemExecutorFactory (#1558)

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

technoboy pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere-elasticjob.git


The following commit(s) were added to refs/heads/master by this push:
     new 77d49ff  Refactor JobItemExecutorFactory (#1558)
77d49ff is described below

commit 77d49ffae7f546b5d155cbe5784cb056f95d01a8
Author: wwj <22...@qq.com>
AuthorDate: Thu Oct 15 18:49:31 2020 +0800

    Refactor JobItemExecutorFactory (#1558)
    
    * Refactor JobItemExecutorFactory, uniformly use ElasticJobServiceLoader to load TypedSPI implementation classes. (#1557)
---
 .../executor/item/JobItemExecutorFactory.java      | 26 +++++-----------------
 ...cjob.executor.item.impl.ClassedJobItemExecutor} |  1 -
 ...ticjob.executor.item.impl.TypedJobItemExecutor} |  3 +--
 ...cjob.executor.item.impl.ClassedJobItemExecutor} |  0
 ...ticjob.executor.item.impl.TypedJobItemExecutor} |  0
 ...ticjob.executor.item.impl.TypedJobItemExecutor} |  0
 ...cjob.executor.item.impl.ClassedJobItemExecutor} |  0
 ...cjob.executor.item.impl.ClassedJobItemExecutor} |  0
 ...cjob.executor.item.impl.ClassedJobItemExecutor} |  1 -
 ...sticjob.executor.item.impl.TypedJobItemExecutor |  5 ++---
 10 files changed, 9 insertions(+), 27 deletions(-)

diff --git a/elasticjob-executor/elasticjob-executor-kernel/src/main/java/org/apache/shardingsphere/elasticjob/executor/item/JobItemExecutorFactory.java b/elasticjob-executor/elasticjob-executor-kernel/src/main/java/org/apache/shardingsphere/elasticjob/executor/item/JobItemExecutorFactory.java
index 82029ba..2d758f3 100644
--- a/elasticjob-executor/elasticjob-executor-kernel/src/main/java/org/apache/shardingsphere/elasticjob/executor/item/JobItemExecutorFactory.java
+++ b/elasticjob-executor/elasticjob-executor-kernel/src/main/java/org/apache/shardingsphere/elasticjob/executor/item/JobItemExecutorFactory.java
@@ -23,11 +23,12 @@ import org.apache.shardingsphere.elasticjob.api.ElasticJob;
 import org.apache.shardingsphere.elasticjob.infra.exception.JobConfigurationException;
 import org.apache.shardingsphere.elasticjob.executor.item.impl.ClassedJobItemExecutor;
 import org.apache.shardingsphere.elasticjob.executor.item.impl.TypedJobItemExecutor;
+import org.apache.shardingsphere.elasticjob.infra.spi.ElasticJobServiceLoader;
 
+import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.ServiceLoader;
-import java.util.concurrent.ConcurrentHashMap;
 
 /**
  * Job item executor factory.
@@ -35,21 +36,11 @@ import java.util.concurrent.ConcurrentHashMap;
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
 public final class JobItemExecutorFactory {
     
-    private static final Map<Class, ClassedJobItemExecutor> CLASSED_EXECUTORS = new ConcurrentHashMap<>();
-    
-    private static final Map<String, TypedJobItemExecutor> TYPED_EXECUTORS = new ConcurrentHashMap<>();
+    private static final Map<Class, ClassedJobItemExecutor> CLASSED_EXECUTORS = new HashMap<>();
     
     static {
-        for (JobItemExecutor each : ServiceLoader.load(JobItemExecutor.class)) {
-            if (each instanceof ClassedJobItemExecutor) {
-                ClassedJobItemExecutor typedJobItemExecutor = (ClassedJobItemExecutor) each;
-                CLASSED_EXECUTORS.put(typedJobItemExecutor.getElasticJobClass(), typedJobItemExecutor);
-            }
-            if (each instanceof TypedJobItemExecutor) {
-                TypedJobItemExecutor typedJobItemExecutor = (TypedJobItemExecutor) each;
-                TYPED_EXECUTORS.put(typedJobItemExecutor.getType(), typedJobItemExecutor);
-            }
-        }
+        ElasticJobServiceLoader.registerTypedService(TypedJobItemExecutor.class);
+        ServiceLoader.load(ClassedJobItemExecutor.class).forEach(each -> CLASSED_EXECUTORS.put(each.getElasticJobClass(), each));
     }
     
     /**
@@ -75,11 +66,6 @@ public final class JobItemExecutorFactory {
      * @return job item executor
      */
     public static JobItemExecutor getExecutor(final String elasticJobType) {
-        for (Entry<String, TypedJobItemExecutor> entry : TYPED_EXECUTORS.entrySet()) {
-            if (entry.getKey().equals(elasticJobType)) {
-                return entry.getValue();
-            }
-        }
-        throw new JobConfigurationException("Can not find executor for elastic job type `%s`", elasticJobType);
+        return ElasticJobServiceLoader.getCachedInstance(TypedJobItemExecutor.class, elasticJobType);
     }
 }
diff --git a/elasticjob-executor/elasticjob-executor-kernel/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor b/elasticjob-executor/elasticjob-executor-kernel/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.ClassedJobItemExecutor
similarity index 91%
copy from elasticjob-executor/elasticjob-executor-kernel/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor
copy to elasticjob-executor/elasticjob-executor-kernel/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.ClassedJobItemExecutor
index 1fbaddb..19f0f28 100644
--- a/elasticjob-executor/elasticjob-executor-kernel/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor
+++ b/elasticjob-executor/elasticjob-executor-kernel/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.ClassedJobItemExecutor
@@ -16,4 +16,3 @@
 #
 
 org.apache.shardingsphere.elasticjob.executor.fixture.executor.ClassedFooJobExecutor
-org.apache.shardingsphere.elasticjob.executor.fixture.executor.TypedFooJobExecutor
diff --git a/elasticjob-executor/elasticjob-executor-kernel/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor b/elasticjob-executor/elasticjob-executor-kernel/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.TypedJobItemExecutor
similarity index 90%
rename from elasticjob-executor/elasticjob-executor-kernel/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor
rename to elasticjob-executor/elasticjob-executor-kernel/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.TypedJobItemExecutor
index 1fbaddb..9fde771 100644
--- a/elasticjob-executor/elasticjob-executor-kernel/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor
+++ b/elasticjob-executor/elasticjob-executor-kernel/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.TypedJobItemExecutor
@@ -7,7 +7,7 @@
 # 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.
@@ -15,5 +15,4 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.elasticjob.executor.fixture.executor.ClassedFooJobExecutor
 org.apache.shardingsphere.elasticjob.executor.fixture.executor.TypedFooJobExecutor
diff --git a/elasticjob-executor/elasticjob-executor-type/elasticjob-dataflow-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor b/elasticjob-executor/elasticjob-executor-type/elasticjob-dataflow-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.ClassedJobItemExecutor
similarity index 100%
rename from elasticjob-executor/elasticjob-executor-type/elasticjob-dataflow-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor
rename to elasticjob-executor/elasticjob-executor-type/elasticjob-dataflow-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.ClassedJobItemExecutor
diff --git a/elasticjob-executor/elasticjob-executor-type/elasticjob-http-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor b/elasticjob-executor/elasticjob-executor-type/elasticjob-http-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.TypedJobItemExecutor
similarity index 100%
rename from elasticjob-executor/elasticjob-executor-type/elasticjob-http-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor
rename to elasticjob-executor/elasticjob-executor-type/elasticjob-http-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.TypedJobItemExecutor
diff --git a/elasticjob-executor/elasticjob-executor-type/elasticjob-script-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor b/elasticjob-executor/elasticjob-executor-type/elasticjob-script-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.TypedJobItemExecutor
similarity index 100%
copy from elasticjob-executor/elasticjob-executor-type/elasticjob-script-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor
copy to elasticjob-executor/elasticjob-executor-type/elasticjob-script-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.TypedJobItemExecutor
diff --git a/elasticjob-executor/elasticjob-executor-type/elasticjob-simple-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor b/elasticjob-executor/elasticjob-executor-type/elasticjob-simple-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.ClassedJobItemExecutor
similarity index 100%
rename from elasticjob-executor/elasticjob-executor-type/elasticjob-simple-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor
rename to elasticjob-executor/elasticjob-executor-type/elasticjob-simple-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.ClassedJobItemExecutor
diff --git a/elasticjob-lite/elasticjob-lite-core/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor b/elasticjob-lite/elasticjob-lite-core/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.ClassedJobItemExecutor
similarity index 100%
rename from elasticjob-lite/elasticjob-lite-core/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor
rename to elasticjob-lite/elasticjob-lite-core/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.ClassedJobItemExecutor
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.ClassedJobItemExecutor
similarity index 91%
rename from elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor
rename to elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.ClassedJobItemExecutor
index 1492fe4..326c084 100644
--- a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.ClassedJobItemExecutor
@@ -15,4 +15,3 @@
 # limitations under the License.
 #
 org.apache.shardingsphere.elasticjob.lite.spring.boot.job.executor.CustomClassedJobExecutor
-org.apache.shardingsphere.elasticjob.lite.spring.boot.job.executor.PrintJobExecutor
diff --git a/elasticjob-executor/elasticjob-executor-type/elasticjob-script-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.TypedJobItemExecutor
similarity index 90%
rename from elasticjob-executor/elasticjob-executor-type/elasticjob-script-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor
rename to elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.TypedJobItemExecutor
index 9226a56..a113f2b 100644
--- a/elasticjob-executor/elasticjob-executor-type/elasticjob-script-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.JobItemExecutor
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/resources/META-INF/services/org.apache.shardingsphere.elasticjob.executor.item.impl.TypedJobItemExecutor
@@ -7,12 +7,11 @@
 # 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.
 #
-
-org.apache.shardingsphere.elasticjob.script.executor.ScriptJobExecutor
+org.apache.shardingsphere.elasticjob.lite.spring.boot.job.executor.PrintJobExecutor