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/09 03:54:23 UTC

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

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 2824b28b781 Improve swap yaml data to rule configuration in encrypt (#26208)
2824b28b781 is described below

commit 2824b28b78107e57ded1a0aaf9fcb3ec591c711f
Author: ChenJiaHao <Pa...@163.com>
AuthorDate: Fri Jun 9 11:54:16 2023 +0800

    Improve swap yaml data to rule configuration in encrypt (#26208)
---
 .../metadata/converter/EncryptNodeConverter.java   | 82 ++++++++++++++++++++--
 ...lCompatibleEncryptRuleConfigurationSwapper.java | 20 +++++-
 .../NewYamlEncryptRuleConfigurationSwapper.java    | 22 ++++--
 .../converter/EncryptNodeConverterTest.java        | 64 +++++++++++++++++
 ...patibleEncryptRuleConfigurationSwapperTest.java | 32 ++++++++-
 ...NewYamlEncryptRuleConfigurationSwapperTest.java | 33 ++++++++-
 6 files changed, 238 insertions(+), 15 deletions(-)

diff --git a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/metadata/converter/EncryptNodeConverter.java b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/metadata/converter/EncryptNodeConverter.java
index 17b19eb4357..84678344f98 100644
--- a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/metadata/converter/EncryptNodeConverter.java
+++ b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/metadata/converter/EncryptNodeConverter.java
@@ -20,33 +20,103 @@ package org.apache.shardingsphere.encrypt.metadata.converter;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 
+import java.util.Optional;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 /**
  * Encrypt node converter.
  */
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
 public final class EncryptNodeConverter {
     
-    private static final String TABLES = "tables";
+    private static final String ROOT_NODE = "encrypt";
+    
+    private static final String TABLES_NODE = "tables";
+    
+    private static final String ENCRYPTORS_NODE = "encryptors";
     
-    private static final String ENCRYPTORS = "encryptors";
+    private static final String RULES_NODE_PREFIX = "/([\\w\\-]+)/([\\w\\-]+)/rules/";
+    
+    private static final String RULE_NAME_PATTERN = "/([\\w\\-]+)?";
     
     /**
      * 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 encryptor path.
-     * 
+     *
      * @param encryptorName encryptor name
      * @return encryptor path
      */
     public static String getEncryptorPath(final String encryptorName) {
-        return String.join("/", ENCRYPTORS, encryptorName);
+        return String.join("/", ENCRYPTORS_NODE, encryptorName);
+    }
+    
+    /**
+     * Is encrypt path.
+     *
+     * @param rulePath rule path
+     * @return true or false
+     */
+    public static boolean isEncryptPath(final String rulePath) {
+        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "\\.*", Pattern.CASE_INSENSITIVE);
+        Matcher matcher = pattern.matcher(rulePath);
+        return matcher.find();
+    }
+    
+    /**
+     * Is encrypt 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 encryptor path.
+     *
+     * @param rulePath rule path
+     * @return true or false
+     */
+    public static boolean isEncryptorPath(final String rulePath) {
+        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "/" + ENCRYPTORS_NODE + "\\.*", Pattern.CASE_INSENSITIVE);
+        Matcher matcher = pattern.matcher(rulePath);
+        return matcher.find();
+    }
+    
+    /**
+     * 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 + RULE_NAME_PATTERN, Pattern.CASE_INSENSITIVE);
+        Matcher matcher = pattern.matcher(rulePath);
+        return matcher.find() ? Optional.of(matcher.group(3)) : Optional.empty();
+    }
+    
+    /**
+     *  Get encryptor name.
+     *
+     * @param rulePath rule path
+     * @return encryptor name
+     */
+    public static Optional<String> getEncryptorName(final String rulePath) {
+        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "/" + ENCRYPTORS_NODE + RULE_NAME_PATTERN, Pattern.CASE_INSENSITIVE);
+        Matcher matcher = pattern.matcher(rulePath);
+        return matcher.find() ? Optional.of(matcher.group(3)) : Optional.empty();
     }
 }
diff --git a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlCompatibleEncryptRuleConfigurationSwapper.java b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlCompatibleEncryptRuleConfigurationSwapper.java
index aac2ab3dea1..9dd82f32aa8 100644
--- a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlCompatibleEncryptRuleConfigurationSwapper.java
+++ b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlCompatibleEncryptRuleConfigurationSwapper.java
@@ -21,16 +21,20 @@ import org.apache.shardingsphere.encrypt.api.config.CompatibleEncryptRuleConfigu
 import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfiguration;
 import org.apache.shardingsphere.encrypt.constant.EncryptOrder;
 import org.apache.shardingsphere.encrypt.metadata.converter.EncryptNodeConverter;
