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 2023/06/07 13:30:45 UTC

[shardingsphere] branch master updated: Improve swap yaml data to rule configuration in shadow (#26101)

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 7fb3849f650 Improve swap yaml data to rule configuration in shadow (#26101)
7fb3849f650 is described below

commit 7fb3849f65099558a94709ed2ef199db90e8b248
Author: ChenJiaHao <Pa...@163.com>
AuthorDate: Wed Jun 7 21:30:34 2023 +0800

    Improve swap yaml data to rule configuration in shadow (#26101)
---
 .../mask/metadata/converter/MaskNodeConverter.java |   2 +-
 .../NewYamlMaskRuleConfigurationSwapper.java       |   2 +-
 .../metadata/converter/MaskNodeConverterTest.java  |   2 +-
 .../metadata/converter/ShadowNodeConverter.java    | 128 +++++++++++++++++++--
 .../NewYamlShadowRuleConfigurationSwapper.java     |  37 +++++-
 .../converter/ShadowNodeConverterTest.java         |  85 ++++++++++++++
 .../NewYamlShadowRuleConfigurationSwapperTest.java |  39 ++++++-
 7 files changed, 274 insertions(+), 21 deletions(-)

diff --git a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/metadata/converter/MaskNodeConverter.java b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/metadata/converter/MaskNodeConverter.java
index 3bcfde93901..a38af5a3d3c 100644
--- a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/metadata/converter/MaskNodeConverter.java
+++ b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/metadata/converter/MaskNodeConverter.java
@@ -54,7 +54,7 @@ public final class MaskNodeConverter {
      * @param maskAlgorithmName mask algorithm name
      * @return mask algorithm path
      */
-    public static String getMaskAlgorithmNamePath(final String maskAlgorithmName) {
+    public static String getMaskAlgorithmPath(final String maskAlgorithmName) {
         return String.join("/", ALGORITHMS_NODE, maskAlgorithmName);
     }
     
diff --git a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/yaml/swapper/NewYamlMaskRuleConfigurationSwapper.java b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/yaml/swapper/NewYamlMaskRuleConfigurationSwapper.java
index d39288be014..ca2dd407de0 100644
--- a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/yaml/swapper/NewYamlMaskRuleConfigurationSwapper.java
+++ b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/yaml/swapper/NewYamlMaskRuleConfigurationSwapper.java
@@ -54,7 +54,7 @@ public final class NewYamlMaskRuleConfigurationSwapper implements NewYamlRuleCon
             result.add(new YamlDataNode(MaskNodeConverter.getTableNamePath(each.getName()), YamlEngine.marshal(tableSwapper.swapToYamlConfiguration(each))));
         }
         for (Entry<String, AlgorithmConfiguration> entry : data.getMaskAlgorithms().entrySet()) {
-            result.add(new YamlDataNode(MaskNodeConverter.getMaskAlgorithmNamePath(entry.getKey()), YamlEngine.marshal(algorithmSwapper.swapToYamlConfiguration(entry.getValue()))));
+            result.add(new YamlDataNode(MaskNodeConverter.getMaskAlgorithmPath(entry.getKey()), YamlEngine.marshal(algorithmSwapper.swapToYamlConfiguration(entry.getValue()))));
         }
         return result;
     }
diff --git a/features/mask/core/src/test/java/org/apache/shardingsphere/mask/metadata/converter/MaskNodeConverterTest.java b/features/mask/core/src/test/java/org/apache/shardingsphere/mask/metadata/converter/MaskNodeConverterTest.java
index eb6899dc6be..5ea8603207e 100644
--- a/features/mask/core/src/test/java/org/apache/shardingsphere/mask/metadata/converter/MaskNodeConverterTest.java
+++ b/features/mask/core/src/test/java/org/apache/shardingsphere/mask/metadata/converter/MaskNodeConverterTest.java
@@ -35,7 +35,7 @@ class MaskNodeConverterTest {
     
     @Test
     void assertGetAlgorithmPath() {
-        assertThat(MaskNodeConverter.getMaskAlgorithmNamePath("MD5"), is("algorithms/MD5"));
+        assertThat(MaskNodeConverter.getMaskAlgorithmPath("MD5"), is("algorithms/MD5"));
     }
     
     @Test
diff --git a/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/metadata/converter/ShadowNodeConverter.java b/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/metadata/converter/ShadowNodeConverter.java
index e7f3f8e3f36..e871f7049ab 100644
--- a/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/metadata/converter/ShadowNodeConverter.java
+++ b/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/metadata/converter/ShadowNodeConverter.java
@@ -20,56 +20,160 @@ package org.apache.shardingsphere.shadow.metadata.converter;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 
+import java.util.Optional;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 /**
  * Shadow node converter.
  */
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
 public final class ShadowNodeConverter {
     
-    private static final String DATA_SOURCES = "data_sources";
+    private static final String ROOT_NODE = "shadow";
+    
+    private static final String DATA_SOURCES_NODE = "data_sources";
+    
+    private static final String TABLES_NODE = "tables";
     
-    private static final String TABLES = "tables";
+    private static final String ALGORITHMS_NODE = "algorithms";
     
-    private static final String SHADOW_ALGORITHMS = "shadow_algorithms";
+    private static final String DEFAULT_ALGORITHM_NAME = "default_algorithm_name";
     
-    private static final String DEFAULT_SHADOW_ALGORITHM_NAME = "default_shadow_algorithm_name";
+    private static final String RULES_NODE_PREFIX = "/([\\w\\-]+)/([\\w\\-]+)/rules/";
     
     /**
      * Get data source path.
-     * 
+     *
      * @param dataSourceName data source name
      * @return data source path
      */
     public static String getDataSourcePath(final String dataSourceName) {
-        return String.join("/", DATA_SOURCES, dataSourceName);
+        return String.join("/", DATA_SOURCES_NODE, dataSourceName);
     }
     
     /**
      * Get table name path.
-     * 
+     *
      * @param tableName table name
      * @return table name path
      */
     public static String getTableNamePath(final String tableName) {
-        return String.join("/", TABLES, tableName);
+        return String.join("/", TABLES_NODE, tableName);
     }
     
     /**
      * Get shadow algorithm path.
-     * 
+     *
      * @param shadowAlgorithmName shadow algorithm name
      * @return shadow algorithm path
      */
     public static String getShadowAlgorithmPath(final String shadowAlgorithmName) {
-        return String.join("/", SHADOW_ALGORITHMS, shadowAlgorithmName);
+        return String.join("/", ALGORITHMS_NODE, shadowAlgorithmName);
     }
     
     /**
      * Get default shadow algorithm path.
-     * 
+     *
      * @return default shadow algorithm path
      */
     public static String getDefaultShadowAlgorithmPath() {
-        return String.join("/", DEFAULT_SHADOW_ALGORITHM_NAME);
+        return String.join("/", DEFAULT_ALGORITHM_NAME);
+    }
+    
+    /**
+     * Is shadow path.
+     *
+     * @param rulePath rule path
+     * @return true or false
+     */
+    public static boolean isShadowPath(final String rulePath) {
+        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "\\.*", Pattern.CASE_INSENSITIVE);
+        Matcher matcher = pattern.matcher(rulePath);
+        return matcher.find();
+    }
+    
+    /**
+     * Is shadow data sources path.
+     *
+     * @param rulePath rule path
+     * @return true or false
+     */
+    public static boolean isDataSourcePath(final String rulePath) {
+        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "/" + DATA_SOURCES_NODE + "\\.*", Pattern.CASE_INSENSITIVE);
+        Matcher matcher = pattern.matcher(rulePath);
+        return matcher.find();
+    }
+    
+    /**
+     * Is shadow table path.
+     *
+     * @param rulePath rule path
+     * @return true or false
+     */
+    public static boolean isTablePath(final String rulePath) {
+        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "/" + TABLES_NODE + "\\.*", Pattern.CASE_INSENSITIVE);
+        Matcher matcher = pattern.matcher(rulePath);
+        return matcher.find();
+    }
+    
+    /**
+     * Is shadow algorithm path.
+     *
+     * @param rulePath rule path
+     * @return true or false
+     */
+    public static boolean isAlgorithmPath(final String rulePath) {
+        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "/" + ALGORITHMS_NODE + "\\.*", Pattern.CASE_INSENSITIVE);
+        Matcher matcher = pattern.matcher(rulePath);
+        return matcher.find();
+    }
+    
+    /**
+     * Is default algorithm name path.
+     *
+     * @param rulePath rule path
+     * @return true or false
+     */
+    public static boolean isDefaultAlgorithmNamePath(final String rulePath) {
+        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "/" + DEFAULT_ALGORITHM_NAME + "$", Pattern.CASE_INSENSITIVE);
+        Matcher matcher = pattern.matcher(rulePath);
+        return matcher.find();
+    }
+    
+    /**
+     * Get data source name.
+     *
+     * @param rulePath rule path
+     * @return data source name
+     */
+    public static Optional<String> getDataSourceName(final String rulePath) {
+        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "/" + DATA_SOURCES_NODE + "/([\\w\\-]+)?", Pattern.CASE_INSENSITIVE);
+        Matcher matcher = pattern.matcher(rulePath);
+        return matcher.find() ? Optional.of(matcher.group(3)) : Optional.empty();
+    }
+    
+    /**
+     * Get table name.
+     *
+     * @param rulePath rule path
+     * @return table name
+     */
+    public static Optional<String> getTableName(final String rulePath) {
+        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "/" + TABLES_NODE + "/([\\w\\-]+)?", Pattern.CASE_INSENSITIVE);
+        Matcher matcher = pattern.matcher(rulePath);
+        return matcher.find() ? Optional.of(matcher.group(3)) : Optional.empty();
+    }
+    
+    /**
+     * Get algorithm name.
+     *
+     * @param rulePath rule path
+     * @return algorithm name
+     */
+    public static Optional<String> getAlgorithmName(final String rulePath) {
+        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "/" + ALGORITHMS_NODE + "/([\\w\\-]+)?", Pattern.CASE_INSENSITIVE);
+        Matcher matcher = pattern.matcher(rulePath);
+        return matcher.find() ? Optional.of(matcher.group(3)) : Optional.empty();
     }
 }
