You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2022/12/26 01:22:11 UTC

[shardingsphere] branch master updated: Implement ResultDecoratorEngine for MaskRule and add mask processing logic (#23088)

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

duanzhengqiang 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 1c26a66b7ed Implement ResultDecoratorEngine for MaskRule and add mask processing logic (#23088)
1c26a66b7ed is described below

commit 1c26a66b7ed195da5b3dcf02e2aeedcd5c7978f6
Author: gxxiong <xi...@foxmail.com>
AuthorDate: Mon Dec 26 09:21:58 2022 +0800

    Implement ResultDecoratorEngine for MaskRule and add mask processing logic (#23088)
    
    * Implement ResultDecoratorEngine for MaskRule and add mask processing logic
    
    * shardingsphere-jdbc-core pom add shardingsphere-mask-core
    
    * remove MaskContext
---
 .../mask/merge/MaskResultDecoratorEngine.java      | 56 ++++++++++++
 .../mask/merge/dql/MaskAlgorithmMetaData.java      | 99 ++++++++++++++++++++++
 .../mask/merge/dql/MaskDQLResultDecorator.java     | 45 ++++++++++
 .../mask/merge/dql/MaskMergedResult.java           | 69 +++++++++++++++
 .../apache/shardingsphere/mask/rule/MaskRule.java  | 13 +++
 .../apache/shardingsphere/mask/rule/MaskTable.java | 11 +++
 .../mask/rule/builder/MaskRuleBuilder.java         | 51 +++++++++++
 ...gsphere.infra.merge.engine.ResultProcessEngine} |  5 +-
 ...nfra.rule.builder.database.DatabaseRuleBuilder} |  5 +-
 ...rg.apache.shardingsphere.mask.spi.MaskAlgorithm |  7 +-
 .../mask/merge/MaskResultDecoratorEngineTest.java  | 63 ++++++++++++++
 .../mask/merge/dql/MaskAlgorithmMetaDataTest.java  | 75 ++++++++++++++++
 .../mask/merge/dql/MaskDQLResultDecoratorTest.java | 51 +++++++++++
 .../mask/merge/dql/MaskMergedResultTest.java       | 80 +++++++++++++++++
 .../mask/rule/builder/MaskRuleBuilderTest.java     | 42 +++++++++
 jdbc/core/pom.xml                                  |  5 ++
 16 files changed, 670 insertions(+), 7 deletions(-)

diff --git a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/merge/MaskResultDecoratorEngine.java b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/merge/MaskResultDecoratorEngine.java
new file mode 100644
index 00000000000..753c10a4377
--- /dev/null
+++ b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/merge/MaskResultDecoratorEngine.java
@@ -0,0 +1,56 @@
+/*
+ * 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.mask.merge;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import org.apache.shardingsphere.infra.merge.engine.decorator.ResultDecorator;
+import org.apache.shardingsphere.infra.merge.engine.decorator.ResultDecoratorEngine;
+import org.apache.shardingsphere.infra.merge.engine.decorator.impl.TransparentResultDecorator;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.mask.constant.MaskOrder;
+import org.apache.shardingsphere.mask.merge.dql.MaskAlgorithmMetaData;
+import org.apache.shardingsphere.mask.merge.dql.MaskDQLResultDecorator;
+import org.apache.shardingsphere.mask.rule.MaskRule;
+
+/**
+ * Result decorator engine for mask.
+ */
+public final class MaskResultDecoratorEngine implements ResultDecoratorEngine<MaskRule> {
+    
+    @Override
+    public ResultDecorator<?> newInstance(final ShardingSphereDatabase database,
+                                          final MaskRule maskRule, final ConfigurationProperties props, final SQLStatementContext<?> sqlStatementContext) {
+        if (sqlStatementContext instanceof SelectStatementContext) {
+            MaskAlgorithmMetaData algorithmMetaData = new MaskAlgorithmMetaData(database, maskRule, (SelectStatementContext) sqlStatementContext);
+            return new MaskDQLResultDecorator(algorithmMetaData);
+        }
+        return new TransparentResultDecorator();
+    }
+    
+    @Override
+    public int getOrder() {
+        return MaskOrder.ORDER;
+    }
+    
+    @Override
+    public Class<MaskRule> getTypeClass() {
+        return MaskRule.class;
+    }
+}
diff --git a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/merge/dql/MaskAlgorithmMetaData.java b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/merge/dql/MaskAlgorithmMetaData.java
new file mode 100644
index 00000000000..4b1ec05b752
--- /dev/null
+++ b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/merge/dql/MaskAlgorithmMetaData.java
@@ -0,0 +1,99 @@
+/*
+ * 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.mask.merge.dql;
+
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.binder.segment.select.projection.Projection;
+import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.ColumnProjection;
+import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.mask.rule.MaskRule;
+import org.apache.shardingsphere.mask.spi.MaskAlgorithm;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Mask algorithm meta data.
+ */
+@SuppressWarnings("rawtypes")
+@RequiredArgsConstructor
+public final class MaskAlgorithmMetaData {
+    
+    private final ShardingSphereDatabase database;
+    
+    private final MaskRule maskRule;
+    
+    private final SelectStatementContext selectStatementContext;
+    
+    /**
+     * Find mask algorithm.
+     *
+     * @param tableName table name
+     * @param columnName column name
+     * @return maskAlgorithm
+     */
+    public Optional<MaskAlgorithm> findMaskAlgorithm(final String tableName, final String columnName) {
+        return maskRule.findMaskAlgorithm(tableName, columnName);
+    }
+    
+    /**
+     * Find mask algorithm.
+     *
+     * @param columnIndex column index
+     * @return maskAlgorithm
+     */
+    public Optional<MaskAlgorithm> findMaskAlgorithmByColumnIndex(final int columnIndex) {
+        Optional<ColumnProjection> columnProjection = findColumnProjection(columnIndex);
+        if (!columnProjection.isPresent()) {
+            return Optional.empty();
+        }
+        TablesContext tablesContext = selectStatementContext.getTablesContext();
+        String schemaName = tablesContext.getSchemaName().orElseGet(() -> DatabaseTypeEngine.getDefaultSchemaName(selectStatementContext.getDatabaseType(), database.getName()));
+        Map<String, String> expressionTableNames = tablesContext.findTableNamesByColumnProjection(
+                Collections.singletonList(columnProjection.get()), database.getSchema(schemaName));
+        Optional<String> tableName = findTableName(columnProjection.get(), expressionTableNames);
+        return maskRule.findMaskAlgorithm(tableName.get(), columnProjection.get().getName());
+    }
+    
+    private Optional<ColumnProjection> findColumnProjection(final int columnIndex) {
+        List<Projection> expandProjections = selectStatementContext.getProjectionsContext().getExpandProjections();
+        if (expandProjections.size() < columnIndex) {
+            return Optional.empty();
+        }
+        Projection projection = expandProjections.get(columnIndex - 1);
+        return projection instanceof ColumnProjection ? Optional.of((ColumnProjection) projection) : Optional.empty();
+    }
+    
+    private Optional<String> findTableName(final ColumnProjection columnProjection, final Map<String, String> columnTableNames) {
+        String tableName = columnTableNames.get(columnProjection.getExpression());
+        if (null != tableName) {
+            return Optional.of(tableName);
+        }
+        for (String each : selectStatementContext.getTablesContext().getTableNames()) {
+            if (maskRule.findMaskAlgorithm(each, columnProjection.getName()).isPresent()) {
+                return Optional.of(each);
+            }
+        }
+        return Optional.empty();
+    }
+}
diff --git a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/merge/dql/MaskDQLResultDecorator.java b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/merge/dql/MaskDQLResultDecorator.java
new file mode 100644
index 00000000000..00b9ce45cbc
--- /dev/null
+++ b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/merge/dql/MaskDQLResultDecorator.java
@@ -0,0 +1,45 @@
+/*
+ * 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.mask.merge.dql;
+
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
+import org.apache.shardingsphere.infra.merge.engine.decorator.ResultDecorator;
+import org.apache.shardingsphere.infra.merge.result.MergedResult;
+import org.apache.shardingsphere.infra.merge.result.impl.transparent.TransparentMergedResult;
+import org.apache.shardingsphere.mask.rule.MaskRule;
+
+/**
+ * DQL result decorator for mask.
+ */
+@RequiredArgsConstructor
+public final class MaskDQLResultDecorator implements ResultDecorator<MaskRule> {
+    
+    private final MaskAlgorithmMetaData metaData;
+    
+    @Override
+    public MergedResult decorate(final QueryResult queryResult, final SQLStatementContext<?> sqlStatementContext, final MaskRule rule) {
+        return new MaskMergedResult(metaData, new TransparentMergedResult(queryResult));
+    }
+    
+    @Override
+    public MergedResult decorate(final MergedResult mergedResult, final SQLStatementContext<?> sqlStatementContext, final MaskRule rule) {
+        return new MaskMergedResult(metaData, mergedResult);
+    }
+}
diff --git a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/merge/dql/MaskMergedResult.java b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/merge/dql/MaskMergedResult.java
new file mode 100644
index 00000000000..04b4ca8a9cf
--- /dev/null
+++ b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/merge/dql/MaskMergedResult.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mask.merge.dql;
+
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.merge.result.MergedResult;
+import org.apache.shardingsphere.mask.spi.MaskAlgorithm;
+
+import java.io.InputStream;
+import java.sql.SQLException;
+import java.util.Calendar;
+import java.util.Optional;
+
+/**
+ * Merged result for mask.
+ */
+@RequiredArgsConstructor
+public final class MaskMergedResult implements MergedResult {
+    
+    private final MaskAlgorithmMetaData metaData;
+    
+    private final MergedResult mergedResult;
+    
+    @Override
+    public boolean next() throws SQLException {
+        return mergedResult.next();
+    }
+    
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    @Override
+    public Object getValue(final int columnIndex, final Class<?> type) throws SQLException {
+        Optional<MaskAlgorithm> maskAlgorithm = metaData.findMaskAlgorithmByColumnIndex(columnIndex);
+        if (!maskAlgorithm.isPresent()) {
+            return mergedResult.getValue(columnIndex, type);
+        }
+        Object originalValue = mergedResult.getValue(columnIndex, Object.class);
+        return null == originalValue ? null : maskAlgorithm.get().mask(originalValue);
+    }
+    
+    @Override
+    public Object getCalendarValue(final int columnIndex, final Class<?> type, final Calendar calendar) throws SQLException {
+        return mergedResult.getCalendarValue(columnIndex, type, calendar);
+    }
+    
+    @Override
+    public InputStream getInputStream(final int columnIndex, final String type) throws SQLException {
+        return mergedResult.getInputStream(columnIndex, type);
+    }
+    
+    @Override
+    public boolean wasNull() throws SQLException {
+        return mergedResult.wasNull();
+    }
+}
diff --git a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/MaskRule.java b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/MaskRule.java
index 4bb1c266e47..579c59f802a 100644
--- a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/MaskRule.java
+++ b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/MaskRule.java
@@ -28,6 +28,7 @@ import org.apache.shardingsphere.mask.spi.MaskAlgorithm;
 import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.Map;
+import java.util.Optional;
 
 /**
  * Mask rule.
@@ -48,6 +49,18 @@ public final class MaskRule implements DatabaseRule, TableContainedRule {
         ruleConfig.getTables().forEach(each -> tables.put(each.getName().toLowerCase(), new MaskTable(each)));
     }
     
+    /**
+     * Find mask algorithm.
+     *
+     * @param logicTable logic table name
+     * @param logicColumn logic column name
+     * @return maskAlgorithm
+     */
+    public Optional<MaskAlgorithm> findMaskAlgorithm(final String logicTable, final String logicColumn) {
+        String lowerCaseLogicTable = logicTable.toLowerCase();
+        return tables.containsKey(lowerCaseLogicTable) ? tables.get(lowerCaseLogicTable).findMaskAlgorithmName(logicColumn).map(maskAlgorithms::get) : Optional.empty();
+    }
+    
     @Override
     public Collection<String> getTables() {
         return tables.keySet();
diff --git a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/MaskTable.java b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/MaskTable.java
index 01237219ff6..1e3f24ab927 100644
--- a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/MaskTable.java
+++ b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/MaskTable.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.mask.api.config.rule.MaskColumnRuleConfiguratio
 import org.apache.shardingsphere.mask.api.config.rule.MaskTableRuleConfiguration;
 
 import java.util.Map;
+import java.util.Optional;
 import java.util.TreeMap;
 
 /**
@@ -36,4 +37,14 @@ public final class MaskTable {
             columns.put(each.getLogicColumn(), new MaskColumn(each.getLogicColumn(), each.getMaskAlgorithm()));
         }
     }
+    
+    /**
+     * Find mask algorithm name.
+     *
+     * @param logicColumn column name
+     * @return mask algorithm name
+     */
+    public Optional<String> findMaskAlgorithmName(final String logicColumn) {
+        return columns.containsKey(logicColumn) ? Optional.of(columns.get(logicColumn).getMaskAlgorithm()) : Optional.empty();
+    }
 }
diff --git a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/builder/MaskRuleBuilder.java b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/builder/MaskRuleBuilder.java
new file mode 100644
index 00000000000..afa30544624
--- /dev/null
+++ b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/builder/MaskRuleBuilder.java
@@ -0,0 +1,51 @@
+/*
+ * 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.mask.rule.builder;
+
+import org.apache.shardingsphere.infra.instance.InstanceContext;
+import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.rule.builder.database.DatabaseRuleBuilder;
+import org.apache.shardingsphere.mask.api.config.MaskRuleConfiguration;
+import org.apache.shardingsphere.mask.constant.MaskOrder;
+import org.apache.shardingsphere.mask.rule.MaskRule;
+
+import javax.sql.DataSource;
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ * Mask rule builder.
+ */
+public final class MaskRuleBuilder implements DatabaseRuleBuilder<MaskRuleConfiguration> {
+    
+    @Override
+    public MaskRule build(final MaskRuleConfiguration config, final String databaseName,
+                          final Map<String, DataSource> dataSources, final Collection<ShardingSphereRule> builtRules, final InstanceContext instanceContext) {
+        return new MaskRule(config);
+    }
+    
+    @Override
+    public int getOrder() {
+        return MaskOrder.ORDER;
+    }
+    
+    @Override
+    public Class<MaskRuleConfiguration> getTypeClass() {
+        return MaskRuleConfiguration.class;
+    }
+}
diff --git a/features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.mask.spi.MaskAlgorithm b/features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.merge.engine.ResultProcessEngine
similarity index 78%
copy from features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.mask.spi.MaskAlgorithm
copy to features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.merge.engine.ResultProcessEngine
index 826c5f76174..b1550d11679 100644
--- a/features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.mask.spi.MaskAlgorithm
+++ b/features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.merge.engine.ResultProcessEngine
@@ -14,6 +14,5 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-org.apache.shardingsphere.mask.algorithm.cover.KeepFirstNLastMMaskAlgorithm
-org.apache.shardingsphere.mask.algorithm.cover.KeepFromXToYMaskAlgorithm
-org.apache.shardingsphere.mask.algorithm.hash.MD5MaskAlgorithm
+
+org.apache.shardingsphere.mask.merge.MaskResultDecoratorEngine
diff --git a/features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.mask.spi.MaskAlgorithm b/features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.database.DatabaseRuleBuilder
similarity index 78%
copy from features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.mask.spi.MaskAlgorithm
copy to features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.database.DatabaseRuleBuilder
index 826c5f76174..3a3bb72649f 100644
--- a/features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.mask.spi.MaskAlgorithm
+++ b/features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.builder.database.DatabaseRuleBuilder
@@ -14,6 +14,5 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-org.apache.shardingsphere.mask.algorithm.cover.KeepFirstNLastMMaskAlgorithm
-org.apache.shardingsphere.mask.algorithm.cover.KeepFromXToYMaskAlgorithm
-org.apache.shardingsphere.mask.algorithm.hash.MD5MaskAlgorithm
+
+org.apache.shardingsphere.mask.rule.builder.MaskRuleBuilder
diff --git a/features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.mask.spi.MaskAlgorithm b/features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.mask.spi.MaskAlgorithm
index 826c5f76174..c54ea5caaed 100644
--- a/features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.mask.spi.MaskAlgorithm
+++ b/features/mask/core/src/main/resources/META-INF/services/org.apache.shardingsphere.mask.spi.MaskAlgorithm
@@ -14,6 +14,11 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
+
+org.apache.shardingsphere.mask.algorithm.hash.MD5MaskAlgorithm
 org.apache.shardingsphere.mask.algorithm.cover.KeepFirstNLastMMaskAlgorithm
 org.apache.shardingsphere.mask.algorithm.cover.KeepFromXToYMaskAlgorithm
-org.apache.shardingsphere.mask.algorithm.hash.MD5MaskAlgorithm
+org.apache.shardingsphere.mask.algorithm.cover.MaskAfterSpecialCharAlgorithm
+org.apache.shardingsphere.mask.algorithm.cover.MaskBeforeSpecialCharAlgorithm
+org.apache.shardingsphere.mask.algorithm.cover.MaskFirstNLastMMaskAlgorithm
+org.apache.shardingsphere.mask.algorithm.cover.MaskFromXToYMaskAlgorithm
diff --git a/features/mask/core/src/test/java/org/apache/shardingsphere/mask/merge/MaskResultDecoratorEngineTest.java b/features/mask/core/src/test/java/org/apache/shardingsphere/mask/merge/MaskResultDecoratorEngineTest.java
new file mode 100644
index 00000000000..b3c4b1e3ae4
--- /dev/null
+++ b/features/mask/core/src/test/java/org/apache/shardingsphere/mask/merge/MaskResultDecoratorEngineTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.mask.merge;
+
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import org.apache.shardingsphere.infra.merge.engine.ResultProcessEngineFactory;
+import org.apache.shardingsphere.infra.merge.engine.decorator.ResultDecorator;
+import org.apache.shardingsphere.infra.merge.engine.decorator.impl.TransparentResultDecorator;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.mask.merge.dql.MaskDQLResultDecorator;
+import org.apache.shardingsphere.mask.rule.MaskRule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Collections;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.mock;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class MaskResultDecoratorEngineTest {
+    
+    @Mock
+    private MaskRule rule;
+    
+    @Mock
+    private ShardingSphereDatabase database;
+    
+    @Test
+    public void assertNewInstanceWithSelectStatement() {
+        MaskResultDecoratorEngine engine = (MaskResultDecoratorEngine) ResultProcessEngineFactory.getInstances(Collections.singleton(rule)).get(rule);
+        ResultDecorator<?> actual = engine.newInstance(database, rule, mock(ConfigurationProperties.class), mock(SelectStatementContext.class, RETURNS_DEEP_STUBS));
+        assertThat(actual, instanceOf(MaskDQLResultDecorator.class));
+    }
+    
+    @Test
+    public void assertNewInstanceWithOtherStatement() {
+        MaskResultDecoratorEngine engine = (MaskResultDecoratorEngine) ResultProcessEngineFactory.getInstances(Collections.singleton(rule)).get(rule);
+        ResultDecorator<?> actual = engine.newInstance(database, rule, mock(ConfigurationProperties.class), mock(InsertStatementContext.class));
+        assertThat(actual, instanceOf(TransparentResultDecorator.class));
+    }
+}
diff --git a/features/mask/core/src/test/java/org/apache/shardingsphere/mask/merge/dql/MaskAlgorithmMetaDataTest.java b/features/mask/core/src/test/java/org/apache/shardingsphere/mask/merge/dql/MaskAlgorithmMetaDataTest.java
new file mode 100644
index 00000000000..98396ca26b1
--- /dev/null
+++ b/features/mask/core/src/test/java/org/apache/shardingsphere/mask/merge/dql/MaskAlgorithmMetaDataTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.mask.merge.dql;
+
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
+import org.apache.shardingsphere.infra.database.DefaultDatabase;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereSchema;
+import org.apache.shardingsphere.mask.factory.MaskAlgorithmFactory;
+import org.apache.shardingsphere.mask.rule.MaskRule;
+import org.apache.shardingsphere.mask.spi.MaskAlgorithm;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Answers;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Optional;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class MaskAlgorithmMetaDataTest {
+    
+    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
+    private ShardingSphereDatabase database;
+    
+    @Mock
+    private ShardingSphereSchema schema;
+    
+    @Mock
+    private MaskRule maskRule;
+    
+    @Mock
+    private SelectStatementContext selectStatementContext;
+    
+    private MaskAlgorithm<?, ?> maskAlgorithm;
+    
+    @Before
+    public void setUp() {
+        when(database.getSchema(DefaultDatabase.LOGIC_NAME)).thenReturn(schema);
+        maskAlgorithm = (MaskAlgorithm<?, ?>) MaskAlgorithmFactory.newInstance(new AlgorithmConfiguration("MD5", new Properties()));
+    }
+    
+    @SuppressWarnings("rawtypes")
+    @Test
+    public void assertFindMaskAlgorithm() {
+        when(maskRule.findMaskAlgorithm("t_order", "id")).thenReturn(Optional.of(maskAlgorithm));
+        MaskAlgorithmMetaData maskAlgorithmMetaData = new MaskAlgorithmMetaData(database, maskRule, selectStatementContext);
+        Optional<MaskAlgorithm> actual = maskAlgorithmMetaData.findMaskAlgorithm("t_order", "id");
+        assertTrue(actual.isPresent());
+        assertThat(actual.get().getType(), is("MD5"));
+    }
+}
diff --git a/features/mask/core/src/test/java/org/apache/shardingsphere/mask/merge/dql/MaskDQLResultDecoratorTest.java b/features/mask/core/src/test/java/org/apache/shardingsphere/mask/merge/dql/MaskDQLResultDecoratorTest.java
new file mode 100644
index 00000000000..a2b77c42ffe
--- /dev/null
+++ b/features/mask/core/src/test/java/org/apache/shardingsphere/mask/merge/dql/MaskDQLResultDecoratorTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.mask.merge.dql;
+
+import org.apache.shardingsphere.mask.rule.MaskRule;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
+import org.apache.shardingsphere.infra.merge.result.MergedResult;
+import org.junit.Test;
+
+import java.sql.SQLException;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class MaskDQLResultDecoratorTest {
+    
+    @Test
+    public void assertDecorateQueryResult() throws SQLException {
+        QueryResult queryResult = mock(QueryResult.class);
+        when(queryResult.next()).thenReturn(true);
+        MaskDQLResultDecorator decorator = new MaskDQLResultDecorator(mock(MaskAlgorithmMetaData.class));
+        MergedResult actual = decorator.decorate(queryResult, mock(SQLStatementContext.class), mock(MaskRule.class));
+        assertTrue(actual.next());
+    }
+    
+    @Test
+    public void assertDecorateMergedResult() throws SQLException {
+        MergedResult mergedResult = mock(MergedResult.class);
+        when(mergedResult.next()).thenReturn(true);
+        MaskDQLResultDecorator decorator = new MaskDQLResultDecorator(mock(MaskAlgorithmMetaData.class));
+        MergedResult actual = decorator.decorate(mergedResult, mock(SQLStatementContext.class), mock(MaskRule.class));
+        assertTrue(actual.next());
+    }
+}
diff --git a/features/mask/core/src/test/java/org/apache/shardingsphere/mask/merge/dql/MaskMergedResultTest.java b/features/mask/core/src/test/java/org/apache/shardingsphere/mask/merge/dql/MaskMergedResultTest.java
new file mode 100644
index 00000000000..44f24ae6340
--- /dev/null
+++ b/features/mask/core/src/test/java/org/apache/shardingsphere/mask/merge/dql/MaskMergedResultTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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.mask.merge.dql;
+
+import org.apache.shardingsphere.infra.merge.result.MergedResult;
+import org.apache.shardingsphere.mask.spi.MaskAlgorithm;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.io.InputStream;
+import java.sql.SQLException;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.Optional;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertFalse;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class MaskMergedResultTest {
+    
+    @Mock
+    private MaskAlgorithmMetaData metaData;
+    
+    @Mock
+    private MergedResult mergedResult;
+    
+    @Test
+    public void assertNext() throws SQLException {
+        assertFalse(new MaskMergedResult(metaData, mergedResult).next());
+    }
+    
+    @Test
+    public void assertGetValue() throws SQLException {
+        when(mergedResult.getValue(1, Object.class)).thenReturn("VALUE");
+        MaskAlgorithm<String, String> maskAlgorithm = mock(MaskAlgorithm.class);
+        when(maskAlgorithm.mask("VALUE")).thenReturn("MASK_VALUE");
+        when(metaData.findMaskAlgorithmByColumnIndex(1)).thenReturn(Optional.of(maskAlgorithm));
+        assertThat(new MaskMergedResult(metaData, mergedResult).getValue(1, String.class), is("MASK_VALUE"));
+    }
+    
+    @Test
+    public void assertGetCalendarValue() throws SQLException {
+        Calendar calendar = Calendar.getInstance();
+        when(mergedResult.getCalendarValue(1, Date.class, calendar)).thenReturn(new Date(0L));
+        assertThat(new MaskMergedResult(metaData, mergedResult).getCalendarValue(1, Date.class, calendar), is(new Date(0L)));
+    }
+    
+    @Test
+    public void assertGetInputStream() throws SQLException {
+        InputStream inputStream = mock(InputStream.class);
+        when(mergedResult.getInputStream(1, "asc")).thenReturn(inputStream);
+        assertThat(new MaskMergedResult(metaData, mergedResult).getInputStream(1, "asc"), is(inputStream));
+    }
+    
+    @Test
+    public void assertWasNull() throws SQLException {
+        assertFalse(new MaskMergedResult(metaData, mergedResult).wasNull());
+    }
+}
diff --git a/features/mask/core/src/test/java/org/apache/shardingsphere/mask/rule/builder/MaskRuleBuilderTest.java b/features/mask/core/src/test/java/org/apache/shardingsphere/mask/rule/builder/MaskRuleBuilderTest.java
new file mode 100644
index 00000000000..b94491d0e88
--- /dev/null
+++ b/features/mask/core/src/test/java/org/apache/shardingsphere/mask/rule/builder/MaskRuleBuilderTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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.mask.rule.builder;
+
+import org.apache.shardingsphere.infra.instance.InstanceContext;
+import org.apache.shardingsphere.infra.rule.builder.database.DatabaseRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.database.DatabaseRuleBuilderFactory;
+import org.apache.shardingsphere.mask.api.config.MaskRuleConfiguration;
+import org.apache.shardingsphere.mask.rule.MaskRule;
+import org.junit.Test;
+
+import java.util.Collections;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.mock;
+
+public final class MaskRuleBuilderTest {
+    
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    @Test
+    public void assertBuild() {
+        MaskRuleConfiguration ruleConfig = mock(MaskRuleConfiguration.class);
+        DatabaseRuleBuilder builder = DatabaseRuleBuilderFactory.getInstanceMap(Collections.singletonList(ruleConfig)).get(ruleConfig);
+        assertThat(builder.build(ruleConfig, "", Collections.emptyMap(), Collections.emptyList(), mock(InstanceContext.class)), instanceOf(MaskRule.class));
+    }
+}
diff --git a/jdbc/core/pom.xml b/jdbc/core/pom.xml
index 8aab9232919..c53fd3877d2 100644
--- a/jdbc/core/pom.xml
+++ b/jdbc/core/pom.xml
@@ -135,6 +135,11 @@
             <artifactId>shardingsphere-encrypt-core</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-mask-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
             <artifactId>shardingsphere-shadow-core</artifactId>