You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2022/04/26 10:43:04 UTC

[shardingsphere] branch master updated: Add SQLRouterFactory (#17121)

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

panjuan 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 e2bc11cab42 Add SQLRouterFactory (#17121)
e2bc11cab42 is described below

commit e2bc11cab425d65e62cc2a1cac3faace53259ef4
Author: Liang Zhang <zh...@apache.org>
AuthorDate: Tue Apr 26 18:42:55 2022 +0800

    Add SQLRouterFactory (#17121)
---
 .../route/DatabaseDiscoverySQLRouterTest.java      | 10 +----
 .../route/ReadwriteSplittingSQLRouterTest.java     | 10 +----
 .../infra/route/SQLRouterFactory.java              | 49 ++++++++++++++++++++++
 .../route/engine/impl/PartialSQLRouteExecutor.java |  9 +---
 4 files changed, 55 insertions(+), 23 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/test/java/org/apache/shardingsphere/dbdiscovery/route/DatabaseDiscoverySQLRouterTest.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/test/java/org/apache/shardingsphere/dbdiscovery/route/DatabaseDiscoverySQLRouterTest.java
index 1fec8265149..feed3cd30c1 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/test/java/org/apache/shardingsphere/dbdiscovery/route/DatabaseDiscoverySQLRouterTest.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/test/java/org/apache/shardingsphere/dbdiscovery/route/DatabaseDiscoverySQLRouterTest.java
@@ -28,12 +28,10 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
-import org.apache.shardingsphere.infra.route.SQLRouter;
+import org.apache.shardingsphere.infra.route.SQLRouterFactory;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.spi.type.ordered.OrderedSPIRegistry;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.junit.Before;
 import org.junit.Test;
@@ -67,10 +65,6 @@ public final class DatabaseDiscoverySQLRouterTest {
     
     private DatabaseDiscoverySQLRouter sqlRouter;
     
-    static {
-        ShardingSphereServiceLoader.register(SQLRouter.class);
-    }
-    
     @Before
     public void setUp() {
         DatabaseDiscoveryDataSourceRuleConfiguration dataSourceConfig = new DatabaseDiscoveryDataSourceRuleConfiguration(
@@ -80,7 +74,7 @@ public final class DatabaseDiscoverySQLRouterTest {
                 Collections.singletonMap("ha_heartbeat", new DatabaseDiscoveryHeartBeatConfiguration(new Properties())),
                 Collections.singletonMap("CORE.FIXTURE", algorithmConfig));
         rule = new DatabaseDiscoveryRule(DATA_SOURCE_NAME, Collections.singletonMap(PRIMARY_DATA_SOURCE, mock(DataSource.class)), config);
-        sqlRouter = (DatabaseDiscoverySQLRouter) OrderedSPIRegistry.getRegisteredServices(SQLRouter.class, Collections.singleton(rule)).get(rule);
+        sqlRouter = (DatabaseDiscoverySQLRouter) SQLRouterFactory.newInstance(Collections.singleton(rule)).get(rule);
     }
     
     @Test
diff --git a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/test/java/org/apache/shardingsphere/readwritesplitting/route/ReadwriteSplittingSQLRouterTest.java b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/test/java/org/apache/shardingsphere/readwritesplitting/route/ReadwriteSplittingSQLRouterTest.java
index 3ea5738209b..31ba0ae3e53 100644
--- a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/test/java/org/apache/shardingsphere/readwritesplitting/route/ReadwriteSplittingSQLRouterTest.java
+++ b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/test/java/org/apache/shardingsphere/readwritesplitting/route/ReadwriteSplittingSQLRouterTest.java
@@ -25,15 +25,13 @@ import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
-import org.apache.shardingsphere.infra.route.SQLRouter;
+import org.apache.shardingsphere.infra.route.SQLRouterFactory;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
 import org.apache.shardingsphere.readwritesplitting.api.ReadwriteSplittingRuleConfiguration;
 import org.apache.shardingsphere.readwritesplitting.api.rule.ReadwriteSplittingDataSourceRuleConfiguration;
 import org.apache.shardingsphere.readwritesplitting.rule.ReadwriteSplittingRule;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.spi.type.ordered.OrderedSPIRegistry;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.LockSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
@@ -75,10 +73,6 @@ public final class ReadwriteSplittingSQLRouterTest {
     
     private ReadwriteSplittingSQLRouter sqlRouter;
     
-    static {
-        ShardingSphereServiceLoader.register(SQLRouter.class);
-    }
-    
     @Before
     public void setUp() {
         Properties props = new Properties();
@@ -86,7 +80,7 @@ public final class ReadwriteSplittingSQLRouterTest {
         props.setProperty("read-data-source-names", READ_DATASOURCE);
         rule = new ReadwriteSplittingRule(new ReadwriteSplittingRuleConfiguration(Collections.singleton(
                 new ReadwriteSplittingDataSourceRuleConfiguration(DATASOURCE_NAME, "Static", props, "")), Collections.emptyMap()));
-        sqlRouter = (ReadwriteSplittingSQLRouter) OrderedSPIRegistry.getRegisteredServices(SQLRouter.class, Collections.singleton(rule)).get(rule);
+        sqlRouter = (ReadwriteSplittingSQLRouter) SQLRouterFactory.newInstance(Collections.singleton(rule)).get(rule);
     }
     
     @Test
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/SQLRouterFactory.java b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/SQLRouterFactory.java
new file mode 100644
index 00000000000..78e874fb641
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/SQLRouterFactory.java
@@ -0,0 +1,49 @@
+/*
+ * 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.route;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.spi.type.ordered.OrderedSPIRegistry;
+
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ * SQL Router factory.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class SQLRouterFactory {
+    
+    static {
+        ShardingSphereServiceLoader.register(SQLRouter.class);
+    }
+    
+    /**
+     * Create new instance of SQL router.
+     * 
+     * @param rules rules
+     * @return new instance of SQL router
+     */
+    @SuppressWarnings("rawtypes")
+    public static Map<ShardingSphereRule, SQLRouter> newInstance(final Collection<ShardingSphereRule> rules) {
+        return OrderedSPIRegistry.getRegisteredServices(SQLRouter.class, rules);
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/engine/impl/PartialSQLRouteExecutor.java b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/engine/impl/PartialSQLRouteExecutor.java
index 4a625fc800e..a1bc8224a35 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/engine/impl/PartialSQLRouteExecutor.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/engine/impl/PartialSQLRouteExecutor.java
@@ -25,13 +25,12 @@ import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.hint.HintManager;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.SQLRouter;
+import org.apache.shardingsphere.infra.route.SQLRouterFactory;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
 import org.apache.shardingsphere.infra.route.engine.SQLRouteExecutor;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.spi.type.ordered.OrderedSPIRegistry;
 
 import javax.sql.DataSource;
 import java.util.Collection;
@@ -45,10 +44,6 @@ import java.util.Optional;
  */
 public final class PartialSQLRouteExecutor implements SQLRouteExecutor {
     
-    static {
-        ShardingSphereServiceLoader.register(SQLRouter.class);
-    }
-    
     private final ConfigurationProperties props;
     
     @SuppressWarnings("rawtypes")
@@ -56,7 +51,7 @@ public final class PartialSQLRouteExecutor implements SQLRouteExecutor {
     
     public PartialSQLRouteExecutor(final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) {
         this.props = props;
-        routers = OrderedSPIRegistry.getRegisteredServices(SQLRouter.class, rules);
+        routers = SQLRouterFactory.newInstance(rules);
     }
     
     @Override