You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by wu...@apache.org on 2022/04/13 21:57:14 UTC

[shardingsphere] branch master updated: Add ExecuteProcessReporterFactory (#16804)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new e6a26844d7b Add ExecuteProcessReporterFactory (#16804)
e6a26844d7b is described below

commit e6a26844d7b83aff51c887eaeedd865867f4639e
Author: Liang Zhang <zh...@apache.org>
AuthorDate: Thu Apr 14 05:56:55 2022 +0800

    Add ExecuteProcessReporterFactory (#16804)
---
 .../executor/sql/process/ExecuteProcessEngine.java | 31 +++++++--------
 .../sql/process/spi/ExecuteProcessReporter.java    |  5 ++-
 .../process/spi/ExecuteProcessReporterFactory.java | 45 ++++++++++++++++++++++
 .../fixture/ExecuteProcessReporterFixture.java     | 10 ++---
 4 files changed, 67 insertions(+), 24 deletions(-)

diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/ExecuteProcessEngine.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/ExecuteProcessEngine.java
index 364538f1f25..d7a0baa5d56 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/ExecuteProcessEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/ExecuteProcessEngine.java
@@ -27,9 +27,9 @@ import org.apache.shardingsphere.infra.executor.kernel.model.ExecutorDataMap;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.SQLExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.process.model.ExecuteProcessConstants;
 import org.apache.shardingsphere.infra.executor.sql.process.spi.ExecuteProcessReporter;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.infra.executor.sql.process.spi.ExecuteProcessReporterFactory;
 
-import java.util.Collection;
+import java.util.Optional;
 
 /**
  * Execute process engine.
@@ -37,13 +37,6 @@ import java.util.Collection;
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
 public final class ExecuteProcessEngine {
     
-    private static final Collection<ExecuteProcessReporter> HANDLERS;
-    
-    static {
-        ShardingSphereServiceLoader.register(ExecuteProcessReporter.class);
-        HANDLERS = ShardingSphereServiceLoader.getServiceInstances(ExecuteProcessReporter.class);
-    }
-    
     /**
      * Initialize.
      *
@@ -53,9 +46,10 @@ public final class ExecuteProcessEngine {
      */
     public static void initialize(final LogicSQL logicSQL, final ExecutionGroupContext<? extends SQLExecutionUnit> executionGroupContext, final ConfigurationProperties props) {
         SQLStatementContext<?> context = logicSQL.getSqlStatementContext();
-        if (!HANDLERS.isEmpty() && ExecuteProcessStrategyEvaluator.evaluate(context, executionGroupContext, props)) {
+        Optional<ExecuteProcessReporter> reporter = ExecuteProcessReporterFactory.newInstance();
+        if (reporter.isPresent() && ExecuteProcessStrategyEvaluator.evaluate(context, executionGroupContext, props)) {
             ExecutorDataMap.getValue().put(ExecuteProcessConstants.EXECUTE_ID.name(), executionGroupContext.getExecutionID());
-            HANDLERS.iterator().next().report(logicSQL, executionGroupContext, ExecuteProcessConstants.EXECUTE_STATUS_START);
+            reporter.get().report(logicSQL, executionGroupContext, ExecuteProcessConstants.EXECUTE_STATUS_START);
         }
     }
     
@@ -63,8 +57,9 @@ public final class ExecuteProcessEngine {
      * Clean.
      */
     public static void clean() {
-        if (!HANDLERS.isEmpty() && ExecutorDataMap.getValue().containsKey(ExecuteProcessConstants.EXECUTE_ID.name())) {
-            HANDLERS.iterator().next().reportClean(ExecutorDataMap.getValue().get(ExecuteProcessConstants.EXECUTE_ID.name()).toString());
+        Optional<ExecuteProcessReporter> reporter = ExecuteProcessReporterFactory.newInstance();
+        if (reporter.isPresent() && ExecutorDataMap.getValue().containsKey(ExecuteProcessConstants.EXECUTE_ID.name())) {
+            reporter.get().reportClean(ExecutorDataMap.getValue().get(ExecuteProcessConstants.EXECUTE_ID.name()).toString());
         }
         ExecutorDataMap.getValue().remove(ExecuteProcessConstants.EXECUTE_ID.name());
     }
@@ -76,9 +71,8 @@ public final class ExecuteProcessEngine {
      * @param executionUnit execution unit
      */
     public static void finish(final String executionID, final SQLExecutionUnit executionUnit) {
-        if (!HANDLERS.isEmpty()) {
-            HANDLERS.iterator().next().report(executionID, executionUnit, ExecuteProcessConstants.EXECUTE_STATUS_DONE);
-        }
+        Optional<ExecuteProcessReporter> reporter = ExecuteProcessReporterFactory.newInstance();
+        reporter.ifPresent(optional -> optional.report(executionID, executionUnit, ExecuteProcessConstants.EXECUTE_STATUS_DONE));
     }
     
     /**
@@ -87,8 +81,9 @@ public final class ExecuteProcessEngine {
      * @param executionID execution ID
      */
     public static void finish(final String executionID) {
-        if (!HANDLERS.isEmpty() && ExecutorDataMap.getValue().containsKey(ExecuteProcessConstants.EXECUTE_ID.name())) {
-            HANDLERS.iterator().next().report(executionID, ExecuteProcessConstants.EXECUTE_STATUS_DONE);
+        Optional<ExecuteProcessReporter> reporter = ExecuteProcessReporterFactory.newInstance();
+        if (reporter.isPresent() && ExecutorDataMap.getValue().containsKey(ExecuteProcessConstants.EXECUTE_ID.name())) {
+            reporter.get().report(executionID, ExecuteProcessConstants.EXECUTE_STATUS_DONE);
         }
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/spi/ExecuteProcessReporter.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/spi/ExecuteProcessReporter.java
index 6e2f3076c78..2bdb1c5a6b5 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/spi/ExecuteProcessReporter.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/spi/ExecuteProcessReporter.java
@@ -21,11 +21,14 @@ import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroupContext;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.SQLExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.process.model.ExecuteProcessConstants;
+import org.apache.shardingsphere.spi.annotation.SingletonSPI;
+import org.apache.shardingsphere.spi.type.optional.OptionalSPI;
 
 /**
  * Execute process report.
  */
-public interface ExecuteProcessReporter {
+@SingletonSPI
+public interface ExecuteProcessReporter extends OptionalSPI {
     
     /**
      * Report the summary of this task.
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/spi/ExecuteProcessReporterFactory.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/spi/ExecuteProcessReporterFactory.java
new file mode 100644
index 00000000000..e3337061c45
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/spi/ExecuteProcessReporterFactory.java
@@ -0,0 +1,45 @@
+/*
+ * 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.shardingsphere.infra.executor.sql.process.spi;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.spi.type.optional.OptionalSPIRegistry;
+
+import java.util.Optional;
+
+/**
+ * Execute process report factory.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class ExecuteProcessReporterFactory {
+    
+    static {
+        ShardingSphereServiceLoader.register(ExecuteProcessReporter.class);
+    }
+    
+    /**
+     * Create new instance of execute process reporter.
+     * 
+     * @return new instance of execute process reporter
+     */
+    public static Optional<ExecuteProcessReporter> newInstance() {
+        return OptionalSPIRegistry.findRegisteredService(ExecuteProcessReporter.class);
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/process/fixture/ExecuteProcessReporterFixture.java b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/process/fixture/ExecuteProcessReporterFixture.java
index 1dba278a086..74fec2a7e62 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/process/fixture/ExecuteProcessReporterFixture.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/process/fixture/ExecuteProcessReporterFixture.java
@@ -25,20 +25,20 @@ import org.apache.shardingsphere.infra.executor.sql.process.spi.ExecuteProcessRe
 
 import java.util.LinkedList;
 
-public class ExecuteProcessReporterFixture implements ExecuteProcessReporter {
-
+public final class ExecuteProcessReporterFixture implements ExecuteProcessReporter {
+    
     public static final LinkedList<String> ACTIONS = new LinkedList<>();
-
+    
     @Override
     public void report(final LogicSQL logicSQL, final ExecutionGroupContext<? extends SQLExecutionUnit> executionGroupContext, final ExecuteProcessConstants constants) {
         ACTIONS.add("Report the summary of this task.");
     }
-
+    
     @Override
     public void report(final String executionID, final SQLExecutionUnit executionUnit, final ExecuteProcessConstants constants) {
         ACTIONS.add("Report a unit of this task.");
     }
-
+    
     @Override
     public void report(final String executionID, final ExecuteProcessConstants constants) {
         ACTIONS.add("Report this task on completion.");