diff --git a/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/NewYamlShadowRuleConfigurationSwapper.java b/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/NewYamlShadowRuleConfigurationSwapper.java
index 633612f2a32..50ad3e52ce8 100644
--- a/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/NewYamlShadowRuleConfigurationSwapper.java
+++ b/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/yaml/swapper/NewYamlShadowRuleConfigurationSwapper.java
@@ -21,6 +21,7 @@ import com.google.common.base.Strings;
 import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
 import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
 import org.apache.shardingsphere.infra.util.yaml.datanode.YamlDataNode;
+import org.apache.shardingsphere.infra.yaml.config.pojo.algorithm.YamlAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.yaml.config.swapper.algorithm.YamlAlgorithmConfigurationSwapper;
 import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper;
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
@@ -29,10 +30,14 @@ import org.apache.shardingsphere.shadow.api.config.table.ShadowTableConfiguratio
 import org.apache.shardingsphere.shadow.constant.ShadowOrder;
 import org.apache.shardingsphere.shadow.metadata.converter.ShadowNodeConverter;
 import org.apache.shardingsphere.shadow.yaml.config.datasource.YamlShadowDataSourceConfiguration;
+import org.apache.shardingsphere.shadow.yaml.config.table.YamlShadowTableConfiguration;
 import org.apache.shardingsphere.shadow.yaml.swapper.table.YamlShadowTableConfigurationSwapper;
 
 import java.util.Collection;
