You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2022/07/22 06:20:39 UTC

[shardingsphere] branch master updated: Add rule configuration check logic for Proxy and move sharding rule check logic (#19462)

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

zhaojinchao 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 f1182e80d1c Add rule configuration check logic for Proxy and move sharding rule check logic (#19462)
f1182e80d1c is described below

commit f1182e80d1c34a6d885cfca7899fc3228bec64b1
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Fri Jul 22 14:20:31 2022 +0800

    Add rule configuration check logic for Proxy and move sharding rule check logic (#19462)
    
    * Add rule configuration check logic for Proxy and move sharding rule check logic
    
    * Add java doc for protected method
    
    * Update exception message
    
    * Optimize exception message
    
    * update unit test
---
 .../AbstractShardingRuleConfigurationChecker.java  | 90 +++++++++++++++++++++-
 ...hmProvidedShardingRuleConfigurationChecker.java | 13 +++-
 .../checker/ShardingRuleConfigurationChecker.java  | 13 +++-
 .../AlgorithmProvidedShardingRuleBuilder.java      | 44 -----------
 .../sharding/rule/builder/ShardingRuleBuilder.java | 54 -------------
 ...ovidedShardingRuleConfigurationCheckerTest.java |  6 +-
 .../ShardingRuleConfigurationCheckerTest.java      |  6 +-
 .../AlgorithmProvidedShardingRuleBuilderTest.java  | 31 --------
 .../rule/builder/ShardingRuleBuilderTest.java      | 31 --------
 .../cluster/ClusterContextManagerBuilder.java      |  4 +-
 .../proxy/initializer/BootstrapInitializer.java    | 17 +++-
 11 files changed, 136 insertions(+), 173 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/checker/AbstractShardingRuleConfigurationChecker.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/checker/AbstractShardingRuleConfigurationChecker.java
index a165cb6fe42..4d0af78a54e 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/checker/AbstractShardingRuleConfigurationChecker.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/checker/AbstractShardingRuleConfigurationChecker.java
@@ -20,6 +20,14 @@ package org.apache.shardingsphere.sharding.checker;
 import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.checker.RuleConfigurationChecker;
+import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.audit.ShardingAuditStrategyConfiguration;
+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.ShardingStrategyConfiguration;
+
+import java.util.Collection;
 
 /**
  * Abstract sharding rule configuration checker.
@@ -30,8 +38,86 @@ public abstract class AbstractShardingRuleConfigurationChecker<T extends RuleCon
     
     @Override
     public final void check(final String databaseName, final T config) {
-        Preconditions.checkState(hasAvailableTableConfigurations(config), "No available sharding rule configurations in database `%s`.", databaseName);
+        checkShardingRuleConfiguration(databaseName, config);
     }
     
-    protected abstract boolean hasAvailableTableConfigurations(T config);
+    /**
+     * Check sharding rule configuration.
+     * 
+     * @param databaseName database name
+     * @param config config
+     */
+    protected abstract void checkShardingRuleConfiguration(String databaseName, T config);
+    
+    /**
+     * Check table configuration.
+     * 
+     * @param tables table configurations
+     * @param autoTables autoTable configurations
+     * @param keyGenerators keyGenerators
+     * @param auditors auditors
+     * @param shardingAlgorithms shardingAlgorithms
+     * @param databaseName database name
+     */
+    protected void checkTableConfiguration(final Collection<ShardingTableRuleConfiguration> tables, final Collection<ShardingAutoTableRuleConfiguration> autoTables,
+                                           final Collection<String> keyGenerators, final Collection<String> auditors, final Collection<String> shardingAlgorithms, final String databaseName) {
+        Preconditions.checkState(!tables.isEmpty() || !autoTables.isEmpty(),
+                "No available sharding table or autoTable configurations in database `%s`.", databaseName);
+        for (ShardingTableRuleConfiguration each : tables) {
+            checkKeyGenerateStrategy(each.getKeyGenerateStrategy(), keyGenerators, databaseName);
+            checkAuditStrategy(each.getAuditStrategy(), auditors, databaseName);
+            checkShardingStrategy(each.getDatabaseShardingStrategy(), shardingAlgorithms, databaseName);
+            checkShardingStrategy(each.getTableShardingStrategy(), shardingAlgorithms, databaseName);
+        }
+        for (ShardingAutoTableRuleConfiguration each : autoTables) {
+            checkKeyGenerateStrategy(each.getKeyGenerateStrategy(), keyGenerators, databaseName);
+            checkAuditStrategy(each.getAuditStrategy(), auditors, databaseName);
+            checkShardingStrategy(each.getShardingStrategy(), shardingAlgorithms, databaseName);
+        }
+    }
+    
+    /**
+     * Check key generate strategy.
+     * 
+     * @param keyGenerateStrategy key generate strategy
+     * @param keyGenerators keyGenerators
+     * @param databaseName database name
+     */
+    protected void checkKeyGenerateStrategy(final KeyGenerateStrategyConfiguration keyGenerateStrategy, final Collection<String> keyGenerators, final String databaseName) {
+        if (null == keyGenerateStrategy) {
+            return;
+        }
+        Preconditions.checkState(keyGenerators.contains(keyGenerateStrategy.getKeyGeneratorName()),
+                "Can not find keyGenerator `%s` in database `%s`.", keyGenerateStrategy.getKeyGeneratorName(), databaseName);
+    }
+    
+    /**
+     * Check audit strategy.
+     * 
+     * @param auditStrategy audit strategy
+     * @param auditors auditors
+     * @param databaseName database name
+     */
+    protected void checkAuditStrategy(final ShardingAuditStrategyConfiguration auditStrategy, final Collection<String> auditors, final String databaseName) {
+        if (null == auditStrategy) {
+            return;
+        }
+        Preconditions.checkState(auditors.containsAll(auditStrategy.getAuditorNames()),
+                "Can not find all auditors `%s` in database `%s`.", auditStrategy.getAuditorNames(), databaseName);
+    }
+    
+    /**
+     * Check sharding strategy.
+     * 
+     * @param shardingStrategy sharding strategy
+     * @param shardingAlgorithms shardingAlgorithms
+     * @param databaseName database name
+     */
+    protected void checkShardingStrategy(final ShardingStrategyConfiguration shardingStrategy, final Collection<String> shardingAlgorithms, final String databaseName) {
+        if (null == shardingStrategy || shardingStrategy instanceof NoneShardingStrategyConfiguration) {
+            return;
+        }
+        Preconditions.checkState(shardingAlgorithms.contains(shardingStrategy.getShardingAlgorithmName()),
+                "Can not find shardingAlgorithm `%s` in database `%s`.", shardingStrategy.getShardingAlgorithmName(), databaseName);
+    }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/checker/AlgorithmProvidedShardingRuleConfigurationChecker.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/checker/AlgorithmProvidedShardingRuleConfigurationChecker.java
index 35be860711a..e04efd19a74 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/checker/AlgorithmProvidedShardingRuleConfigurationChecker.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/checker/AlgorithmProvidedShardingRuleConfigurationChecker.java
@@ -20,14 +20,23 @@ package org.apache.shardingsphere.sharding.checker;
 import org.apache.shardingsphere.sharding.algorithm.config.AlgorithmProvidedShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
 
+import java.util.Collection;
+
 /**
  * Algorithm provided sharding rule configuration checker.
  */
 public final class AlgorithmProvidedShardingRuleConfigurationChecker extends AbstractShardingRuleConfigurationChecker<AlgorithmProvidedShardingRuleConfiguration> {
     
     @Override
-    protected boolean hasAvailableTableConfigurations(final AlgorithmProvidedShardingRuleConfiguration config) {
-        return !config.getTables().isEmpty() || null != config.getDefaultTableShardingStrategy() || !config.getAutoTables().isEmpty();
+    protected void checkShardingRuleConfiguration(final String databaseName, final AlgorithmProvidedShardingRuleConfiguration config) {
+        Collection<String> keyGenerators = config.getKeyGenerators().keySet();
+        Collection<String> auditors = config.getAuditors().keySet();
+        Collection<String> shardingAlgorithms = config.getShardingAlgorithms().keySet();
+        checkTableConfiguration(config.getTables(), config.getAutoTables(), keyGenerators, auditors, shardingAlgorithms, databaseName);
+        checkKeyGenerateStrategy(config.getDefaultKeyGenerateStrategy(), keyGenerators, databaseName);
+        checkAuditStrategy(config.getDefaultAuditStrategy(), auditors, databaseName);
+        checkShardingStrategy(config.getDefaultDatabaseShardingStrategy(), shardingAlgorithms, databaseName);
+        checkShardingStrategy(config.getDefaultTableShardingStrategy(), shardingAlgorithms, databaseName);
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/checker/ShardingRuleConfigurationChecker.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/checker/ShardingRuleConfigurationChecker.java
index 3a80b7eb501..8e9c5eb550a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/checker/ShardingRuleConfigurationChecker.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/checker/ShardingRuleConfigurationChecker.java
@@ -20,14 +20,23 @@ package org.apache.shardingsphere.sharding.checker;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
 
+import java.util.Collection;
+
 /**
  * Sharding rule configuration checker.
  */
 public final class ShardingRuleConfigurationChecker extends AbstractShardingRuleConfigurationChecker<ShardingRuleConfiguration> {
     
     @Override
-    protected boolean hasAvailableTableConfigurations(final ShardingRuleConfiguration config) {
-        return !config.getTables().isEmpty() || null != config.getDefaultTableShardingStrategy() || !config.getAutoTables().isEmpty();
+    protected void checkShardingRuleConfiguration(final String databaseName, final ShardingRuleConfiguration config) {
+        Collection<String> keyGenerators = config.getKeyGenerators().keySet();
+        Collection<String> auditors = config.getAuditors().keySet();
+        Collection<String> shardingAlgorithms = config.getShardingAlgorithms().keySet();
+        checkTableConfiguration(config.getTables(), config.getAutoTables(), keyGenerators, auditors, shardingAlgorithms, databaseName);
+        checkKeyGenerateStrategy(config.getDefaultKeyGenerateStrategy(), keyGenerators, databaseName);
+        checkAuditStrategy(config.getDefaultAuditStrategy(), auditors, databaseName);
+        checkShardingStrategy(config.getDefaultDatabaseShardingStrategy(), shardingAlgorithms, databaseName);
+        checkShardingStrategy(config.getDefaultTableShardingStrategy(), shardingAlgorithms, databaseName);
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilder.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilder.java
index ff0ee13aa87..fdfc3b384bb 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilder.java
@@ -22,15 +22,8 @@ import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.builder.database.DatabaseRuleBuilder;
 import org.apache.shardingsphere.sharding.algorithm.config.AlgorithmProvidedShardingRuleConfiguration;
-import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
-import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
-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.ShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
-import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
 
 import javax.sql.DataSource;
 import java.util.Collection;
@@ -45,46 +38,9 @@ public final class AlgorithmProvidedShardingRuleBuilder implements DatabaseRuleB
     public ShardingRule build(final AlgorithmProvidedShardingRuleConfiguration config, final String databaseName,
                               final Map<String, DataSource> dataSources, final Collection<ShardingSphereRule> builtRules, final InstanceContext instanceContext) {
         Preconditions.checkArgument(null != dataSources && !dataSources.isEmpty(), "Data sources cannot be empty.");
-        Preconditions.checkArgument(isValidRuleConfiguration(config), "Invalid sharding configuration in AlgorithmProvidedShardingRuleConfiguration.");
         return new ShardingRule(config, dataSources.keySet(), instanceContext);
     }
     
-    private boolean isValidRuleConfiguration(final AlgorithmProvidedShardingRuleConfiguration config) {
-        Map<String, KeyGenerateAlgorithm> keyGenerators = config.getKeyGenerators();
-        Map<String, ShardingAlgorithm> shardingAlgorithms = config.getShardingAlgorithms();
-        if (isInvalidKeyGenerateStrategy(config.getDefaultKeyGenerateStrategy(), keyGenerators)
-                || isInvalidShardingStrategy(config.getDefaultDatabaseShardingStrategy(), shardingAlgorithms)
-                || isInvalidShardingStrategy(config.getDefaultTableShardingStrategy(), shardingAlgorithms)) {
-            return false;
-        }
-        for (ShardingTableRuleConfiguration each : config.getTables()) {
-            if (isInvalidKeyGenerateStrategy(each.getKeyGenerateStrategy(), keyGenerators) || isInvalidShardingStrategy(each.getDatabaseShardingStrategy(), shardingAlgorithms)
-                    || isInvalidShardingStrategy(each.getTableShardingStrategy(), shardingAlgorithms)) {
-                return false;
-            }
-        }
-        for (ShardingAutoTableRuleConfiguration each : config.getAutoTables()) {
-            if (isInvalidKeyGenerateStrategy(each.getKeyGenerateStrategy(), keyGenerators) || isInvalidShardingStrategy(each.getShardingStrategy(), shardingAlgorithms)) {
-                return false;
-            }
-        }
-        return true;
-    }
-    
-    private boolean isInvalidKeyGenerateStrategy(final KeyGenerateStrategyConfiguration keyGenerateStrategy, final Map<String, KeyGenerateAlgorithm> keyGenerators) {
-        if (null == keyGenerateStrategy) {
-            return false;
-        }
-        return !keyGenerators.containsKey(keyGenerateStrategy.getKeyGeneratorName());
-    }
-    
-    private boolean isInvalidShardingStrategy(final ShardingStrategyConfiguration shardingStrategy, final Map<String, ShardingAlgorithm> shardingAlgorithms) {
-        if (null == shardingStrategy || shardingStrategy instanceof NoneShardingStrategyConfiguration) {
-            return false;
-        }
-        return !shardingAlgorithms.containsKey(shardingStrategy.getShardingAlgorithmName());
-    }
-    
     @Override
     public int getOrder() {
         return ShardingOrder.ALGORITHM_PROVIDER_ORDER;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilder.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilder.java
index 484a5075d97..6588bbf57ae 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilder.java
@@ -18,17 +18,10 @@
 package org.apache.shardingsphere.sharding.rule.builder;
 
 import com.google.common.base.Preconditions;
-import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.builder.database.DatabaseRuleBuilder;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
-import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
-import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
-import org.apache.shardingsphere.sharding.api.config.strategy.audit.ShardingAuditStrategyConfiguration;
-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.ShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 
@@ -45,56 +38,9 @@ public final class ShardingRuleBuilder implements DatabaseRuleBuilder<ShardingRu
     public ShardingRule build(final ShardingRuleConfiguration config, final String databaseName,
                               final Map<String, DataSource> dataSources, final Collection<ShardingSphereRule> builtRules, final InstanceContext instanceContext) {
         Preconditions.checkArgument(null != dataSources && !dataSources.isEmpty(), "Data source names cannot be empty.");
-        Preconditions.checkArgument(isValidRuleConfiguration(config), "Invalid sharding configuration in ShardingRuleConfiguration.");
         return new ShardingRule(config, dataSources.keySet(), instanceContext);
     }
     
-    private boolean isValidRuleConfiguration(final ShardingRuleConfiguration config) {
-        Map<String, ShardingSphereAlgorithmConfiguration> keyGenerators = config.getKeyGenerators();
-        Map<String, ShardingSphereAlgorithmConfiguration> auditors = config.getAuditors();
-        Map<String, ShardingSphereAlgorithmConfiguration> shardingAlgorithms = config.getShardingAlgorithms();
-        if (isInvalidKeyGenerateStrategy(config.getDefaultKeyGenerateStrategy(), keyGenerators)
-                || isInvalidAuditStrategy(config.getDefaultAuditStrategy(), auditors)
-                || isInvalidShardingStrategy(config.getDefaultDatabaseShardingStrategy(), shardingAlgorithms)
-                || isInvalidShardingStrategy(config.getDefaultTableShardingStrategy(), shardingAlgorithms)) {
-            return false;
-        }
-        for (ShardingTableRuleConfiguration each : config.getTables()) {
-            if (isInvalidKeyGenerateStrategy(each.getKeyGenerateStrategy(), keyGenerators) || isInvalidAuditStrategy(each.getAuditStrategy(), auditors)
-                    || isInvalidShardingStrategy(each.getDatabaseShardingStrategy(), shardingAlgorithms) || isInvalidShardingStrategy(each.getTableShardingStrategy(), shardingAlgorithms)) {
-                return false;
-            }
-        }
-        for (ShardingAutoTableRuleConfiguration each : config.getAutoTables()) {
-            if (isInvalidKeyGenerateStrategy(each.getKeyGenerateStrategy(), keyGenerators) || isInvalidAuditStrategy(each.getAuditStrategy(), auditors)
-                    || isInvalidShardingStrategy(each.getShardingStrategy(), shardingAlgorithms)) {
-                return false;
-            }
-        }
-        return true;
-    }
-    
-    private boolean isInvalidKeyGenerateStrategy(final KeyGenerateStrategyConfiguration keyGenerateStrategy, final Map<String, ShardingSphereAlgorithmConfiguration> keyGenerators) {
-        if (null == keyGenerateStrategy) {
-            return false;
-        }
-        return !keyGenerators.containsKey(keyGenerateStrategy.getKeyGeneratorName());
-    }
-    
-    private boolean isInvalidShardingStrategy(final ShardingStrategyConfiguration shardingStrategy, final Map<String, ShardingSphereAlgorithmConfiguration> shardingAlgorithms) {
-        if (null == shardingStrategy || shardingStrategy instanceof NoneShardingStrategyConfiguration) {
-            return false;
-        }
-        return !shardingAlgorithms.containsKey(shardingStrategy.getShardingAlgorithmName());
-    }
-    
-    private boolean isInvalidAuditStrategy(final ShardingAuditStrategyConfiguration auditStrategy, final Map<String, ShardingSphereAlgorithmConfiguration> auditors) {
-        if (null == auditStrategy) {
-            return false;
-        }
-        return !auditors.keySet().containsAll(auditStrategy.getAuditorNames());
-    }
-    
     @Override
     public int getOrder() {
         return ShardingOrder.ORDER;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/checker/AlgorithmProvidedShardingRuleConfigurationCheckerTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/checker/AlgorithmProvidedShardingRuleConfigurationCheckerTest.java
index c99b5410134..493ddc5d527 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/checker/AlgorithmProvidedShardingRuleConfigurationCheckerTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/checker/AlgorithmProvidedShardingRuleConfigurationCheckerTest.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.sharding.algorithm.config.AlgorithmProvidedShar
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.spi.ShardingAlgorithm;
 import org.junit.Test;
 
 import java.util.Collections;
@@ -42,7 +43,10 @@ public final class AlgorithmProvidedShardingRuleConfigurationCheckerTest {
         AlgorithmProvidedShardingRuleConfiguration config = mock(AlgorithmProvidedShardingRuleConfiguration.class);
         when(config.getTables()).thenReturn(Collections.singleton(mock(ShardingTableRuleConfiguration.class)));
         when(config.getAutoTables()).thenReturn(Collections.singleton(mock(ShardingAutoTableRuleConfiguration.class)));
-        when(config.getDefaultTableShardingStrategy()).thenReturn(mock(ShardingStrategyConfiguration.class));
+        ShardingStrategyConfiguration shardingStrategyConfig = mock(ShardingStrategyConfiguration.class);
+        when(shardingStrategyConfig.getShardingAlgorithmName()).thenReturn("t_order_inline");
+        when(config.getDefaultTableShardingStrategy()).thenReturn(shardingStrategyConfig);
+        when(config.getShardingAlgorithms()).thenReturn(Collections.singletonMap("t_order_inline", mock(ShardingAlgorithm.class)));
         Optional<RuleConfigurationChecker> checker = RuleConfigurationCheckerFactory.findInstance(config);
         assertTrue(checker.isPresent());
         assertThat(checker.get(), instanceOf(AlgorithmProvidedShardingRuleConfigurationChecker.class));
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/checker/ShardingRuleConfigurationCheckerTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/checker/ShardingRuleConfigurationCheckerTest.java
index b20c2549990..106aab72ae2 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/checker/ShardingRuleConfigurationCheckerTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/checker/ShardingRuleConfigurationCheckerTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.sharding.checker;
 
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.checker.RuleConfigurationChecker;
 import org.apache.shardingsphere.infra.config.checker.RuleConfigurationCheckerFactory;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
@@ -50,7 +51,10 @@ public final class ShardingRuleConfigurationCheckerTest {
         ShardingRuleConfiguration result = mock(ShardingRuleConfiguration.class);
         when(result.getTables()).thenReturn(Collections.singleton(mock(ShardingTableRuleConfiguration.class)));
         when(result.getAutoTables()).thenReturn(Collections.singleton(mock(ShardingAutoTableRuleConfiguration.class)));
-        when(result.getDefaultTableShardingStrategy()).thenReturn(mock(ShardingStrategyConfiguration.class));
+        ShardingStrategyConfiguration shardingStrategyConfig = mock(ShardingStrategyConfiguration.class);
+        when(shardingStrategyConfig.getShardingAlgorithmName()).thenReturn("t_order_inline");
+        when(result.getDefaultTableShardingStrategy()).thenReturn(shardingStrategyConfig);
+        when(result.getShardingAlgorithms()).thenReturn(Collections.singletonMap("t_order_inline", mock(ShardingSphereAlgorithmConfiguration.class)));
         return result;
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilderTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilderTest.java
index 4089e2bc1ff..c569e249235 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/builder/AlgorithmProvidedShardingRuleBuilderTest.java
@@ -21,10 +21,7 @@ import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.rule.builder.database.DatabaseRuleBuilder;
 import org.apache.shardingsphere.infra.rule.builder.database.DatabaseRuleBuilderFactory;
 import org.apache.shardingsphere.sharding.algorithm.config.AlgorithmProvidedShardingRuleConfiguration;
-import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
-import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.sharding.spi.KeyGenerateAlgorithm;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -67,32 +64,4 @@ public final class AlgorithmProvidedShardingRuleBuilderTest {
     public void assertBuildWithEmptyDataSourceMap() {
         assertThat(builder.build(ruleConfig, "sharding_db", Collections.emptyMap(), Collections.emptyList(), mock(InstanceContext.class)), instanceOf(ShardingRule.class));
     }
-    
-    @SuppressWarnings("unchecked")
-    @Test(expected = IllegalArgumentException.class)
-    public void assertBuildWithInvalidKeyGenerator() {
-        ruleConfig.setDefaultKeyGenerateStrategy(new KeyGenerateStrategyConfiguration("order_id", "snowflake"));
-        assertThat(builder.build(ruleConfig, "sharding_db", Collections.singletonMap("ds_0", mock(DataSource.class)), Collections.emptyList(), mock(InstanceContext.class)),
-                instanceOf(ShardingRule.class));
-    }
-    
-    @SuppressWarnings("unchecked")
-    @Test(expected = IllegalArgumentException.class)
-    public void assertBuildWithInvalidDatabaseShardingStrategy() {
-        ruleConfig.setDefaultKeyGenerateStrategy(new KeyGenerateStrategyConfiguration("order_id", "snowflake"));
-        ruleConfig.getKeyGenerators().put("snowflake", mock(KeyGenerateAlgorithm.class));
-        ruleConfig.setDefaultDatabaseShardingStrategy(new StandardShardingStrategyConfiguration("user_id", "database_inline"));
-        assertThat(builder.build(ruleConfig, "sharding_db", Collections.singletonMap("ds_0", mock(DataSource.class)), Collections.emptyList(), mock(InstanceContext.class)),
-                instanceOf(ShardingRule.class));
-    }
-    
-    @SuppressWarnings("unchecked")
-    @Test(expected = IllegalArgumentException.class)
-    public void assertBuildWithInvalidTableShardingStrategy() {
-        ruleConfig.setDefaultKeyGenerateStrategy(new KeyGenerateStrategyConfiguration("order_id", "snowflake"));
-        ruleConfig.getKeyGenerators().put("snowflake", mock(KeyGenerateAlgorithm.class));
-        ruleConfig.setDefaultTableShardingStrategy(new StandardShardingStrategyConfiguration("order_id", "t_order_inline"));
-        assertThat(builder.build(ruleConfig, "sharding_db", Collections.singletonMap("ds_0", mock(DataSource.class)), Collections.emptyList(), mock(InstanceContext.class)),
-                instanceOf(ShardingRule.class));
-    }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilderTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilderTest.java
index 910f303cb72..e8b0237f02b 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rule/builder/ShardingRuleBuilderTest.java
@@ -17,13 +17,10 @@
 
 package org.apache.shardingsphere.sharding.rule.builder;
 
-import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.rule.builder.database.DatabaseRuleBuilder;
 import org.apache.shardingsphere.infra.rule.builder.database.DatabaseRuleBuilderFactory;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
-import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
-import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.junit.Before;
 import org.junit.Test;
@@ -67,32 +64,4 @@ public final class ShardingRuleBuilderTest {
     public void assertBuildWithEmptyDataSourceMap() {
         assertThat(builder.build(ruleConfig, "sharding_db", Collections.emptyMap(), Collections.emptyList(), mock(InstanceContext.class)), instanceOf(ShardingRule.class));
     }
-    
-    @SuppressWarnings("unchecked")
-    @Test(expected = IllegalArgumentException.class)
-    public void assertBuildWithInvalidKeyGenerator() {
-        ruleConfig.setDefaultKeyGenerateStrategy(new KeyGenerateStrategyConfiguration("order_id", "snowflake"));
-        assertThat(builder.build(ruleConfig, "sharding_db", Collections.singletonMap("ds_0", mock(DataSource.class)), Collections.emptyList(), mock(InstanceContext.class)),
-                instanceOf(ShardingRule.class));
-    }
-    
-    @SuppressWarnings("unchecked")
-    @Test(expected = IllegalArgumentException.class)
-    public void assertBuildWithInvalidDatabaseShardingStrategy() {
-        ruleConfig.setDefaultKeyGenerateStrategy(new KeyGenerateStrategyConfiguration("order_id", "snowflake"));
-        ruleConfig.getKeyGenerators().put("snowflake", mock(ShardingSphereAlgorithmConfiguration.class));
-        ruleConfig.setDefaultDatabaseShardingStrategy(new StandardShardingStrategyConfiguration("user_id", "database_inline"));
-        assertThat(builder.build(ruleConfig, "sharding_db", Collections.singletonMap("ds_0", mock(DataSource.class)), Collections.emptyList(), mock(InstanceContext.class)),
-                instanceOf(ShardingRule.class));
-    }
-    
-    @SuppressWarnings("unchecked")
-    @Test(expected = IllegalArgumentException.class)
-    public void assertBuildWithInvalidTableShardingStrategy() {
-        ruleConfig.setDefaultKeyGenerateStrategy(new KeyGenerateStrategyConfiguration("order_id", "snowflake"));
-        ruleConfig.getKeyGenerators().put("snowflake", mock(ShardingSphereAlgorithmConfiguration.class));
-        ruleConfig.setDefaultTableShardingStrategy(new StandardShardingStrategyConfiguration("order_id", "t_order_inline"));
-        assertThat(builder.build(ruleConfig, "sharding_db", Collections.singletonMap("ds_0", mock(DataSource.class)), Collections.emptyList(), mock(InstanceContext.class)),
-                instanceOf(ShardingRule.class));
-    }
 }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
index 43c102b3d7a..f858a4a3293 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
@@ -70,8 +70,8 @@ public final class ClusterContextManagerBuilder implements ContextManagerBuilder
     }
     
     private void persistMetaData(final MetaDataContexts metaDataContexts) {
-        metaDataContexts.getMetaData().getDatabases().values().forEach(each -> each.getSchemas().forEach((schemaName, tables) ->
-                metaDataContexts.getPersistService().getDatabaseMetaDataService().persistMetaData(each.getName(), schemaName, tables)));
+        metaDataContexts.getMetaData().getDatabases().values().forEach(
+                each -> each.getSchemas().forEach((schemaName, tables) -> metaDataContexts.getPersistService().getDatabaseMetaDataService().persistMetaData(each.getName(), schemaName, tables)));
     }
     
     private void registerOnline(final MetaDataPersistService persistService, final RegistryCenter registryCenter,
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/BootstrapInitializer.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/BootstrapInitializer.java
index 8a5a49cbe45..e4317feca1a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/BootstrapInitializer.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/BootstrapInitializer.java
@@ -19,6 +19,8 @@ package org.apache.shardingsphere.proxy.initializer;
 
 import lombok.RequiredArgsConstructor;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.infra.config.checker.RuleConfigurationCheckerFactory;
+import org.apache.shardingsphere.infra.config.database.DatabaseConfiguration;
 import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.instance.metadata.InstanceMetaData;
@@ -36,6 +38,7 @@ import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.version.ShardingSphereProxyVersion;
 
 import java.sql.SQLException;
+import java.util.Map.Entry;
 
 /**
  * Bootstrap initializer.
@@ -53,14 +56,22 @@ public final class BootstrapInitializer {
      */
     public void init(final YamlProxyConfiguration yamlConfig, final int port) throws SQLException {
         ModeConfiguration modeConfig = null == yamlConfig.getServerConfiguration().getMode() ? null : new ModeConfigurationYamlSwapper().swapToObject(yamlConfig.getServerConfiguration().getMode());
-        ContextManager contextManager = createContextManager(yamlConfig, modeConfig, port);
+        ProxyConfiguration proxyConfig = new YamlProxyConfigurationSwapper().swap(yamlConfig);
+        checkRuleConfiguration(proxyConfig);
+        ContextManager contextManager = createContextManager(proxyConfig, modeConfig, port);
         ProxyContext.init(contextManager);
         contextManagerInitializedCallback(modeConfig, contextManager);
         ShardingSphereProxyVersion.setVersion(contextManager);
     }
     
-    private ContextManager createContextManager(final YamlProxyConfiguration yamlConfig, final ModeConfiguration modeConfig, final int port) throws SQLException {
-        ProxyConfiguration proxyConfig = new YamlProxyConfigurationSwapper().swap(yamlConfig);
+    @SuppressWarnings("unchecked")
+    private void checkRuleConfiguration(final ProxyConfiguration proxyConfig) {
+        for (Entry<String, DatabaseConfiguration> entry : proxyConfig.getDatabaseConfigurations().entrySet()) {
+            entry.getValue().getRuleConfigurations().forEach(each -> RuleConfigurationCheckerFactory.findInstance(each).ifPresent(optional -> optional.check(entry.getKey(), each)));
+        }
+    }
+    
+    private ContextManager createContextManager(final ProxyConfiguration proxyConfig, final ModeConfiguration modeConfig, final int port) throws SQLException {
         ContextManagerBuilderParameter parameter = new ContextManagerBuilderParameter(modeConfig, proxyConfig.getDatabaseConfigurations(),
                 proxyConfig.getGlobalConfiguration().getRules(), proxyConfig.getGlobalConfiguration().getProperties(), proxyConfig.getGlobalConfiguration().getLabels(),
                 createInstanceMetaData(proxyConfig, port));