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

[shardingsphere] branch master updated: Add KeyGenerateAlgorithmFactory (#16966)

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

jianglongtao 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 72b25d7728a Add KeyGenerateAlgorithmFactory (#16966)
72b25d7728a is described below

commit 72b25d7728ac99f5b7955cc110e5aa00d8c75d75
Author: Liang Zhang <zh...@apache.org>
AuthorDate: Thu Apr 21 00:29:12 2022 +0800

    Add KeyGenerateAlgorithmFactory (#16966)
    
    * Replace unnecessary Collectors.toCollection(LinkedList::new) to Collectors.toList()
    
    * Add KeyGenerateAlgorithmFactory
---
 .../determiner/HintShadowAlgorithmDeterminer.java  |  2 +-
 .../factory/KeyGenerateAlgorithmFactory.java       | 69 ++++++++++++++++++++++
 .../sharding/factory/ShardingAlgorithmFactory.java |  3 +
 .../shardingsphere/sharding/rule/ShardingRule.java | 18 ++----
 .../sharding/rule/TableRuleTest.java               | 16 +----
 .../checker/ShardingTableRuleStatementChecker.java |  8 +--
 .../AlterShardingKeyGeneratorStatementUpdater.java |  7 +--
 .../AlterShardingTableRuleStatementUpdater.java    |  7 ---
 ...CreateShardingKeyGeneratorStatementUpdater.java |  6 +-
 .../CreateShardingTableRuleStatementUpdater.java   |  7 ---
 .../checker/ShardingRuleStatementCheckerTest.java  |  3 -
 ...AlterShardingTableRuleStatementUpdaterTest.java |  3 -
 ...reateShardingTableRuleStatementUpdaterTest.java |  3 -
 .../ral/common/queryable/ShowInstanceHandler.java  |  3 +-
 14 files changed, 89 insertions(+), 66 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminer.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminer.java
index e742df61ee6..a4145a8c933 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminer.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminer.java
@@ -56,6 +56,6 @@ public final class HintShadowAlgorithmDeterminer {
         ShadowOperationType shadowOperationType = shadowDetermineCondition.getShadowOperationType();
         String tableName = shadowDetermineCondition.getTableName();
         return shadowDetermineCondition.getSqlComments().stream().<PreciseHintShadowValue<Comparable<?>>>map(
-                each -> new PreciseHintShadowValue<>(tableName, shadowOperationType, each)).collect(Collectors.toList());
+            each -> new PreciseHintShadowValue<>(tableName, shadowOperationType, each)).collect(Collectors.toList());
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/factory/KeyGenerateAlgorithmFactory.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/factory/KeyGenerateAlgorithmFactory.java
new file mode 100644
index 00000000000..f46080caf10
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/factory/KeyGenerateAlgorithmFactory.java
@@ -0,0 +1,69 @@
+/*
+ * 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.sharding.factory;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmFactory;
+import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
+import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.spi.type.required.RequiredSPIRegistry;
+import org.apache.shardingsphere.spi.type.typed.TypedSPIRegistry;
+
+import java.util.Properties;
+
+/**
+ * Key generate algorithm factory.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class KeyGenerateAlgorithmFactory {
+    
+    static {
+        ShardingSphereServiceLoader.register(KeyGenerateAlgorithm.class);
+    }
+    
+    /**
+     * Create new instance of key generate algorithm.
+     *
+     * @return new instance of key generate algorithm
+     */
+    public static KeyGenerateAlgorithm newInstance() {
+        return RequiredSPIRegistry.getRegisteredService(KeyGenerateAlgorithm.class);
+    }
+    
+    /**
+     * Create new instance of key generate algorithm.
+     *
+     * @param keyGenerateAlgorithmConfig key generate algorithm configuration
+     * @return new instance of key generate algorithm
+     */
+    public static KeyGenerateAlgorithm newInstance(final ShardingSphereAlgorithmConfiguration keyGenerateAlgorithmConfig) {
+        return ShardingSphereAlgorithmFactory.createAlgorithm(keyGenerateAlgorithmConfig, KeyGenerateAlgorithm.class);
+    }
+    
+    /**
+     * Judge whether contains key generate algorithm.
+     *
+     * @param keyGenerateAlgorithmType key generate algorithm type
+     * @return contains key generate algorithm or not
+     */
+    public static boolean contains(final String keyGenerateAlgorithmType) {
+        return TypedSPIRegistry.findRegisteredService(KeyGenerateAlgorithm.class, keyGenerateAlgorithmType, new Properties()).isPresent();
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/factory/ShardingAlgorithmFactory.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/factory/ShardingAlgorithmFactory.java
index a0577bf2ec1..6d8d876087b 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/factory/ShardingAlgorithmFactory.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-api/src/main/java/org/apache/shardingsphere/sharding/factory/ShardingAlgorithmFactory.java
@@ -17,6 +17,8 @@
 
 package org.apache.shardingsphere.sharding.factory;
 
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmFactory;
 import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
@@ -28,6 +30,7 @@ import java.util.Properties;
 /**
  * Sharding algorithm factory.
  */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
 public final class ShardingAlgorithmFactory {
     
     static {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
index e539022dd4e..ced39583a47 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
@@ -23,7 +23,6 @@ import lombok.Getter;
 import org.apache.commons.lang.StringUtils;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
-import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmFactory;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereInstanceRequiredAlgorithm;
 import org.apache.shardingsphere.infra.config.exception.ShardingSphereConfigurationException;
 import org.apache.shardingsphere.infra.datanode.DataNode;
@@ -44,11 +43,10 @@ import org.apache.shardingsphere.sharding.api.config.strategy.sharding.NoneShard
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.sharding.ShardingAutoTableAlgorithm;
+import org.apache.shardingsphere.sharding.factory.KeyGenerateAlgorithmFactory;
 import org.apache.shardingsphere.sharding.factory.ShardingAlgorithmFactory;
 import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
 import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.spi.type.required.RequiredSPIRegistry;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
@@ -82,10 +80,6 @@ public final class ShardingRule implements SchemaRule, DataNodeContainedRule, Ta
     
     private static final String EQUAL = "=";
     
-    static {
-        ShardingSphereServiceLoader.register(KeyGenerateAlgorithm.class);
-    }
-    
     private final Collection<String> dataSourceNames;
     
     private final Map<String, ShardingAlgorithm> shardingAlgorithms = new LinkedHashMap<>();
@@ -111,7 +105,7 @@ public final class ShardingRule implements SchemaRule, DataNodeContainedRule, Ta
     public ShardingRule(final ShardingRuleConfiguration config, final Collection<String> dataSourceNames) {
         this.dataSourceNames = getDataSourceNames(config.getTables(), config.getAutoTables(), dataSourceNames);
         config.getShardingAlgorithms().forEach((key, value) -> shardingAlgorithms.put(key, ShardingAlgorithmFactory.newInstance(value)));
-        config.getKeyGenerators().forEach((key, value) -> keyGenerators.put(key, ShardingSphereAlgorithmFactory.createAlgorithm(value, KeyGenerateAlgorithm.class)));
+        config.getKeyGenerators().forEach((key, value) -> keyGenerators.put(key, KeyGenerateAlgorithmFactory.newInstance(value)));
         tableRules.putAll(createTableRules(config.getTables(), config.getDefaultKeyGenerateStrategy()));
         tableRules.putAll(createAutoTableRules(config.getAutoTables(), config.getDefaultKeyGenerateStrategy()));
         bindingTableRules.putAll(createBindingTableRules(config.getBindingTableGroups()));
@@ -119,8 +113,7 @@ public final class ShardingRule implements SchemaRule, DataNodeContainedRule, Ta
         defaultDatabaseShardingStrategyConfig = null == config.getDefaultDatabaseShardingStrategy() ? new NoneShardingStrategyConfiguration() : config.getDefaultDatabaseShardingStrategy();
         defaultTableShardingStrategyConfig = null == config.getDefaultTableShardingStrategy() ? new NoneShardingStrategyConfiguration() : config.getDefaultTableShardingStrategy();
         defaultKeyGenerateAlgorithm = null == config.getDefaultKeyGenerateStrategy()
-                ? RequiredSPIRegistry.getRegisteredService(KeyGenerateAlgorithm.class)
-                : keyGenerators.get(config.getDefaultKeyGenerateStrategy().getKeyGeneratorName());
+                ? KeyGenerateAlgorithmFactory.newInstance() : keyGenerators.get(config.getDefaultKeyGenerateStrategy().getKeyGeneratorName());
         defaultShardingColumn = config.getDefaultShardingColumn();
         shardingTableDataNodes = createShardingTableDataNodes(tableRules);
         Preconditions.checkArgument(isValidBindingTableConfiguration(config.getBindingTableGroups()), "Invalid binding table configuration in ShardingRuleConfiguration.");
@@ -136,9 +129,8 @@ public final class ShardingRule implements SchemaRule, DataNodeContainedRule, Ta
         broadcastTables = createBroadcastTables(config.getBroadcastTables());
         defaultDatabaseShardingStrategyConfig = null == config.getDefaultDatabaseShardingStrategy() ? new NoneShardingStrategyConfiguration() : config.getDefaultDatabaseShardingStrategy();
         defaultTableShardingStrategyConfig = null == config.getDefaultTableShardingStrategy() ? new NoneShardingStrategyConfiguration() : config.getDefaultTableShardingStrategy();
-        defaultKeyGenerateAlgorithm = null == config.getDefaultKeyGenerateStrategy()
-                ? RequiredSPIRegistry.getRegisteredService(KeyGenerateAlgorithm.class)
-                : keyGenerators.get(config.getDefaultKeyGenerateStrategy().getKeyGeneratorName());
+        defaultKeyGenerateAlgorithm = null == config.getDefaultKeyGenerateStrategy() 
+                ? KeyGenerateAlgorithmFactory.newInstance() : keyGenerators.get(config.getDefaultKeyGenerateStrategy().getKeyGeneratorName());
         defaultShardingColumn = config.getDefaultShardingColumn();
         shardingTableDataNodes = createShardingTableDataNodes(tableRules);
         Preconditions.checkArgument(isValidBindingTableConfiguration(config.getBindingTableGroups()), "Invalid binding table configuration in ShardingRuleConfiguration.");
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/TableRuleTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/TableRuleTest.java
index 4c96b5ac724..bf84b336b30 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/TableRuleTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/TableRuleTest.java
@@ -26,17 +26,14 @@ import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfi
 import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.NoneShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
-import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
-import java.util.List;
 import java.util.LinkedList;
+import java.util.List;
 import java.util.Map;
-import java.util.Collection;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertFalse;
@@ -45,11 +42,6 @@ import static org.junit.Assert.assertTrue;
 
 public final class TableRuleTest {
     
-    @BeforeClass
-    public static void beforeClass() {
-        ShardingSphereServiceLoader.register(KeyGenerateAlgorithm.class);
-    }
-    
     @Test
     public void assertCreateMinTableRule() {
         ShardingTableRuleConfiguration tableRuleConfig = new ShardingTableRuleConfiguration("LOGIC_TABLE");
@@ -82,7 +74,6 @@ public final class TableRuleTest {
     
     @Test
     public void assertCreateAutoTableRuleWithModAlgorithm() {
-        ShardingSphereServiceLoader.register(KeyGenerateAlgorithm.class);
         ShardingAutoTableRuleConfiguration tableRuleConfig = new ShardingAutoTableRuleConfiguration("LOGIC_TABLE", "ds0,ds1");
         tableRuleConfig.setShardingStrategy(new StandardShardingStrategyConfiguration("col_1", "MOD"));
         ModShardingAlgorithm shardingAlgorithm = new ModShardingAlgorithm();
@@ -99,7 +90,6 @@ public final class TableRuleTest {
     
     @Test
     public void assertCreateAutoTableRuleWithModAlgorithmWithoutActualDataSources() {
-        ShardingSphereServiceLoader.register(KeyGenerateAlgorithm.class);
         ShardingAutoTableRuleConfiguration tableRuleConfig = new ShardingAutoTableRuleConfiguration("LOGIC_TABLE", null);
         tableRuleConfig.setShardingStrategy(new StandardShardingStrategyConfiguration("col_1", "MOD"));
         ModShardingAlgorithm shardingAlgorithm = new ModShardingAlgorithm();
@@ -173,7 +163,7 @@ public final class TableRuleTest {
     }
     
     @Test
-    public void assertCreateTableRuleWithdataSourceNames() {
+    public void assertCreateTableRuleWithDataSourceNames() {
         Collection<String> dataSourceNames = new LinkedList<>();
         String logicTableName = "table_0";
         dataSourceNames.add("ds0");
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/checker/ShardingTableRuleStatementChecker.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/checker/ShardingTableRuleStatementChecker.java
index b8fbb096a2d..1266bb9e7d3 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/checker/ShardingTableRuleStatementChecker.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/checker/ShardingTableRuleStatementChecker.java
@@ -37,9 +37,8 @@ import org.apache.shardingsphere.sharding.distsql.parser.segment.AbstractTableRu
 import org.apache.shardingsphere.sharding.distsql.parser.segment.AutoTableRuleSegment;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingStrategySegment;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.TableRuleSegment;
+import org.apache.shardingsphere.sharding.factory.KeyGenerateAlgorithmFactory;
 import org.apache.shardingsphere.sharding.factory.ShardingAlgorithmFactory;
-import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
-import org.apache.shardingsphere.spi.type.typed.TypedSPIRegistry;
 
 import java.util.Arrays;
 import java.util.Collection;
@@ -50,7 +49,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
-import java.util.Properties;
 import java.util.Set;
 import java.util.stream.Collectors;
 
@@ -175,9 +173,7 @@ public final class ShardingTableRuleStatementChecker {
                 .peek(each -> each.getKeyGenerateAlgorithmName().filter(op -> null == currentRuleConfig || !currentRuleConfig.getKeyGenerators().containsKey(op)).ifPresent(notExistKeyGenerator::add))
                 .filter(each -> !each.getKeyGenerateAlgorithmName().isPresent()).forEach(each -> requiredKeyGenerators.add(each.getKeyGenerateAlgorithmSegment().getName()));
         DistSQLException.predictionThrow(notExistKeyGenerator.isEmpty(), () -> new RequiredAlgorithmMissedException("key generator", notExistKeyGenerator));
-        Collection<String> invalidKeyGenerators = requiredKeyGenerators.stream().distinct()
-                .filter(each -> !TypedSPIRegistry.findRegisteredService(KeyGenerateAlgorithm.class, each, new Properties()).isPresent())
-                .collect(Collectors.toList());
+        Collection<String> invalidKeyGenerators = requiredKeyGenerators.stream().distinct().filter(each -> !KeyGenerateAlgorithmFactory.contains(each)).collect(Collectors.toList());
         DistSQLException.predictionThrow(invalidKeyGenerators.isEmpty(), () -> new InvalidAlgorithmConfigurationException("key generator", invalidKeyGenerators));
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/AlterShardingKeyGeneratorStatementUpdater.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/AlterShardingKeyGeneratorStatementUpdater.java
index 15435d3ffb5..63b752ad0f4 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/AlterShardingKeyGeneratorStatementUpdater.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/AlterShardingKeyGeneratorStatementUpdater.java
@@ -29,12 +29,10 @@ import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.distsql.handler.converter.ShardingTableRuleStatementConverter;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingKeyGeneratorSegment;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.AlterShardingKeyGeneratorStatement;
-import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
-import org.apache.shardingsphere.spi.type.typed.TypedSPIRegistry;
+import org.apache.shardingsphere.sharding.factory.KeyGenerateAlgorithmFactory;
 
 import java.util.Collection;
 import java.util.Map;
-import java.util.Properties;
 import java.util.stream.Collectors;
 
 /**
@@ -66,8 +64,7 @@ public final class AlterShardingKeyGeneratorStatementUpdater implements RuleDefi
     private void checkAlgorithmType(final AlterShardingKeyGeneratorStatement sqlStatement) throws DistSQLException {
         Collection<String> requireNames = sqlStatement.getKeyGeneratorSegments().stream()
                 .map(ShardingKeyGeneratorSegment::getAlgorithmSegment).map(AlgorithmSegment::getName).collect(Collectors.toList());
-        Collection<String> invalidAlgorithmNames = requireNames.stream()
-                .filter(each -> !TypedSPIRegistry.findRegisteredService(KeyGenerateAlgorithm.class, each, new Properties()).isPresent()).collect(Collectors.toList());
+        Collection<String> invalidAlgorithmNames = requireNames.stream().filter(each -> !KeyGenerateAlgorithmFactory.contains(each)).collect(Collectors.toList());
         DistSQLException.predictionThrow(invalidAlgorithmNames.isEmpty(), () -> new InvalidAlgorithmConfigurationException("sharding", invalidAlgorithmNames));
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/AlterShardingTableRuleStatementUpdater.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/AlterShardingTableRuleStatementUpdater.java
index fd5065c7e6d..cbed93a377f 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/AlterShardingTableRuleStatementUpdater.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/AlterShardingTableRuleStatementUpdater.java
@@ -27,8 +27,6 @@ import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfi
 import org.apache.shardingsphere.sharding.distsql.handler.checker.ShardingTableRuleStatementChecker;
 import org.apache.shardingsphere.sharding.distsql.handler.converter.ShardingTableRuleStatementConverter;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.AlterShardingTableRuleStatement;
-import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
 
 import java.util.Collection;
 import java.util.stream.Collectors;
@@ -38,11 +36,6 @@ import java.util.stream.Collectors;
  */
 public final class AlterShardingTableRuleStatementUpdater implements RuleDefinitionAlterUpdater<AlterShardingTableRuleStatement, ShardingRuleConfiguration> {
     
-    static {
-        // TODO consider about register once only
-        ShardingSphereServiceLoader.register(KeyGenerateAlgorithm.class);
-    }
-    
     @Override
     public void checkSQLStatement(final ShardingSphereMetaData shardingSphereMetaData, final AlterShardingTableRuleStatement sqlStatement,
                                   final ShardingRuleConfiguration currentRuleConfig) throws DistSQLException {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingKeyGeneratorStatementUpdater.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingKeyGeneratorStatementUpdater.java
index fc8b33c349d..e5fe71ff371 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingKeyGeneratorStatementUpdater.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingKeyGeneratorStatementUpdater.java
@@ -28,12 +28,10 @@ import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.distsql.handler.converter.ShardingTableRuleStatementConverter;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingKeyGeneratorSegment;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingKeyGeneratorStatement;
-import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
-import org.apache.shardingsphere.spi.type.typed.TypedSPIRegistry;
+import org.apache.shardingsphere.sharding.factory.KeyGenerateAlgorithmFactory;
 
 import java.util.Collection;
 import java.util.Map;
-import java.util.Properties;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
@@ -70,7 +68,7 @@ public final class CreateShardingKeyGeneratorStatementUpdater implements RuleDef
     
     private void checkKeyGeneratorAlgorithm(final CreateShardingKeyGeneratorStatement sqlStatement) throws DistSQLException {
         Collection<String> notExistedKeyGeneratorAlgorithms = sqlStatement.getKeyGeneratorSegments().stream().map(ShardingKeyGeneratorSegment::getAlgorithmSegment).map(AlgorithmSegment::getName)
-                .filter(each -> !TypedSPIRegistry.findRegisteredService(KeyGenerateAlgorithm.class, each, new Properties()).isPresent()).collect(Collectors.toList());
+                .filter(each -> !KeyGenerateAlgorithmFactory.contains(each)).collect(Collectors.toList());
         DistSQLException.predictionThrow(notExistedKeyGeneratorAlgorithms.isEmpty(), () -> new InvalidAlgorithmConfigurationException("sharding", notExistedKeyGeneratorAlgorithms));
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingTableRuleStatementUpdater.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingTableRuleStatementUpdater.java
index b4da66bee20..d6fb26797fb 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingTableRuleStatementUpdater.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateShardingTableRuleStatementUpdater.java
@@ -24,19 +24,12 @@ import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.distsql.handler.checker.ShardingTableRuleStatementChecker;
 import org.apache.shardingsphere.sharding.distsql.handler.converter.ShardingTableRuleStatementConverter;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingTableRuleStatement;
-import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
 
 /**
  * Create sharding table rule statement updater.
  */
 public final class CreateShardingTableRuleStatementUpdater implements RuleDefinitionCreateUpdater<CreateShardingTableRuleStatement, ShardingRuleConfiguration> {
     
-    static {
-        // TODO consider about register once only
-        ShardingSphereServiceLoader.register(KeyGenerateAlgorithm.class);
-    }
-    
     @Override
     public void checkSQLStatement(final ShardingSphereMetaData shardingSphereMetaData, final CreateShardingTableRuleStatement sqlStatement,
                                   final ShardingRuleConfiguration currentRuleConfig) throws DistSQLException {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/checker/ShardingRuleStatementCheckerTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/checker/ShardingRuleStatementCheckerTest.java
index e9178206bac..e336170bb45 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/checker/ShardingRuleStatementCheckerTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/checker/ShardingRuleStatementCheckerTest.java
@@ -38,8 +38,6 @@ import org.apache.shardingsphere.sharding.distsql.parser.segment.AutoTableRuleSe
 import org.apache.shardingsphere.sharding.distsql.parser.segment.KeyGenerateStrategySegment;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingStrategySegment;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.TableRuleSegment;
-import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -74,7 +72,6 @@ public final class ShardingRuleStatementCheckerTest {
     
     @Before
     public void before() {
-        ShardingSphereServiceLoader.register(KeyGenerateAlgorithm.class);
         when(shardingSphereMetaData.getName()).thenReturn("schema");
         when(shardingSphereMetaData.getResource()).thenReturn(shardingSphereResource);
         when(shardingSphereMetaData.getRuleMetaData()).thenReturn(shardingSphereRuleMetaData);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/AlterShardingTableRuleStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/AlterShardingTableRuleStatementUpdaterTest.java
index 5c95d23cb26..ca7e2b542ee 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/AlterShardingTableRuleStatementUpdaterTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/AlterShardingTableRuleStatementUpdaterTest.java
@@ -36,8 +36,6 @@ import org.apache.shardingsphere.sharding.distsql.parser.segment.KeyGenerateStra
 import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingStrategySegment;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.TableRuleSegment;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.AlterShardingTableRuleStatement;
-import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -76,7 +74,6 @@ public final class AlterShardingTableRuleStatementUpdaterTest {
     
     @Before
     public void before() {
-        ShardingSphereServiceLoader.register(KeyGenerateAlgorithm.class);
         when(shardingSphereMetaData.getName()).thenReturn("schema");
         when(shardingSphereMetaData.getResource()).thenReturn(shardingSphereResource);
         when(shardingSphereMetaData.getRuleMetaData()).thenReturn(shardingSphereRuleMetaData);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java
index a4c60e5cbf0..378894f6af1 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-distsql/shardingsphere-sharding-distsql-handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateShardingTableRuleStatementUpdaterTest.java
@@ -37,8 +37,6 @@ import org.apache.shardingsphere.sharding.distsql.parser.segment.KeyGenerateStra
 import org.apache.shardingsphere.sharding.distsql.parser.segment.ShardingStrategySegment;
 import org.apache.shardingsphere.sharding.distsql.parser.segment.TableRuleSegment;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateShardingTableRuleStatement;
-import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -77,7 +75,6 @@ public final class CreateShardingTableRuleStatementUpdaterTest {
     
     @Before
     public void before() {
-        ShardingSphereServiceLoader.register(KeyGenerateAlgorithm.class);
         when(shardingSphereMetaData.getName()).thenReturn("schema");
         when(shardingSphereMetaData.getResource()).thenReturn(shardingSphereResource);
         when(shardingSphereMetaData.getRuleMetaData()).thenReturn(shardingSphereRuleMetaData);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowInstanceHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowInstanceHandler.java
index f72fb9c466e..bb9645df34c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowInstanceHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowInstanceHandler.java
@@ -81,7 +81,8 @@ public final class ShowInstanceHandler extends QueryableRALBackendHandler<ShowIn
     private Collection<List<Object>> buildInstanceRows(final InstanceContext instanceContext, final MetaDataPersistService persistService) {
         Collection<ComputeNodeInstance> instances = persistService.getComputeNodePersistService().loadAllComputeNodeInstances();
         return instances.isEmpty()
-                ? Collections.emptyList() : instances.stream().filter(Objects::nonNull).map(each -> buildRow(each, instanceContext.getModeConfiguration().getType())).collect(Collectors.toList());
+                ? Collections.emptyList()
+                : instances.stream().filter(Objects::nonNull).map(each -> buildRow(each, instanceContext.getModeConfiguration().getType())).collect(Collectors.toList());
     }
     
     private List<Object> buildRow(final ComputeNodeInstance instance, final String modeType) {