+import org.apache.shardingsphere.encrypt.yaml.config.rule.YamlCompatibleEncryptTableRuleConfiguration;
 import org.apache.shardingsphere.encrypt.yaml.swapper.rule.YamlCompatibleEncryptTableRuleConfigurationSwapper;
 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 java.util.Collection;
-import java.util.Collections;
+import java.util.HashMap;
 import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.Map;
 import java.util.Map.Entry;
 
 /**
@@ -60,8 +64,18 @@ public final class NewYamlCompatibleEncryptRuleConfigurationSwapper implements N
     
     @Override
     public CompatibleEncryptRuleConfiguration swapToObject(final Collection<YamlDataNode> dataNodes) {
-        // TODO to be completed
-        return new CompatibleEncryptRuleConfiguration(Collections.emptyList(), Collections.emptyMap());
+        Collection<EncryptTableRuleConfiguration> tables = new LinkedList<>();
+        Map<String, AlgorithmConfiguration> encryptors = new HashMap<>();
+        for (YamlDataNode each : dataNodes) {
+            if (EncryptNodeConverter.isTablePath(each.getKey())) {
+                EncryptNodeConverter.getTableName(each.getKey())
+                        .ifPresent(tableName -> tables.add(tableSwapper.swapToObject(YamlEngine.unmarshal(each.getValue(), YamlCompatibleEncryptTableRuleConfiguration.class))));
+            } else if (EncryptNodeConverter.isEncryptorPath(each.getKey())) {
+                EncryptNodeConverter.getEncryptorName(each.getKey())
+                        .ifPresent(encryptorName -> encryptors.put(encryptorName, algorithmSwapper.swapToObject(YamlEngine.unmarshal(each.getValue(), YamlAlgorithmConfiguration.class))));
+            }
+        }
+        return new CompatibleEncryptRuleConfiguration(tables, encryptors);
     }
     
     @Override
diff --git a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlEncryptRuleConfigurationSwapper.java b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlEncryptRuleConfigurationSwapper.java
index e544f0283ad..eb2e2ecccee 100644
--- a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlEncryptRuleConfigurationSwapper.java
+++ b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlEncryptRuleConfigurationSwapper.java
@@ -20,17 +20,21 @@ package org.apache.shardingsphere.encrypt.yaml.swapper;
 import org.apache.shardingsphere.encrypt.api.config.EncryptRuleConfiguration;
 import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfiguration;
 import org.apache.shardingsphere.encrypt.constant.EncryptOrder;
+import org.apache.shardingsphere.encrypt.metadata.converter.EncryptNodeConverter;
+import org.apache.shardingsphere.encrypt.yaml.config.rule.YamlEncryptTableRuleConfiguration;
 import org.apache.shardingsphere.encrypt.yaml.swapper.rule.YamlEncryptTableRuleConfigurationSwapper;
 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.encrypt.metadata.converter.EncryptNodeConverter;
 
 import java.util.Collection;
-import java.util.Collections;
+import java.util.HashMap;
 import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.Map;
 import java.util.Map.Entry;
 
 /**
@@ -57,8 +61,18 @@ public final class NewYamlEncryptRuleConfigurationSwapper implements NewYamlRule
     
     @Override
     public EncryptRuleConfiguration swapToObject(final Collection<YamlDataNode> dataNodes) {
-        // TODO to be completed
-        return new EncryptRuleConfiguration(Collections.emptyList(), Collections.emptyMap());
+        Collection<EncryptTableRuleConfiguration> tables = new LinkedList<>();
+        Map<String, AlgorithmConfiguration> encryptors = new HashMap<>();
+        for (YamlDataNode each : dataNodes) {
+            if (EncryptNodeConverter.isTablePath(each.getKey())) {
+                EncryptNodeConverter.getTableName(each.getKey())
+                        .ifPresent(tableName -> tables.add(tableSwapper.swapToObject(YamlEngine.unmarshal(each.getValue(), YamlEncryptTableRuleConfiguration.class))));
+            } else if (EncryptNodeConverter.isEncryptorPath(each.getKey())) {
+                EncryptNodeConverter.getEncryptorName(each.getKey())
+                        .ifPresent(encryptorName -> encryptors.put(encryptorName, algorithmSwapper.swapToObject(YamlEngine.unmarshal(each.getValue(), YamlAlgorithmConfiguration.class))));
+            }
+        }
+        return new EncryptRuleConfiguration(tables, encryptors);
     }
     
     @Override
diff --git a/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/metadata/converter/EncryptNodeConverterTest.java b/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/metadata/converter/EncryptNodeConverterTest.java
new file mode 100644
index 00000000000..13e3fce0cdb
--- /dev/null
+++ b/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/metadata/converter/EncryptNodeConverterTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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.encrypt.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 EncryptNodeConverterTest {
+    
+    @Test
+    void assertGetTableNamePath() {
+        assertThat(EncryptNodeConverter.getTableNamePath("foo_table"), is("tables/foo_table"));
+    }
+    
+    @Test
+    void assertGetAlgorithmPath() {
+        assertThat(EncryptNodeConverter.getEncryptorPath("AES"), is("encryptors/AES"));
+    }
+    
+    @Test
+    void assertCheckIsTargetRuleByRulePath() {
+        assertTrue(EncryptNodeConverter.isEncryptPath("/metadata/foo_db/rules/encrypt/tables/foo_table"));
+        assertFalse(EncryptNodeConverter.isEncryptPath("/metadata/foo_db/rules/foo/tables/foo_table"));
+        assertTrue(EncryptNodeConverter.isTablePath("/metadata/foo_db/rules/encrypt/tables/foo_table"));
+        assertFalse(EncryptNodeConverter.isTablePath("/metadata/foo_db/rules/encrypt/encryptors/AES"));
+        assertTrue(EncryptNodeConverter.isEncryptorPath("/metadata/foo_db/rules/encrypt/encryptors/AES"));
+        assertFalse(EncryptNodeConverter.isEncryptorPath("/metadata/foo_db/rules/encrypt/tables/foo_table"));
+    }
+    
+    @Test
+    void assertGetTableNameByRulePath() {
+        Optional<String> actual = EncryptNodeConverter.getTableName("/metadata/foo_db/rules/encrypt/tables/foo_table");
+        assertTrue(actual.isPresent());
+        assertThat(actual.get(), is("foo_table"));
+    }
+    
+    @Test
+    void assertGetAlgorithmNameByRulePath() {
+        Optional<String> actual = EncryptNodeConverter.getEncryptorName("/metadata/foo_db/rules/encrypt/encryptors/AES");
+        assertTrue(actual.isPresent());
+        assertThat(actual.get(), is("AES"));
+    }
+}
diff --git a/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlCompatibleEncryptRuleConfigurationSwapperTest.java b/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlCompatibleEncryptRuleConfigurationSwapperTest.java
index 69b4c805ffb..5666d6802d0 100644
--- a/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlCompatibleEncryptRuleConfigurationSwapperTest.java
+++ b/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlCompatibleEncryptRuleConfigurationSwapperTest.java
@@ -58,7 +58,37 @@ class NewYamlCompatibleEncryptRuleConfigurationSwapperTest {
     
     private CompatibleEncryptRuleConfiguration createMaximumEncryptRule() {
         Collection<EncryptTableRuleConfiguration> tables = new LinkedList<>();
-        tables.add(new EncryptTableRuleConfiguration("foo", Collections.singleton(new EncryptColumnRuleConfiguration("foo_column", new EncryptColumnItemRuleConfiguration("FIXTURE")))));
+        tables.add(new EncryptTableRuleConfiguration("foo", Collections.singleton(new EncryptColumnRuleConfiguration("foo_column", new EncryptColumnItemRuleConfiguration("FIXTURE", "FOO")))));
         return new CompatibleEncryptRuleConfiguration(tables, Collections.singletonMap("FOO", new AlgorithmConfiguration("FOO", new Properties())));
     }
+    
+    @Test
+    void assertSwapToObjectEmpty() {
+        Collection<YamlDataNode> config = new LinkedList<>();
+        CompatibleEncryptRuleConfiguration result = swapper.swapToObject(config);
+        assertThat(result.getTables().size(), is(0));
+        assertThat(result.getEncryptors().size(), is(0));
+    }
+    
+    @Test
+    void assertSwapToObject() {
+        Collection<YamlDataNode> config = new LinkedList<>();
+        config.add(new YamlDataNode("/metadata/foo_db/rules/encrypt/tables/foo", "columns:\n"
+                + "  foo_column:\n"
+                + "    cipherColumn: FIXTURE\n"
+                + "    encryptorName: FOO\n"
+                + "    logicColumn: foo_column\n"
+                + "name: foo\n"));
+        config.add(new YamlDataNode("/metadata/foo_db/rules/encrypt/encryptors/FOO", "type: FOO\n"));
+        CompatibleEncryptRuleConfiguration result = swapper.swapToObject(config);
+        assertThat(result.getTables().size(), is(1));
+        assertThat(result.getTables().iterator().next().getName(), is("foo"));
+        assertThat(result.getTables().iterator().next().getColumns().size(), is(1));
+        assertThat(result.getTables().iterator().next().getColumns().iterator().next().getName(), is("foo_column"));
+        assertThat(result.getTables().iterator().next().getColumns().iterator().next().getCipher().getName(), is("FIXTURE"));
+        assertThat(result.getTables().iterator().next().getColumns().iterator().next().getCipher().getEncryptorName(), is("FOO"));
+        assertThat(result.getEncryptors().size(), is(1));
+        assertThat(result.getEncryptors().get("FOO").getType(), is("FOO"));
+        assertThat(result.getEncryptors().get("FOO").getProps().size(), is(0));
+    }
 }
diff --git a/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlEncryptRuleConfigurationSwapperTest.java b/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlEncryptRuleConfigurationSwapperTest.java
index 42775eff7e0..3b3dd6350d1 100644
--- a/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlEncryptRuleConfigurationSwapperTest.java
+++ b/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/yaml/swapper/NewYamlEncryptRuleConfigurationSwapperTest.java
@@ -57,7 +57,38 @@ class NewYamlEncryptRuleConfigurationSwapperTest {
     
     private EncryptRuleConfiguration createMaximumEncryptRule() {
         Collection<EncryptTableRuleConfiguration> tables = new LinkedList<>();
-        tables.add(new EncryptTableRuleConfiguration("foo", Collections.singleton(new EncryptColumnRuleConfiguration("foo_column", new EncryptColumnItemRuleConfiguration("FIXTURE")))));
+        tables.add(new EncryptTableRuleConfiguration("foo", Collections.singleton(new EncryptColumnRuleConfiguration("foo_column", new EncryptColumnItemRuleConfiguration("FIXTURE", "FOO")))));
         return new EncryptRuleConfiguration(tables, Collections.singletonMap("FOO", new AlgorithmConfiguration("FOO", new Properties())));
     }
+    
+    @Test
+    void assertSwapToObjectEmpty() {
+        Collection<YamlDataNode> config = new LinkedList<>();
+        EncryptRuleConfiguration result = swapper.swapToObject(config);
+        assertThat(result.getTables().size(), is(0));
+        assertThat(result.getEncryptors().size(), is(0));
+    }
+    
+    @Test
+    void assertSwapToObject() {
+        Collection<YamlDataNode> config = new LinkedList<>();
+        config.add(new YamlDataNode("/metadata/foo_db/rules/encrypt/tables/foo", "columns:\n"
+                + "  foo_column:\n"
+                + "    cipher:\n"
+                + "      encryptorName: FOO\n"
+                + "      name: FIXTURE\n"
+                + "    name: foo_column\n"
+                + "name: foo\n"));
+        config.add(new YamlDataNode("/metadata/foo_db/rules/encrypt/encryptors/FOO", "type: FOO\n"));
+        EncryptRuleConfiguration result = swapper.swapToObject(config);
+        assertThat(result.getTables().size(), is(1));
+        assertThat(result.getTables().iterator().next().getName(), is("foo"));
+        assertThat(result.getTables().iterator().next().getColumns().size(), is(1));
+        assertThat(result.getTables().iterator().next().getColumns().iterator().next().getName(), is("foo_column"));
+        assertThat(result.getTables().iterator().next().getColumns().iterator().next().getCipher().getName(), is("FIXTURE"));
+        assertThat(result.getTables().iterator().next().getColumns().iterator().next().getCipher().getEncryptorName(), is("FOO"));
+        assertThat(result.getEncryptors().size(), is(1));
+        assertThat(result.getEncryptors().get("FOO").getType(), is("FOO"));
+        assertThat(result.getEncryptors().get("FOO").getProps().size(), is(0));
+    }
 }