+import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.Map;
 import java.util.Map.Entry;
 
 /**
@@ -41,7 +46,7 @@ import java.util.Map.Entry;
  */
 public final class NewYamlShadowRuleConfigurationSwapper implements NewYamlRuleConfigurationSwapper<ShadowRuleConfiguration> {
     
-    private final YamlShadowTableConfigurationSwapper tableConfigurationSwapper = new YamlShadowTableConfigurationSwapper();
+    private final YamlShadowTableConfigurationSwapper tableSwapper = new YamlShadowTableConfigurationSwapper();
     
     private final YamlAlgorithmConfigurationSwapper algorithmSwapper = new YamlAlgorithmConfigurationSwapper();
     
@@ -52,7 +57,7 @@ public final class NewYamlShadowRuleConfigurationSwapper implements NewYamlRuleC
             result.add(new YamlDataNode(ShadowNodeConverter.getDataSourcePath(each.getName()), YamlEngine.marshal(swapToDataSourceYamlConfiguration(each))));
         }
         for (Entry<String, ShadowTableConfiguration> entry : data.getTables().entrySet()) {
-            result.add(new YamlDataNode(ShadowNodeConverter.getTableNamePath(entry.getKey()), YamlEngine.marshal(tableConfigurationSwapper.swapToYamlConfiguration(entry.getValue()))));
+            result.add(new YamlDataNode(ShadowNodeConverter.getTableNamePath(entry.getKey()), YamlEngine.marshal(tableSwapper.swapToYamlConfiguration(entry.getValue()))));
         }
         for (Entry<String, AlgorithmConfiguration> entry : data.getShadowAlgorithms().entrySet()) {
             result.add(new YamlDataNode(ShadowNodeConverter.getShadowAlgorithmPath(entry.getKey()), YamlEngine.marshal(algorithmSwapper.swapToYamlConfiguration(entry.getValue()))));
@@ -72,8 +77,32 @@ public final class NewYamlShadowRuleConfigurationSwapper implements NewYamlRuleC
     
     @Override
     public ShadowRuleConfiguration swapToObject(final Collection<YamlDataNode> dataNodes) {
-        // TODO to be completed
-        return new ShadowRuleConfiguration();
+        Collection<ShadowDataSourceConfiguration> dataSources = new LinkedList<>();
+        Map<String, ShadowTableConfiguration> tables = new LinkedHashMap<>();
+        Map<String, AlgorithmConfiguration> algorithms = new LinkedHashMap<>();
+        ShadowRuleConfiguration result = new ShadowRuleConfiguration();
+        for (YamlDataNode each : dataNodes) {
+            if (ShadowNodeConverter.isDataSourcePath(each.getKey())) {
+                ShadowNodeConverter.getDataSourceName(each.getKey())
+                        .ifPresent(dataSourceName -> dataSources.add(swapDataSource(dataSourceName, YamlEngine.unmarshal(each.getValue(), YamlShadowDataSourceConfiguration.class))));
+            } else if (ShadowNodeConverter.isTablePath(each.getKey())) {
+                ShadowNodeConverter.getTableName(each.getKey())
+                        .ifPresent(tableName -> tables.put(tableName, tableSwapper.swapToObject(YamlEngine.unmarshal(each.getValue(), YamlShadowTableConfiguration.class))));
+            } else if (ShadowNodeConverter.isAlgorithmPath(each.getKey())) {
+                ShadowNodeConverter.getAlgorithmName(each.getKey())
+                        .ifPresent(algorithmName -> algorithms.put(algorithmName, algorithmSwapper.swapToObject(YamlEngine.unmarshal(each.getValue(), YamlAlgorithmConfiguration.class))));
+            } else if (ShadowNodeConverter.isDefaultAlgorithmNamePath(each.getKey())) {
+                result.setDefaultShadowAlgorithmName(each.getValue());
+            }
+        }
+        result.setDataSources(dataSources);
+        result.setTables(tables);
+        result.setShadowAlgorithms(algorithms);
+        return result;
+    }
+    
+    private ShadowDataSourceConfiguration swapDataSource(final String name, final YamlShadowDataSourceConfiguration yamlConfig) {
+        return new ShadowDataSourceConfiguration(name, yamlConfig.getProductionDataSourceName(), yamlConfig.getShadowDataSourceName());
     }
     
     @Override
diff --git a/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/metadata/converter/ShadowNodeConverterTest.java b/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/metadata/converter/ShadowNodeConverterTest.java
new file mode 100644
index 00000000000..2f8eadc0751
--- /dev/null
+++ b/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/metadata/converter/ShadowNodeConverterTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.metadata.converter;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Optional;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class ShadowNodeConverterTest {
+    
+    @Test
+    void assertGetDataSourcePath() {
+        assertThat(ShadowNodeConverter.getDataSourcePath("foo_db"), is("data_sources/foo_db"));
+    }
+    
+    @Test
+    void assertGetTableNamePath() {
+        assertThat(ShadowNodeConverter.getTableNamePath("foo_table"), is("tables/foo_table"));
+    }
+    
+    @Test
+    void assertGetAlgorithmPath() {
+        assertThat(ShadowNodeConverter.getShadowAlgorithmPath("SQL_HINT"), is("algorithms/SQL_HINT"));
+    }
+    
+    @Test
+    void assertGetDefaultShadowAlgorithmPath() {
+        assertThat(ShadowNodeConverter.getDefaultShadowAlgorithmPath(), is("default_algorithm_name"));
+    }
+    
+    @Test
+    void assertCheckIsTargetRuleByRulePath() {
+        assertTrue(ShadowNodeConverter.isShadowPath("/metadata/foo_db/rules/shadow/tables/foo_table"));
+        assertFalse(ShadowNodeConverter.isShadowPath("/metadata/foo_db/rules/foo/tables/foo_table"));
+        assertTrue(ShadowNodeConverter.isDataSourcePath("/metadata/foo_db/rules/shadow/data_sources/ds_shadow"));
+        assertFalse(ShadowNodeConverter.isDataSourcePath("/metadata/foo_db/rules/shadow/tables/foo_table"));
+        assertTrue(ShadowNodeConverter.isTablePath("/metadata/foo_db/rules/shadow/tables/foo_table"));
+        assertFalse(ShadowNodeConverter.isTablePath("/metadata/foo_db/rules/shadow/algorithms/MD5"));
+        assertTrue(ShadowNodeConverter.isAlgorithmPath("/metadata/foo_db/rules/shadow/algorithms/MD5"));
+        assertFalse(ShadowNodeConverter.isAlgorithmPath("/metadata/foo_db/rules/shadow/tables/foo_table"));
+        assertTrue(ShadowNodeConverter.isDefaultAlgorithmNamePath("/metadata/foo_db/rules/shadow/default_algorithm_name"));
+        assertFalse(ShadowNodeConverter.isDefaultAlgorithmNamePath("/metadata/foo_db/rules/shadow/default_algorithm_name/s"));
+    }
+    
+    @Test
+    void assertGetDataSourceNameByRulePath() {
+        Optional<String> actual = ShadowNodeConverter.getDataSourceName("/metadata/foo_db/rules/shadow/data_sources/foo_db");
+        assertTrue(actual.isPresent());
+        assertThat(actual.get(), is("foo_db"));
+    }
+    
+    @Test
+    void assertGetTableNameByRulePath() {
+        Optional<String> actual = ShadowNodeConverter.getTableName("/metadata/foo_db/rules/shadow/tables/foo_table");
+        assertTrue(actual.isPresent());
+        assertThat(actual.get(), is("foo_table"));
+    }
+    
+    @Test
+    void assertGetAlgorithmNameByRulePath() {
+        Optional<String> actual = ShadowNodeConverter.getAlgorithmName("/metadata/foo_db/rules/shadow/algorithms/SQL_HINT");
+        assertTrue(actual.isPresent());
+        assertThat(actual.get(), is("SQL_HINT"));
+    }
+}
diff --git a/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/yaml/swapper/NewYamlShadowRuleConfigurationSwapperTest.java b/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/yaml/swapper/NewYamlShadowRuleConfigurationSwapperTest.java
index c41e9798d48..cb4f3689658 100644
--- a/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/yaml/swapper/NewYamlShadowRuleConfigurationSwapperTest.java
+++ b/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/yaml/swapper/NewYamlShadowRuleConfigurationSwapperTest.java
@@ -54,8 +54,8 @@ class NewYamlShadowRuleConfigurationSwapperTest {
         Iterator<YamlDataNode> iterator = result.iterator();
         assertThat(iterator.next().getKey(), is("data_sources/foo"));
         assertThat(iterator.next().getKey(), is("tables/foo_table"));
-        assertThat(iterator.next().getKey(), is("shadow_algorithms/FIXTURE"));
-        assertThat(iterator.next().getKey(), is("default_shadow_algorithm_name"));
+        assertThat(iterator.next().getKey(), is("algorithms/FIXTURE"));
+        assertThat(iterator.next().getKey(), is("default_algorithm_name"));
     }
     
     private ShadowRuleConfiguration createMaximumShadowRule() {
@@ -72,4 +72,39 @@ class NewYamlShadowRuleConfigurationSwapperTest {
         result.setDefaultShadowAlgorithmName("FIXTURE");
         return result;
     }
+    
+    @Test
+    void assertSwapToObjectEmpty() {
+        Collection<YamlDataNode> config = new LinkedList<>();
+        ShadowRuleConfiguration result = swapper.swapToObject(config);
+        assertThat(result.getTables().size(), is(0));
+        assertThat(result.getShadowAlgorithms().size(), is(0));
+    }
+    
+    @Test
+    void assertSwapToObject() {
+        Collection<YamlDataNode> config = new LinkedList<>();
+        config.add(new YamlDataNode("/metadata/foo_db/rules/shadow/data_sources/foo_db", "productionDataSourceName: ds_0\n"
+                + "shadowDataSourceName: ds_1\n"));
+        config.add(new YamlDataNode("/metadata/foo_db/rules/shadow/tables/foo_table", "dataSourceNames:\n"
+                + "- ds_0\n"
+                + "shadowAlgorithmNames:\n"
+                + "- FIXTURE\n"));
+        config.add(new YamlDataNode("/metadata/foo_db/rules/shadow/algorithms/FIXTURE", "type: FIXTURE\n"));
+        config.add(new YamlDataNode("/metadata/foo_db/rules/shadow/default_algorithm_name", "FIXTURE"));
+        ShadowRuleConfiguration result = swapper.swapToObject(config);
+        assertThat(result.getDataSources().size(), is(1));
+        assertThat(result.getDataSources().iterator().next().getName(), is("foo_db"));
+        assertThat(result.getDataSources().iterator().next().getProductionDataSourceName(), is("ds_0"));
+        assertThat(result.getDataSources().iterator().next().getShadowDataSourceName(), is("ds_1"));
+        assertThat(result.getTables().size(), is(1));
+        assertThat(result.getTables().get("foo_table").getDataSourceNames().size(), is(1));
+        assertThat(result.getTables().get("foo_table").getDataSourceNames().iterator().next(), is("ds_0"));
+        assertThat(result.getTables().get("foo_table").getShadowAlgorithmNames().size(), is(1));
+        assertThat(result.getTables().get("foo_table").getShadowAlgorithmNames().iterator().next(), is("FIXTURE"));
+        assertThat(result.getShadowAlgorithms().size(), is(1));
+        assertThat(result.getShadowAlgorithms().get("FIXTURE").getType(), is("FIXTURE"));
+        assertThat(result.getShadowAlgorithms().get("FIXTURE").getProps().size(), is(0));
+        assertThat(result.getDefaultShadowAlgorithmName(), is("FIXTURE"));
+    }
 }