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 2021/08/06 05:53:50 UTC

[shardingsphere] branch master updated: Shadow core reconstruction (#11676)

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 cf3e166  Shadow core reconstruction (#11676)
cf3e166 is described below

commit cf3e166b16d9a1451ee927658da088dc88399dfa
Author: gin <ja...@163.com>
AuthorDate: Fri Aug 6 13:53:09 2021 +0800

    Shadow core reconstruction (#11676)
    
    * Add shadow algorithm basic implement.
    
    * Refactor the shadow rule builder.
    
    * Refactor the yaml shadow rule configuration.
    
    * Refactor the yaml shadow rule configuration swapper.
---
 .../ColumnRegularMatchShadowAlgorithm.java}        |  26 ++++--
 .../SimpleSQLNoteShadowAlgorithm.java}             |  25 +++--
 .../AlgorithmProvidedShadowRuleConfiguration.java  |  52 +++++++++++
 .../shadow/constant/ShadowOrder.java               |   5 +
 .../shardingsphere/shadow/rule/ShadowRule.java     |  39 ++++++++
 .../AlgorithmProvidedShadowRuleBuilder.java        |  52 +++++++++++
 ...=> AbstractShadowRuleConfigurationChecker.java} |  25 +++--
 ...thmProvidedShadowRuleConfigurationChecker.java} |  19 ++--
 .../checker/ShadowRuleConfigurationChecker.java    |   7 +-
 .../yaml/config/YamlShadowRuleConfiguration.java   |  12 +++
 .../YamlShadowDataSourceConfiguration.java}        |  21 +++--
 .../table/YamlShadowTableConfiguration.java}       |  22 +++--
 ...eAlgorithmProviderConfigurationYamlSwapper.java | 102 +++++++++++++++++++++
 .../ShadowRuleConfigurationYamlSwapper.java        |  62 +++++++++++--
 .../ShadowDataSourceConfigurationYamlSwapper.java  |  41 +++++++++
 .../table/ShadowTableConfigurationYamlSwapper.java |  40 ++++++++
 ...here.infra.rule.builder.scope.SchemaRuleBuilder |   1 +
 ...ere.infra.rule.checker.RuleConfigurationChecker |   1 +
 ...aml.config.swapper.YamlRuleConfigurationSwapper |   1 +
 ...ache.shardingsphere.shadow.spi.ShadowAlgorithm} |   3 +-
 20 files changed, 479 insertions(+), 77 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/ColumnRegularMatchShadowAlgorithm.java
similarity index 63%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/ColumnRegularMatchShadowAlgorithm.java
index 1f65107..91ec03f 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/ColumnRegularMatchShadowAlgorithm.java
@@ -15,19 +15,25 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.shadow.constant;
+package org.apache.shardingsphere.shadow.algorithm;
 
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
+
+import java.util.Properties;
 
 /**
- * Shadow order.
+ * Column match shadow algorithm.
  */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class ShadowOrder {
+@Getter
+@Setter
+public final class ColumnRegularMatchShadowAlgorithm implements ShadowAlgorithm {
+    
+    private Properties props = new Properties();
     
-    /**
-     * Shadow order.
-     */
-    public static final int ORDER = 40;
+    @Override
+    public String getType() {
+        return "COLUMN-REGULAR-MATCH";
+    }
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/SimpleSQLNoteShadowAlgorithm.java
similarity index 64%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/SimpleSQLNoteShadowAlgorithm.java
index 1f65107..7f15cbf 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/SimpleSQLNoteShadowAlgorithm.java
@@ -15,19 +15,24 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.shadow.constant;
+package org.apache.shardingsphere.shadow.algorithm;
 
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
+import java.util.Properties;
 
 /**
- * Shadow order.
+ * Simple note shadow algorithm.
  */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class ShadowOrder {
+@Getter
+@Setter
+public final class SimpleSQLNoteShadowAlgorithm implements ShadowAlgorithm {
     
-    /**
-     * Shadow order.
-     */
-    public static final int ORDER = 40;
+    private Properties props = new Properties();
+    
+    @Override
+    public String getType() {
+        return "SIMPLE-NOTE";
+    }
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/config/AlgorithmProvidedShadowRuleConfiguration.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/config/AlgorithmProvidedShadowRuleConfiguration.java
new file mode 100644
index 0000000..7cb9b57
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/algorithm/config/AlgorithmProvidedShadowRuleConfiguration.java
@@ -0,0 +1,52 @@
+/*
+ * 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.shadow.algorithm.config;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.infra.config.scope.SchemaRuleConfiguration;
+import org.apache.shardingsphere.shadow.api.config.datasource.ShadowDataSourceConfiguration;
+import org.apache.shardingsphere.shadow.api.config.table.ShadowTableConfiguration;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
+
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Algorithm provided shadow rule configuration.
+ */
+@Getter
+@Setter
+@RequiredArgsConstructor
+public final class AlgorithmProvidedShadowRuleConfiguration implements SchemaRuleConfiguration {
+    
+    // fixme remove three fields when the api refactoring is complete
+    private final String column;
+    
+    private final List<String> sourceDataSourceNames;
+    
+    private final List<String> shadowDataSourceNames;
+    
+    private Map<String, ShadowDataSourceConfiguration> dataSources = new LinkedHashMap<>();
+    
+    private Map<String, ShadowTableConfiguration> shadowTables = new LinkedHashMap<>();
+    
+    private Map<String, ShadowAlgorithm> shadowAlgorithms = new LinkedHashMap<>();
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java
index 1f65107..e6bd70a 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java
@@ -30,4 +30,9 @@ public final class ShadowOrder {
      * Shadow order.
      */
     public static final int ORDER = 40;
+    
+    /**
+     * Algorithm provider shadow order.
+     */
+    public static final int ALGORITHM_PROVIDER_ORDER = ORDER + 1;
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java
index 22b2552..b5a2e0f 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/ShadowRule.java
@@ -21,11 +21,17 @@ import lombok.Getter;
 import org.apache.shardingsphere.infra.rule.identifier.level.FeatureRule;
 import org.apache.shardingsphere.infra.rule.identifier.scope.SchemaRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
+import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.shadow.algorithm.config.AlgorithmProvidedShadowRuleConfiguration;
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.api.config.datasource.ShadowDataSourceConfiguration;
+import org.apache.shardingsphere.shadow.api.config.table.ShadowTableConfiguration;
+import org.apache.shardingsphere.shadow.spi.ShadowAlgorithm;
 
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Map.Entry;
 
@@ -35,16 +41,49 @@ import java.util.Map.Entry;
 @Getter
 public final class ShadowRule implements FeatureRule, SchemaRule, DataSourceContainedRule {
     
+    static {
+        ShardingSphereServiceLoader.register(ShadowAlgorithm.class);
+    }
+    
     private final Map<String, String> shadowMappings;
     
     private final String column;
     
+    private final Map<String, ShadowDataSourceConfiguration> dataSources = new LinkedHashMap<>();
+    
+    private final Map<String, ShadowTableConfiguration> shadowTables = new LinkedHashMap<>();
+    
+    private final Map<String, ShadowAlgorithm> shadowAlgorithms = new LinkedHashMap<>();
+    
     public ShadowRule(final ShadowRuleConfiguration shadowRuleConfig) {
         column = shadowRuleConfig.getColumn();
         shadowMappings = new HashMap<>(shadowRuleConfig.getShadowDataSourceNames().size());
         for (int i = 0; i < shadowRuleConfig.getSourceDataSourceNames().size(); i++) {
             shadowMappings.put(shadowRuleConfig.getSourceDataSourceNames().get(i), shadowRuleConfig.getShadowDataSourceNames().get(i));
         }
+        if (!shadowRuleConfig.getDataSources().isEmpty()) {
+            dataSources.putAll(shadowRuleConfig.getDataSources());
+        }
+        if (!shadowRuleConfig.getShadowTables().isEmpty()) {
+            shadowTables.putAll(shadowRuleConfig.getShadowTables());
+        }
+    }
+    
+    public ShadowRule(final AlgorithmProvidedShadowRuleConfiguration shadowRuleConfig) {
+        column = shadowRuleConfig.getColumn();
+        shadowMappings = new HashMap<>(shadowRuleConfig.getShadowDataSourceNames().size());
+        for (int i = 0; i < shadowRuleConfig.getSourceDataSourceNames().size(); i++) {
+            shadowMappings.put(shadowRuleConfig.getSourceDataSourceNames().get(i), shadowRuleConfig.getShadowDataSourceNames().get(i));
+        }
+        if (!shadowRuleConfig.getDataSources().isEmpty()) {
+            dataSources.putAll(shadowRuleConfig.getDataSources());
+        }
+        if (!shadowRuleConfig.getShadowTables().isEmpty()) {
+            shadowTables.putAll(shadowRuleConfig.getShadowTables());
+        }
+        if (!shadowRuleConfig.getShadowAlgorithms().isEmpty()) {
+            shadowAlgorithms.putAll(shadowRuleConfig.getShadowAlgorithms());
+        }
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/builder/AlgorithmProvidedShadowRuleBuilder.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/builder/AlgorithmProvidedShadowRuleBuilder.java
new file mode 100644
index 0000000..678e886
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/builder/AlgorithmProvidedShadowRuleBuilder.java
@@ -0,0 +1,52 @@
+/*
+ * 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.shadow.rule.builder;
+
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.rule.builder.level.FeatureRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.scope.SchemaRuleBuilder;
+import org.apache.shardingsphere.shadow.algorithm.config.AlgorithmProvidedShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.constant.ShadowOrder;
+import org.apache.shardingsphere.shadow.rule.ShadowRule;
+
+import javax.sql.DataSource;
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ * Algorithm provided shadow rule builder.
+ */
+public final class AlgorithmProvidedShadowRuleBuilder implements FeatureRuleBuilder, SchemaRuleBuilder<AlgorithmProvidedShadowRuleConfiguration> {
+    
+    @Override
+    public ShadowRule build(final String schemaName, final Map<String, DataSource> dataSourceMap, final DatabaseType databaseType, final AlgorithmProvidedShadowRuleConfiguration config,
+                            final Collection<ShardingSphereRule> rules) {
+        return new ShadowRule(config);
+    }
+    
+    @Override
+    public int getOrder() {
+        return ShadowOrder.ALGORITHM_PROVIDER_ORDER;
+    }
+    
+    @Override
+    public Class<AlgorithmProvidedShadowRuleConfiguration> getTypeClass() {
+        return AlgorithmProvidedShadowRuleConfiguration.class;
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowRuleConfigurationChecker.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/AbstractShadowRuleConfigurationChecker.java
similarity index 51%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowRuleConfigurationChecker.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/AbstractShadowRuleConfigurationChecker.java
index 19bf7fa..27b34a7 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowRuleConfigurationChecker.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/AbstractShadowRuleConfigurationChecker.java
@@ -18,28 +18,25 @@
 package org.apache.shardingsphere.shadow.rule.checker;
 
 import com.google.common.base.Preconditions;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.checker.RuleConfigurationChecker;
+import org.apache.shardingsphere.shadow.algorithm.config.AlgorithmProvidedShadowRuleConfiguration;
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
-import org.apache.shardingsphere.shadow.constant.ShadowOrder;
 
 /**
- * Shadow rule configuration checker.
+ * Abstract shadow rule configuration checker.
+ *
+ * @param <T> rule configuration
  */
-public final class ShadowRuleConfigurationChecker implements RuleConfigurationChecker<ShadowRuleConfiguration> {
+public abstract class AbstractShadowRuleConfigurationChecker<T extends RuleConfiguration> implements RuleConfigurationChecker<T> {
     
-    @Override
-    public void check(final String schemaName, final ShadowRuleConfiguration config) {
-        boolean isShadow = !config.getColumn().isEmpty() && null != config.getSourceDataSourceNames() && null != config.getShadowDataSourceNames();
+    protected final void checkShadowRule(final String schemaName, final ShadowRuleConfiguration shadowRuleConfiguration) {
+        boolean isShadow = !shadowRuleConfiguration.getColumn().isEmpty() && null != shadowRuleConfiguration.getSourceDataSourceNames() && null != shadowRuleConfiguration.getShadowDataSourceNames();
         Preconditions.checkState(isShadow, "No available shadow rule configuration in `%s` for governance.", schemaName);
     }
     
-    @Override
-    public int getOrder() {
-        return ShadowOrder.ORDER;
-    }
-    
-    @Override
-    public Class<ShadowRuleConfiguration> getTypeClass() {
-        return ShadowRuleConfiguration.class;
+    protected final void checkShadowRule(final String schemaName, final AlgorithmProvidedShadowRuleConfiguration shadowRuleConfiguration) {
+        boolean isShadow = !shadowRuleConfiguration.getColumn().isEmpty() && null != shadowRuleConfiguration.getSourceDataSourceNames() && null != shadowRuleConfiguration.getShadowDataSourceNames();
+        Preconditions.checkState(isShadow, "No available shadow rule configuration in `%s` for governance.", schemaName);
     }
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowRuleConfigurationChecker.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/AlgorithmProvidedShadowRuleConfigurationChecker.java
similarity index 55%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowRuleConfigurationChecker.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/AlgorithmProvidedShadowRuleConfigurationChecker.java
index 19bf7fa..7c5e582 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowRuleConfigurationChecker.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/AlgorithmProvidedShadowRuleConfigurationChecker.java
@@ -17,29 +17,26 @@
 
 package org.apache.shardingsphere.shadow.rule.checker;
 
-import com.google.common.base.Preconditions;
-import org.apache.shardingsphere.infra.rule.checker.RuleConfigurationChecker;
-import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.algorithm.config.AlgorithmProvidedShadowRuleConfiguration;
 import org.apache.shardingsphere.shadow.constant.ShadowOrder;
 
 /**
- * Shadow rule configuration checker.
+ * Algorithm provided shadow rule configuration checker.
  */
-public final class ShadowRuleConfigurationChecker implements RuleConfigurationChecker<ShadowRuleConfiguration> {
+public class AlgorithmProvidedShadowRuleConfigurationChecker extends AbstractShadowRuleConfigurationChecker<AlgorithmProvidedShadowRuleConfiguration> {
     
     @Override
-    public void check(final String schemaName, final ShadowRuleConfiguration config) {
-        boolean isShadow = !config.getColumn().isEmpty() && null != config.getSourceDataSourceNames() && null != config.getShadowDataSourceNames();
-        Preconditions.checkState(isShadow, "No available shadow rule configuration in `%s` for governance.", schemaName);
+    public void check(final String schemaName, final AlgorithmProvidedShadowRuleConfiguration config) {
+        checkShadowRule(schemaName, config);
     }
     
     @Override
     public int getOrder() {
-        return ShadowOrder.ORDER;
+        return ShadowOrder.ALGORITHM_PROVIDER_ORDER;
     }
     
     @Override
-    public Class<ShadowRuleConfiguration> getTypeClass() {
-        return ShadowRuleConfiguration.class;
+    public Class<AlgorithmProvidedShadowRuleConfiguration> getTypeClass() {
+        return AlgorithmProvidedShadowRuleConfiguration.class;
     }
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowRuleConfigurationChecker.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowRuleConfigurationChecker.java
index 19bf7fa..5051506 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowRuleConfigurationChecker.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/rule/checker/ShadowRuleConfigurationChecker.java
@@ -17,20 +17,17 @@
 
 package org.apache.shardingsphere.shadow.rule.checker;
 
-import com.google.common.base.Preconditions;
-import org.apache.shardingsphere.infra.rule.checker.RuleConfigurationChecker;
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
 import org.apache.shardingsphere.shadow.constant.ShadowOrder;
 
 /**
  * Shadow rule configuration checker.
  */
-public final class ShadowRuleConfigurationChecker implements RuleConfigurationChecker<ShadowRuleConfiguration> {
+public final class ShadowRuleConfigurationChecker extends AbstractShadowRuleConfigurationChecker<ShadowRuleConfiguration> {
     
     @Override
     public void check(final String schemaName, final ShadowRuleConfiguration config) {
-        boolean isShadow = !config.getColumn().isEmpty() && null != config.getSourceDataSourceNames() && null != config.getShadowDataSourceNames();
-        Preconditions.checkState(isShadow, "No available shadow rule configuration in `%s` for governance.", schemaName);
+        checkShadowRule(schemaName, config);
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/config/YamlShadowRuleConfiguration.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/config/YamlShadowRuleConfiguration.java
index 51a3868..1ddd258 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/config/YamlShadowRuleConfiguration.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/config/YamlShadowRuleConfiguration.java
@@ -20,9 +20,14 @@ package org.apache.shardingsphere.shadow.yaml.config;
 import lombok.Getter;
 import lombok.Setter;
 import org.apache.shardingsphere.infra.yaml.config.pojo.YamlRuleConfiguration;
+import org.apache.shardingsphere.infra.yaml.config.pojo.algorithm.YamlShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.yaml.config.datasource.YamlShadowDataSourceConfiguration;
+import org.apache.shardingsphere.shadow.yaml.config.table.YamlShadowTableConfiguration;
 
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 
 /**
  * Shadow rule configuration.
@@ -31,12 +36,19 @@ import java.util.List;
 @Setter
 public final class YamlShadowRuleConfiguration implements YamlRuleConfiguration {
     
+    // fixme remove three fields when the api refactoring is complete
     private String column;
     
     private List<String> sourceDataSourceNames;
     
     private List<String> shadowDataSourceNames;
     
+    private Map<String, YamlShadowDataSourceConfiguration> dataSources = new LinkedHashMap<>();
+    
+    private Map<String, YamlShadowTableConfiguration> shadowTables = new LinkedHashMap<>();
+    
+    private Map<String, YamlShardingSphereAlgorithmConfiguration> shadowAlgorithms = new LinkedHashMap<>();
+    
     @Override
     public Class<ShadowRuleConfiguration> getRuleConfigurationType() {
         return ShadowRuleConfiguration.class;
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/config/datasource/YamlShadowDataSourceConfiguration.java
similarity index 66%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/config/datasource/YamlShadowDataSourceConfiguration.java
index 1f65107..680f71c 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/config/datasource/YamlShadowDataSourceConfiguration.java
@@ -15,19 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.shadow.constant;
+package org.apache.shardingsphere.shadow.yaml.config.datasource;
 
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.infra.yaml.config.pojo.YamlConfiguration;
 
 /**
- * Shadow order.
+ * Shadow data source configuration for YAML.
  */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class ShadowOrder {
+@Getter
+@Setter
+public final class YamlShadowDataSourceConfiguration implements YamlConfiguration {
     
-    /**
-     * Shadow order.
-     */
-    public static final int ORDER = 40;
+    private String sourceDataSourceName;
+    
+    private String shadowDataSourceName;
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/config/table/YamlShadowTableConfiguration.java
similarity index 64%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/config/table/YamlShadowTableConfiguration.java
index 1f65107..572a6b4 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/constant/ShadowOrder.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/config/table/YamlShadowTableConfiguration.java
@@ -15,19 +15,21 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.shadow.constant;
+package org.apache.shardingsphere.shadow.yaml.config.table;
 
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.infra.yaml.config.pojo.YamlConfiguration;
+
+import java.util.Collection;
+import java.util.LinkedList;
 
 /**
- * Shadow order.
+ * Shadow table configuration for YAML.
  */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class ShadowOrder {
+@Getter
+@Setter
+public final class YamlShadowTableConfiguration implements YamlConfiguration {
     
-    /**
-     * Shadow order.
-     */
-    public static final int ORDER = 40;
+    private Collection<String> shadowAlgorithmNames = new LinkedList<>();
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/ShadowRuleAlgorithmProviderConfigurationYamlSwapper.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/ShadowRuleAlgorithmProviderConfigurationYamlSwapper.java
new file mode 100644
index 0000000..923d8f2
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/ShadowRuleAlgorithmProviderConfigurationYamlSwapper.java
@@ -0,0 +1,102 @@
+/*
+ * 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.shadow.yaml.swapper;
+
+import org.apache.shardingsphere.infra.yaml.config.pojo.algorithm.YamlShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.infra.yaml.config.swapper.YamlRuleConfigurationSwapper;
+import org.apache.shardingsphere.shadow.algorithm.config.AlgorithmProvidedShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.constant.ShadowOrder;
+import org.apache.shardingsphere.shadow.yaml.config.YamlShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.yaml.swapper.datasource.ShadowDataSourceConfigurationYamlSwapper;
+import org.apache.shardingsphere.shadow.yaml.swapper.table.ShadowTableConfigurationYamlSwapper;
+
+/**
+ * Shadow rule algorithm provider configuration YAML swapper.
+ */
+public final class ShadowRuleAlgorithmProviderConfigurationYamlSwapper implements YamlRuleConfigurationSwapper<YamlShadowRuleConfiguration, AlgorithmProvidedShadowRuleConfiguration> {
+    
+    private final ShadowDataSourceConfigurationYamlSwapper dataSourceConfigurationSwapper = new ShadowDataSourceConfigurationYamlSwapper();
+    
+    private final ShadowTableConfigurationYamlSwapper tableConfigurationYamlSwapper = new ShadowTableConfigurationYamlSwapper();
+    
+    @Override
+    public YamlShadowRuleConfiguration swapToYamlConfiguration(final AlgorithmProvidedShadowRuleConfiguration dataConfiguration) {
+        YamlShadowRuleConfiguration result = new YamlShadowRuleConfiguration();
+        parseBasicShadowRule(dataConfiguration, result);
+        parseDataSources(dataConfiguration, result);
+        parseShadowTables(dataConfiguration, result);
+        parseShadowAlgorithms(dataConfiguration, result);
+        return result;
+    }
+    
+    private void parseShadowAlgorithms(final AlgorithmProvidedShadowRuleConfiguration dataConfiguration, final YamlShadowRuleConfiguration yamlConfiguration) {
+        dataConfiguration.getShadowAlgorithms().forEach((key, value) ->
+                yamlConfiguration.getShadowAlgorithms().put(key, new YamlShardingSphereAlgorithmConfiguration(value.getType(), value.getProps())));
+    }
+    
+    private void parseShadowTables(final AlgorithmProvidedShadowRuleConfiguration dataConfiguration, final YamlShadowRuleConfiguration yamlConfiguration) {
+        dataConfiguration.getShadowTables().forEach((key, value) -> yamlConfiguration.getShadowTables().put(key, tableConfigurationYamlSwapper.swapToYamlConfiguration(value)));
+    }
+    
+    private void parseDataSources(final AlgorithmProvidedShadowRuleConfiguration dataConfiguration, final YamlShadowRuleConfiguration yamlConfiguration) {
+        dataConfiguration.getDataSources().forEach((key, value) -> yamlConfiguration.getDataSources().put(key, dataSourceConfigurationSwapper.swapToYamlConfiguration(value)));
+    }
+    
+    // fixme remove method when the api refactoring is complete
+    private void parseBasicShadowRule(final AlgorithmProvidedShadowRuleConfiguration dataConfiguration, final YamlShadowRuleConfiguration yamlConfiguration) {
+        yamlConfiguration.setColumn(dataConfiguration.getColumn());
+        yamlConfiguration.setSourceDataSourceNames(dataConfiguration.getSourceDataSourceNames());
+        yamlConfiguration.setShadowDataSourceNames(dataConfiguration.getShadowDataSourceNames());
+    }
+    
+    @Override
+    public AlgorithmProvidedShadowRuleConfiguration swapToObject(final YamlShadowRuleConfiguration yamlConfiguration) {
+        AlgorithmProvidedShadowRuleConfiguration result = createBasicAlgorithmProvidedShadowRule(yamlConfiguration);
+        parseYamlDataSources(yamlConfiguration, result);
+        parseYamlShadowTables(yamlConfiguration, result);
+        return result;
+    }
+    
+    private void parseYamlShadowTables(final YamlShadowRuleConfiguration yamlConfiguration, final AlgorithmProvidedShadowRuleConfiguration dataConfiguration) {
+        yamlConfiguration.getShadowTables().forEach((key, value) -> dataConfiguration.getShadowTables().put(key, tableConfigurationYamlSwapper.swapToObject(value)));
+    }
+    
+    private void parseYamlDataSources(final YamlShadowRuleConfiguration yamlConfiguration, final AlgorithmProvidedShadowRuleConfiguration dataConfiguration) {
+        yamlConfiguration.getDataSources().forEach((key, value) -> dataConfiguration.getDataSources().put(key, dataSourceConfigurationSwapper.swapToObject(value)));
+    }
+    
+    // fixme remove method when the api refactoring is complete
+    private AlgorithmProvidedShadowRuleConfiguration createBasicAlgorithmProvidedShadowRule(final YamlShadowRuleConfiguration yamlConfiguration) {
+        return new AlgorithmProvidedShadowRuleConfiguration(yamlConfiguration.getColumn(), yamlConfiguration.getSourceDataSourceNames(), yamlConfiguration.getShadowDataSourceNames());
+    }
+    
+    @Override
+    public Class<AlgorithmProvidedShadowRuleConfiguration> getTypeClass() {
+        return AlgorithmProvidedShadowRuleConfiguration.class;
+    }
+    
+    @Override
+    public String getRuleTagName() {
+        return "SHADOW";
+    }
+    
+    @Override
+    public int getOrder() {
+        return ShadowOrder.ALGORITHM_PROVIDER_ORDER;
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/ShadowRuleConfigurationYamlSwapper.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/ShadowRuleConfigurationYamlSwapper.java
index aec47ce..bf1b9e9 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/ShadowRuleConfigurationYamlSwapper.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/ShadowRuleConfigurationYamlSwapper.java
@@ -18,27 +18,77 @@
 package org.apache.shardingsphere.shadow.yaml.swapper;
 
 import org.apache.shardingsphere.infra.yaml.config.swapper.YamlRuleConfigurationSwapper;
+import org.apache.shardingsphere.infra.yaml.config.swapper.algorithm.ShardingSphereAlgorithmConfigurationYamlSwapper;
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
 import org.apache.shardingsphere.shadow.constant.ShadowOrder;
 import org.apache.shardingsphere.shadow.yaml.config.YamlShadowRuleConfiguration;
+import org.apache.shardingsphere.shadow.yaml.swapper.datasource.ShadowDataSourceConfigurationYamlSwapper;
+import org.apache.shardingsphere.shadow.yaml.swapper.table.ShadowTableConfigurationYamlSwapper;
 
 /**
  * Shadow rule configuration yaml swapper.
  */
 public final class ShadowRuleConfigurationYamlSwapper implements YamlRuleConfigurationSwapper<YamlShadowRuleConfiguration, ShadowRuleConfiguration> {
     
+    private final ShadowDataSourceConfigurationYamlSwapper dataSourceConfigurationSwapper = new ShadowDataSourceConfigurationYamlSwapper();
+    
+    private final ShadowTableConfigurationYamlSwapper tableConfigurationYamlSwapper = new ShadowTableConfigurationYamlSwapper();
+    
+    private final ShardingSphereAlgorithmConfigurationYamlSwapper algorithmSwapper = new ShardingSphereAlgorithmConfigurationYamlSwapper();
+    
     @Override
-    public YamlShadowRuleConfiguration swapToYamlConfiguration(final ShadowRuleConfiguration data) {
+    public YamlShadowRuleConfiguration swapToYamlConfiguration(final ShadowRuleConfiguration dataConfiguration) {
         YamlShadowRuleConfiguration result = new YamlShadowRuleConfiguration();
-        result.setColumn(data.getColumn());
-        result.setSourceDataSourceNames(data.getSourceDataSourceNames());
-        result.setShadowDataSourceNames(data.getShadowDataSourceNames());
+        parseBasicShadowRule(dataConfiguration, result);
+        parseDataSources(dataConfiguration, result);
+        parseShadowTables(dataConfiguration, result);
+        parseShadowAlgorithms(dataConfiguration, result);
         return result;
     }
     
+    private void parseShadowAlgorithms(final ShadowRuleConfiguration dataConfiguration, final YamlShadowRuleConfiguration yamlConfiguration) {
+        dataConfiguration.getShadowAlgorithms().forEach((key, value) -> yamlConfiguration.getShadowAlgorithms().put(key, algorithmSwapper.swapToYamlConfiguration(value)));
+    }
+    
+    private void parseShadowTables(final ShadowRuleConfiguration dataConfiguration, final YamlShadowRuleConfiguration yamlConfiguration) {
+        dataConfiguration.getShadowTables().forEach((key, value) -> yamlConfiguration.getShadowTables().put(key, tableConfigurationYamlSwapper.swapToYamlConfiguration(value)));
+    }
+    
+    private void parseDataSources(final ShadowRuleConfiguration dataConfiguration, final YamlShadowRuleConfiguration yamlConfiguration) {
+        dataConfiguration.getDataSources().forEach((key, value) -> yamlConfiguration.getDataSources().put(key, dataSourceConfigurationSwapper.swapToYamlConfiguration(value)));
+    }
+    
+    // fixme remove method when the api refactoring is complete
+    private void parseBasicShadowRule(final ShadowRuleConfiguration dataConfiguration, final YamlShadowRuleConfiguration yamlConfiguration) {
+        yamlConfiguration.setColumn(dataConfiguration.getColumn());
+        yamlConfiguration.setSourceDataSourceNames(dataConfiguration.getSourceDataSourceNames());
+        yamlConfiguration.setShadowDataSourceNames(dataConfiguration.getShadowDataSourceNames());
+    }
+    
     @Override
-    public ShadowRuleConfiguration swapToObject(final YamlShadowRuleConfiguration yamlConfig) {
-        return new ShadowRuleConfiguration(yamlConfig.getColumn(), yamlConfig.getSourceDataSourceNames(), yamlConfig.getShadowDataSourceNames());
+    public ShadowRuleConfiguration swapToObject(final YamlShadowRuleConfiguration yamlConfiguration) {
+        ShadowRuleConfiguration result = createBasicShadowRule(yamlConfiguration);
+        parseYamlDataSources(yamlConfiguration, result);
+        parseYamlShadowTables(yamlConfiguration, result);
+        parseYamlShadowAlgorithms(yamlConfiguration, result);
+        return result;
+    }
+    
+    private void parseYamlShadowAlgorithms(final YamlShadowRuleConfiguration yamlConfiguration, final ShadowRuleConfiguration dataConfiguration) {
+        yamlConfiguration.getShadowAlgorithms().forEach((key, value) -> dataConfiguration.getShadowAlgorithms().put(key, algorithmSwapper.swapToObject(value)));
+    }
+    
+    private void parseYamlShadowTables(final YamlShadowRuleConfiguration yamlConfiguration, final ShadowRuleConfiguration dataConfiguration) {
+        yamlConfiguration.getShadowTables().forEach((key, value) -> dataConfiguration.getShadowTables().put(key, tableConfigurationYamlSwapper.swapToObject(value)));
+    }
+    
+    private void parseYamlDataSources(final YamlShadowRuleConfiguration yamlConfiguration, final ShadowRuleConfiguration dataConfiguration) {
+        yamlConfiguration.getDataSources().forEach((key, value) -> dataConfiguration.getDataSources().put(key, dataSourceConfigurationSwapper.swapToObject(value)));
+    }
+    
+    // fixme remove method when the api refactoring is complete
+    private ShadowRuleConfiguration createBasicShadowRule(final YamlShadowRuleConfiguration yamlConfiguration) {
+        return new ShadowRuleConfiguration(yamlConfiguration.getColumn(), yamlConfiguration.getSourceDataSourceNames(), yamlConfiguration.getShadowDataSourceNames());
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/datasource/ShadowDataSourceConfigurationYamlSwapper.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/datasource/ShadowDataSourceConfigurationYamlSwapper.java
new file mode 100644
index 0000000..46e4a9d
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/datasource/ShadowDataSourceConfigurationYamlSwapper.java
@@ -0,0 +1,41 @@
+/*
+ * 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.shadow.yaml.swapper.datasource;
+
+import org.apache.shardingsphere.infra.yaml.config.swapper.YamlConfigurationSwapper;
+import org.apache.shardingsphere.shadow.api.config.datasource.ShadowDataSourceConfiguration;
+import org.apache.shardingsphere.shadow.yaml.config.datasource.YamlShadowDataSourceConfiguration;
+
+/**
+ * Shadow data source configuration YAML swapper.
+ */
+public final class ShadowDataSourceConfigurationYamlSwapper implements YamlConfigurationSwapper<YamlShadowDataSourceConfiguration, ShadowDataSourceConfiguration> {
+    
+    @Override
+    public YamlShadowDataSourceConfiguration swapToYamlConfiguration(final ShadowDataSourceConfiguration data) {
+        YamlShadowDataSourceConfiguration result = new YamlShadowDataSourceConfiguration();
+        result.setSourceDataSourceName(data.getSourceDataSourceName());
+        result.setShadowDataSourceName(data.getShadowDataSourceName());
+        return result;
+    }
+    
+    @Override
+    public ShadowDataSourceConfiguration swapToObject(final YamlShadowDataSourceConfiguration yamlConfig) {
+        return new ShadowDataSourceConfiguration(yamlConfig.getSourceDataSourceName(), yamlConfig.getShadowDataSourceName());
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/table/ShadowTableConfigurationYamlSwapper.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/table/ShadowTableConfigurationYamlSwapper.java
new file mode 100644
index 0000000..44f5dba
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/table/ShadowTableConfigurationYamlSwapper.java
@@ -0,0 +1,40 @@
+/*
+ * 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.shadow.yaml.swapper.table;
+
+import org.apache.shardingsphere.infra.yaml.config.swapper.YamlConfigurationSwapper;
+import org.apache.shardingsphere.shadow.api.config.table.ShadowTableConfiguration;
+import org.apache.shardingsphere.shadow.yaml.config.table.YamlShadowTableConfiguration;
+
+/**
+ * Shadow table configuration YAML swapper.
+ */
+public class ShadowTableConfigurationYamlSwapper implements YamlConfigurationSwapper<YamlShadowTableConfiguration, ShadowTableConfiguration> {
+    
+    @Override
+    public YamlShadowTableConfiguration swapToYamlConfiguration(final ShadowTableConfiguration data) {
+        YamlShadowTableConfiguration result = new YamlShadowTableConfiguration();
+        result.setShadowAlgorithmNames(data.getShadowAlgorithmNames());
+        return result;
+    }
+    
+    @Override
+    public ShadowTableConfiguration swapToObject(final YamlShadowTableConfiguration yamlConfig) {
+        return new ShadowTableConfiguration(yamlConfig.getShadowAlgorithmNames());
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.scope.SchemaRuleBuilder b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.scope.SchemaRuleBuilder
index fc46697..92e2638 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.scope.SchemaRuleBuilder
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.scope.SchemaRuleBuilder
@@ -16,3 +16,4 @@
 #
 
 org.apache.shardingsphere.shadow.rule.builder.ShadowRuleBuilder
+org.apache.shardingsphere.shadow.rule.builder.AlgorithmProvidedShadowRuleBuilder
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.checker.RuleConfigurationChecker b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.checker.RuleConfigurationChecker
index d3e2430..fc6ddd2 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.checker.RuleConfigurationChecker
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.checker.RuleConfigurationChecker
@@ -16,3 +16,4 @@
 #
 
 org.apache.shardingsphere.shadow.rule.checker.ShadowRuleConfigurationChecker
+org.apache.shardingsphere.shadow.rule.checker.AlgorithmProvidedShadowRuleConfigurationChecker
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.YamlRuleConfigurationSwapper b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.YamlRuleConfigurationSwapper
index 8524f07..9ba39dc 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.YamlRuleConfigurationSwapper
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.YamlRuleConfigurationSwapper
@@ -16,3 +16,4 @@
 #
 
 org.apache.shardingsphere.shadow.yaml.swapper.ShadowRuleConfigurationYamlSwapper
+org.apache.shardingsphere.shadow.yaml.swapper.ShadowRuleAlgorithmProviderConfigurationYamlSwapper
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.scope.SchemaRuleBuilder b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.shadow.spi.ShadowAlgorithm
similarity index 84%
copy from shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.scope.SchemaRuleBuilder
copy to shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.shadow.spi.ShadowAlgorithm
index fc46697..0ccb32c 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.scope.SchemaRuleBuilder
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-core/src/main/resources/META-INF/services/org.apache.shardingsphere.shadow.spi.ShadowAlgorithm
@@ -15,4 +15,5 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.shadow.rule.builder.ShadowRuleBuilder
+org.apache.shardingsphere.shadow.algorithm.ColumnRegularMatchShadowAlgorithm
+org.apache.shardingsphere.shadow.algorithm.SimpleSQLNoteShadowAlgorithm