You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by wu...@apache.org on 2021/05/27 02:33:46 UTC

[shardingsphere] branch master updated: Show encrypt rules & show encrypt table rule (#10490)

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

wuweijie 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 83eb8e3  Show encrypt rules & show encrypt table rule (#10490)
83eb8e3 is described below

commit 83eb8e31ba306265c79ea0a3c418b6938852e49b
Author: Haoran Meng <me...@gmail.com>
AuthorDate: Thu May 27 10:32:43 2021 +0800

    Show encrypt rules & show encrypt table rule (#10490)
    
    * Show encrypt rules & show encrypt table rule
    
    * Show encrypt rules & show encrypt table rule
    
    Co-authored-by: menghaoranss <me...@apache.org>
---
 .../src/main/antlr4/imports/RQLStatement.g4        |   8 ++
 .../distsql/parser/autogen/DistSQLStatement.g4     |   1 +
 .../distsql/parser/core/DistSQLVisitor.java        |  15 ++-
 .../api/DistSQLStatementParserEngineTest.java      |  11 ++
 .../rql/show/ShowEncryptTableRuleStatement.java}   |  49 +++------
 .../text/distsql/rql/RQLBackendHandlerFactory.java |  10 ++
 .../rql/impl/EncryptRulesQueryBackendHandler.java  | 110 ++++++++++++++++++++
 .../impl/EncryptTableRuleQueryBackendHandler.java  |  98 ++++++++++++++++++
 .../impl/EncryptRulesQueryBackendHandlerTest.java  | 111 ++++++++++++++++++++
 .../EncryptTableRuleQueryBackendHandlerTest.java   | 114 +++++++++++++++++++++
 10 files changed, 488 insertions(+), 39 deletions(-)

diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/RQLStatement.g4 b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/RQLStatement.g4
index 86e2374..2714e30 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/RQLStatement.g4
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/RQLStatement.g4
@@ -47,6 +47,10 @@ showEncryptRules
     : SHOW ENCRYPT RULES (FROM schemaName)?
     ;
 
+showEncryptTableRule
+    : SHOW ENCRYPT TABLE RULE tableName (FROM schemaName)?
+    ;
+
 ruleType
     : SHARDING | REPLICA_QUERY | ENCRYPT | SHADOW
     ;
@@ -54,3 +58,7 @@ ruleType
 schemaName
     : IDENTIFIER
     ;
+
+tableName
+    : IDENTIFIER
+    ;
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/DistSQLStatement.g4 b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/DistSQLStatement.g4
index b8e285e..2798467 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/DistSQLStatement.g4
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/DistSQLStatement.g4
@@ -47,6 +47,7 @@ execute
     | showReadwriteSplittingRules
     | showDatabaseDiscoveryRules
     | showEncryptRules
+    | showEncryptTableRule
     | showScalingJobList
     | showScalingJobStatus
     | startScalingJob
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/core/DistSQLVisitor.java b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/core/DistSQLVisitor.java
index ca9fd36..771900b 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/core/DistSQLVisitor.java
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/core/DistSQLVisitor.java
@@ -36,18 +36,19 @@ import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.F
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ResetScalingJobContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.SchemaNameContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShardingTableRuleDefinitionContext;
+import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowDatabaseDiscoveryRulesContext;
+import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowEncryptRulesContext;
+import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowEncryptTableRuleContext;
+import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowReadwriteSplittingRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowResourcesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowScalingJobListContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowScalingJobStatusContext;
+import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowShardingBroadcastTableRulesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowShardingRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.StartScalingJobContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.StopScalingJobContext;
 import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.TableNameContext;
-import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowShardingBroadcastTableRulesContext;
-import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowReadwriteSplittingRulesContext;
-import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowDatabaseDiscoveryRulesContext;
-import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementParser.ShowEncryptRulesContext;
 import org.apache.shardingsphere.distsql.parser.segment.DataSourceSegment;
 import org.apache.shardingsphere.distsql.parser.segment.FunctionSegment;
 import org.apache.shardingsphere.distsql.parser.segment.TableRuleSegment;
@@ -85,6 +86,7 @@ import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShar
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingTableRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowDatabaseDiscoveryRulesStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowEncryptRulesStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowEncryptTableRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowReadwriteSplittingRulesStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowResourcesStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowRuleStatement;
@@ -428,6 +430,11 @@ public final class DistSQLVisitor extends DistSQLStatementBaseVisitor<ASTNode> {
     }
 
     @Override
+    public ASTNode visitShowEncryptTableRule(final ShowEncryptTableRuleContext ctx) {
+        return new ShowEncryptTableRuleStatement(ctx.tableName().getText(), Objects.nonNull(ctx.schemaName()) ? (SchemaSegment) visit(ctx.schemaName()) : null);
+    }
+
+    @Override
     public ASTNode visitShowDatabaseDiscoveryRules(final ShowDatabaseDiscoveryRulesContext ctx) {
         return new ShowDatabaseDiscoveryRulesStatement(Objects.nonNull(ctx.schemaName()) ? (SchemaSegment) visit(ctx.schemaName()) : null);
     }
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/test/java/org/apache/shardingsphere/distsql/parser/api/DistSQLStatementParserEngineTest.java b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/test/java/org/apache/shardingsphere/distsql/parser/api/DistSQLStatementParserEngineTest.java
index f78148a..e4b0426 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/test/java/org/apache/shardingsphere/distsql/parser/api/DistSQLStatementParserEngineTest.java
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/test/java/org/apache/shardingsphere/distsql/parser/api/DistSQLStatementParserEngineTest.java
@@ -45,6 +45,7 @@ import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShar
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingTableRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowDatabaseDiscoveryRulesStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowEncryptRulesStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowEncryptTableRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowReadwriteSplittingRulesStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowShardingBindingTableRulesStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowShardingBroadcastTableRulesStatement;
@@ -165,6 +166,8 @@ public final class DistSQLStatementParserEngineTest {
 
     private static final String RQL_SHOW_ENCRYPT_RULES = "SHOW ENCRYPT RULES FROM encrypt_db";
 
+    private static final String RQL_SHOW_ENCRYPT_TABLE_RULE = "SHOW ENCRYPT TABLE RULE t_encrypt FROM encrypt_db";
+
     private final DistSQLStatementParserEngine engine = new DistSQLStatementParserEngine();
     
     @Test
@@ -463,4 +466,12 @@ public final class DistSQLStatementParserEngineTest {
         assertTrue(sqlStatement instanceof ShowEncryptRulesStatement);
         assertThat(((ShowEncryptRulesStatement) sqlStatement).getSchema().get().getIdentifier().getValue(), is("encrypt_db"));
     }
+
+    @Test
+    public void assertParseShowEncryptTableRule() {
+        SQLStatement sqlStatement = engine.parse(RQL_SHOW_ENCRYPT_TABLE_RULE);
+        assertTrue(sqlStatement instanceof ShowEncryptTableRuleStatement);
+        assertThat(((ShowEncryptTableRuleStatement) sqlStatement).getSchema().get().getIdentifier().getValue(), is("encrypt_db"));
+        assertThat(((ShowEncryptTableRuleStatement) sqlStatement).getTableName(), is("t_encrypt"));
+    }
 }
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/RQLStatement.g4 b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rql/show/ShowEncryptTableRuleStatement.java
similarity index 53%
copy from shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/RQLStatement.g4
copy to shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rql/show/ShowEncryptTableRuleStatement.java
index 86e2374..4235f97 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/antlr4/imports/RQLStatement.g4
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/rql/show/ShowEncryptTableRuleStatement.java
@@ -15,42 +15,21 @@
  * limitations under the License.
  */
 
-grammar RQLStatement;
+package org.apache.shardingsphere.distsql.parser.statement.rql.show;
 
-import Keyword, Literals, Symbol;
+import lombok.Getter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.SchemaSegment;
 
-showResources
-    : SHOW RESOURCES (FROM schemaName)?
-    ;
-
-showRule
-    : SHOW ruleType RULE (FROM schemaName)?
-    ;
-
-showShardingBindingTableRules
-    : SHOW SHARDING BINDING TABLE RULES (FROM schemaName)?
-    ;
-
-showShardingBroadcastTableRules
-    : SHOW SHARDING BROADCAST TABLE RULES (FROM schemaName)?
-    ;
-
-showReadwriteSplittingRules
-    : SHOW READWRITE_SPLITTING RULES (FROM schemaName)?
-    ;
-
-showDatabaseDiscoveryRules
-    : SHOW DB_DISCOVERY RULES (FROM schemaName)?
-    ;
-
-showEncryptRules
-    : SHOW ENCRYPT RULES (FROM schemaName)?
-    ;
+/**
+ * Show encrypt table rule statement.
+ */
+@Getter
+public final class ShowEncryptTableRuleStatement extends ShowRulesStatement {
 
-ruleType
-    : SHARDING | REPLICA_QUERY | ENCRYPT | SHADOW
-    ;
+    private final String tableName;
 
-schemaName
-    : IDENTIFIER
-    ;
+    public ShowEncryptTableRuleStatement(final String tableName, final SchemaSegment schema) {
+        super(schema);
+        this.tableName = tableName;
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/RQLBackendHandlerFactory.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/RQLBackendHandlerFactory.java
index 711f824..e945d1e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/RQLBackendHandlerFactory.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/RQLBackendHandlerFactory.java
@@ -20,6 +20,8 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.rql;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowDatabaseDiscoveryRulesStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowEncryptRulesStatement;
+import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowEncryptTableRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowReadwriteSplittingRulesStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowResourcesStatement;
 import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowShardingBindingTableRulesStatement;
@@ -28,6 +30,8 @@ import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.Bac
 import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rql.impl.DataSourcesQueryBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rql.impl.DatabaseDiscoveryRulesQueryBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.rql.impl.EncryptRulesQueryBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.rql.impl.EncryptTableRuleQueryBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rql.impl.ReadwriteSplittingRulesQueryBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rql.impl.ShardingBindingTableRulesQueryBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.rql.impl.ShardingBroadcastTableRulesQueryBackendHandler;
@@ -64,6 +68,12 @@ public final class RQLBackendHandlerFactory {
         if (sqlStatement instanceof ShowDatabaseDiscoveryRulesStatement) {
             return Optional.of(new DatabaseDiscoveryRulesQueryBackendHandler((ShowDatabaseDiscoveryRulesStatement) sqlStatement, backendConnection));
         }
+        if (sqlStatement instanceof ShowEncryptRulesStatement) {
+            return Optional.of(new EncryptRulesQueryBackendHandler((ShowEncryptRulesStatement) sqlStatement, backendConnection));
+        }
+        if (sqlStatement instanceof ShowEncryptTableRuleStatement) {
+            return Optional.of(new EncryptTableRuleQueryBackendHandler((ShowEncryptTableRuleStatement) sqlStatement, backendConnection));
+        }
         return Optional.empty();
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/impl/EncryptRulesQueryBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/impl/EncryptRulesQueryBackendHandler.java
new file mode 100644
index 0000000..879b383
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/impl/EncryptRulesQueryBackendHandler.java
@@ -0,0 +1,110 @@
+/*
+ * 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.proxy.backend.text.distsql.rql.impl;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Maps;
+import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowEncryptRulesStatement;
+import org.apache.shardingsphere.encrypt.api.config.EncryptRuleConfiguration;
+import org.apache.shardingsphere.encrypt.api.config.rule.EncryptColumnRuleConfiguration;
+import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfiguration;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.query.impl.QueryHeader;
+import org.apache.shardingsphere.proxy.backend.text.SchemaRequiredBackendHandler;
+
+import java.sql.Types;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
+/**
+ * Backend handler for show encrypt rules.
+ */
+public final class EncryptRulesQueryBackendHandler extends SchemaRequiredBackendHandler<ShowEncryptRulesStatement> {
+
+    private Iterator<Entry<String, EncryptColumnRuleConfiguration>> data;
+
+    private Map<String, ShardingSphereAlgorithmConfiguration> encryptors;
+
+    public EncryptRulesQueryBackendHandler(final ShowEncryptRulesStatement sqlStatement, final BackendConnection backendConnection) {
+        super(sqlStatement, backendConnection);
+    }
+    
+    @Override
+    protected ResponseHeader execute(final String schemaName, final ShowEncryptRulesStatement sqlStatement) {
+        loadRuleConfiguration(schemaName);
+        return new QueryResponseHeader(getQueryHeader(schemaName));
+    }
+    
+    private void loadRuleConfiguration(final String schemaName) {
+        Optional<EncryptRuleConfiguration> ruleConfig = ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations()
+                .stream().filter(each -> each instanceof EncryptRuleConfiguration).map(each -> (EncryptRuleConfiguration) each).findAny();
+        data = ruleConfig.map(optional -> getAllEncryptColumns(optional).entrySet().iterator()).orElse(Collections.emptyIterator());
+        encryptors = ruleConfig.map(EncryptRuleConfiguration::getEncryptors).orElse(Maps.newHashMap());
+    }
+
+    private Map<String, EncryptColumnRuleConfiguration> getAllEncryptColumns(final EncryptRuleConfiguration encryptRuleConfiguration) {
+        Map<String, EncryptColumnRuleConfiguration> result = new HashMap<>();
+        for (EncryptTableRuleConfiguration encryptTableRuleConfiguration : encryptRuleConfiguration.getTables()) {
+            encryptTableRuleConfiguration.getColumns().forEach(each -> result.put(Joiner.on(".")
+                    .join(encryptTableRuleConfiguration.getName(), each.getLogicColumn()), each));
+        }
+        return result;
+    }
+    
+    private List<QueryHeader> getQueryHeader(final String schemaName) {
+        List<QueryHeader> result = new LinkedList<>();
+        result.add(new QueryHeader(schemaName, "", "table", "table", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
+        result.add(new QueryHeader(schemaName, "", "logicColumn", "logicColumn", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
+        result.add(new QueryHeader(schemaName, "", "cipherColumn", "cipherColumn", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
+        result.add(new QueryHeader(schemaName, "", "plainColumn", "plainColumn", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
+        result.add(new QueryHeader(schemaName, "", "encryptorType", "encryptorType", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
+        result.add(new QueryHeader(schemaName, "", "encryptorProps", "encryptorProps", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
+        return result;
+    }
+    
+    @Override
+    public boolean next() {
+        return data.hasNext();
+    }
+    
+    @Override
+    public Collection<Object> getRowData() {
+        Entry<String, EncryptColumnRuleConfiguration> entry = data.next();
+        Properties encryptProps = encryptors.get(entry.getValue().getEncryptorName()).getProps();
+        return Arrays.asList(Splitter.on(".").splitToList(entry.getKey()).get(0), entry.getValue().getLogicColumn(),
+                entry.getValue().getCipherColumn(), entry.getValue().getPlainColumn(), encryptors.get(entry.getValue().getEncryptorName()).getType(),
+                Objects.nonNull(encryptProps) ? Joiner.on(",").join(encryptProps.entrySet().stream()
+                        .map(each -> Joiner.on(":").join(each.getKey(), each.getValue())).collect(Collectors.toList())) : "");
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/impl/EncryptTableRuleQueryBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/impl/EncryptTableRuleQueryBackendHandler.java
new file mode 100644
index 0000000..4c3248e
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/impl/EncryptTableRuleQueryBackendHandler.java
@@ -0,0 +1,98 @@
+/*
+ * 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.proxy.backend.text.distsql.rql.impl;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Maps;
+import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowEncryptTableRuleStatement;
+import org.apache.shardingsphere.encrypt.api.config.EncryptRuleConfiguration;
+import org.apache.shardingsphere.encrypt.api.config.rule.EncryptColumnRuleConfiguration;
+import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfiguration;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.query.impl.QueryHeader;
+import org.apache.shardingsphere.proxy.backend.text.SchemaRequiredBackendHandler;
+
+import java.sql.Types;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
+/**
+ * Backend handler for show encrypt table rule.
+ */
+public final class EncryptTableRuleQueryBackendHandler extends SchemaRequiredBackendHandler<ShowEncryptTableRuleStatement> {
+
+    private Iterator<EncryptColumnRuleConfiguration> data;
+
+    private Map<String, ShardingSphereAlgorithmConfiguration> encryptors;
+
+    public EncryptTableRuleQueryBackendHandler(final ShowEncryptTableRuleStatement sqlStatement, final BackendConnection backendConnection) {
+        super(sqlStatement, backendConnection);
+    }
+    
+    @Override
+    protected ResponseHeader execute(final String schemaName, final ShowEncryptTableRuleStatement sqlStatement) {
+        loadTableRuleConfiguration(schemaName, sqlStatement.getTableName());
+        return new QueryResponseHeader(getQueryHeader(schemaName));
+    }
+    
+    private void loadTableRuleConfiguration(final String schemaName, final String tableName) {
+        Optional<EncryptRuleConfiguration> ruleConfig = ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations()
+                .stream().filter(each -> each instanceof EncryptRuleConfiguration).map(each -> (EncryptRuleConfiguration) each).findAny();
+        Optional<EncryptTableRuleConfiguration> table = ruleConfig.map(optional -> optional.getTables()
+                .stream().filter(each -> tableName.equalsIgnoreCase(each.getName())).findAny()).orElse(Optional.empty());
+        data = table.map(optional -> optional.getColumns().stream().iterator()).orElse(Collections.emptyIterator());
+        encryptors = ruleConfig.map(EncryptRuleConfiguration::getEncryptors).orElse(Maps.newHashMap());
+    }
+
+    private List<QueryHeader> getQueryHeader(final String schemaName) {
+        List<QueryHeader> result = new LinkedList<>();
+        result.add(new QueryHeader(schemaName, "", "logicColumn", "logicColumn", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
+        result.add(new QueryHeader(schemaName, "", "cipherColumn", "cipherColumn", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
+        result.add(new QueryHeader(schemaName, "", "plainColumn", "plainColumn", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
+        result.add(new QueryHeader(schemaName, "", "encryptorType", "encryptorType", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
+        result.add(new QueryHeader(schemaName, "", "encryptorProps", "encryptorProps", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
+        return result;
+    }
+    
+    @Override
+    public boolean next() {
+        return data.hasNext();
+    }
+    
+    @Override
+    public Collection<Object> getRowData() {
+        EncryptColumnRuleConfiguration column = data.next();
+        Properties encryptProps = encryptors.get(column.getEncryptorName()).getProps();
+        return Arrays.asList(column.getLogicColumn(), column.getCipherColumn(), column.getPlainColumn(), encryptors.get(column.getEncryptorName()).getType(),
+                Objects.nonNull(encryptProps) ? Joiner.on(",").join(encryptProps.entrySet().stream()
+                        .map(each -> Joiner.on(":").join(each.getKey(), each.getValue())).collect(Collectors.toList())) : "");
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/impl/EncryptRulesQueryBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/impl/EncryptRulesQueryBackendHandlerTest.java
new file mode 100644
index 0000000..60e6f54
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/impl/EncryptRulesQueryBackendHandlerTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.proxy.backend.text.distsql.rql.impl;
+
+import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowEncryptRulesStatement;
+import org.apache.shardingsphere.encrypt.api.config.EncryptRuleConfiguration;
+import org.apache.shardingsphere.encrypt.api.config.rule.EncryptColumnRuleConfiguration;
+import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfiguration;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
+import org.apache.shardingsphere.transaction.context.TransactionContexts;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class EncryptRulesQueryBackendHandlerTest {
+
+    @Mock
+    private BackendConnection backendConnection;
+
+    @Mock
+    private ShowEncryptRulesStatement sqlStatement;
+
+    @Mock
+    private MetaDataContexts metaDataContexts;
+
+    @Mock
+    private TransactionContexts transactionContexts;
+
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+
+    @Mock
+    private ShardingSphereRuleMetaData ruleMetaData;
+
+    private EncryptRulesQueryBackendHandler handler;
+
+    @Before
+    public void setUp() {
+        ProxyContext.getInstance().init(metaDataContexts, transactionContexts);
+        handler = new EncryptRulesQueryBackendHandler(sqlStatement, backendConnection);
+        when(metaDataContexts.getAllSchemaNames()).thenReturn(Collections.singletonList("test"));
+        when(metaDataContexts.getMetaData(eq("test"))).thenReturn(shardingSphereMetaData);
+        when(shardingSphereMetaData.getRuleMetaData()).thenReturn(ruleMetaData);
+        when(ruleMetaData.getConfigurations()).thenReturn(Collections.singleton(buildEncryptRuleConfiguration()));
+    }
+
+    @Test
+    public void assertExecute() {
+        ResponseHeader responseHeader = handler.execute("test", sqlStatement);
+        assertNotNull(responseHeader);
+        assertTrue(responseHeader instanceof QueryResponseHeader);
+        Collection<Object> rowData = handler.getRowData();
+        assertThat(rowData.size(), is(6));
+        assertTrue(rowData.contains("t_encrypt"));
+        assertTrue(rowData.contains("user_id"));
+        assertTrue(rowData.contains("user_cipher"));
+        assertTrue(rowData.contains("user_plain"));
+        assertTrue(rowData.contains("md5"));
+    }
+
+    private EncryptRuleConfiguration buildEncryptRuleConfiguration() {
+        EncryptColumnRuleConfiguration encryptColumnRuleConfiguration =
+                new EncryptColumnRuleConfiguration("user_id", "user_cipher", null, "user_plain", "test");
+        EncryptTableRuleConfiguration encryptTableRuleConfiguration =
+                new EncryptTableRuleConfiguration("t_encrypt", Collections.singleton(encryptColumnRuleConfiguration));
+        ShardingSphereAlgorithmConfiguration shardingSphereAlgorithmConfiguration = new ShardingSphereAlgorithmConfiguration("md5", new Properties());
+        Map<String, ShardingSphereAlgorithmConfiguration> encryptors = new HashMap<>();
+        encryptors.put("test", shardingSphereAlgorithmConfiguration);
+        EncryptRuleConfiguration result =
+                new EncryptRuleConfiguration(Collections.singleton(encryptTableRuleConfiguration), encryptors);
+        return result;
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/impl/EncryptTableRuleQueryBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/impl/EncryptTableRuleQueryBackendHandlerTest.java
new file mode 100644
index 0000000..eae0e12
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/rql/impl/EncryptTableRuleQueryBackendHandlerTest.java
@@ -0,0 +1,114 @@
+/*
+ * 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.proxy.backend.text.distsql.rql.impl;
+
+import org.apache.shardingsphere.distsql.parser.statement.rql.show.ShowEncryptTableRuleStatement;
+import org.apache.shardingsphere.encrypt.api.config.EncryptRuleConfiguration;
+import org.apache.shardingsphere.encrypt.api.config.rule.EncryptColumnRuleConfiguration;
+import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfiguration;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
+import org.apache.shardingsphere.transaction.context.TransactionContexts;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class EncryptTableRuleQueryBackendHandlerTest {
+
+    @Mock
+    private BackendConnection backendConnection;
+
+    @Mock
+    private ShowEncryptTableRuleStatement sqlStatement;
+
+    @Mock
+    private MetaDataContexts metaDataContexts;
+
+    @Mock
+    private TransactionContexts transactionContexts;
+
+    @Mock
+    private ShardingSphereMetaData shardingSphereMetaData;
+
+    @Mock
+    private ShardingSphereRuleMetaData ruleMetaData;
+
+    private EncryptTableRuleQueryBackendHandler handler;
+
+    @Before
+    public void setUp() {
+        ProxyContext.getInstance().init(metaDataContexts, transactionContexts);
+        handler = new EncryptTableRuleQueryBackendHandler(sqlStatement, backendConnection);
+        when(metaDataContexts.getAllSchemaNames()).thenReturn(Collections.singletonList("test"));
+        when(metaDataContexts.getMetaData(eq("test"))).thenReturn(shardingSphereMetaData);
+        when(shardingSphereMetaData.getRuleMetaData()).thenReturn(ruleMetaData);
+        when(ruleMetaData.getConfigurations()).thenReturn(Collections.singleton(buildEncryptRuleConfiguration()));
+    }
+
+    @Test
+    public void assertExecute() {
+        when(sqlStatement.getTableName()).thenReturn("t_encrypt");
+        ResponseHeader responseHeader = handler.execute("test", sqlStatement);
+        assertNotNull(responseHeader);
+        assertTrue(responseHeader instanceof QueryResponseHeader);
+        Collection<Object> rowData = handler.getRowData();
+        assertThat(rowData.size(), is(5));
+        assertTrue(rowData.contains("user_id"));
+        assertTrue(rowData.contains("user_cipher"));
+        assertTrue(rowData.contains("user_plain"));
+        assertTrue(rowData.contains("aes"));
+        assertTrue(rowData.contains("aes-key-value:123"));
+    }
+
+    private EncryptRuleConfiguration buildEncryptRuleConfiguration() {
+        EncryptColumnRuleConfiguration encryptColumnRuleConfiguration =
+                new EncryptColumnRuleConfiguration("user_id", "user_cipher", null, "user_plain", "test");
+        EncryptTableRuleConfiguration encryptTableRuleConfiguration =
+                new EncryptTableRuleConfiguration("t_encrypt", Collections.singleton(encryptColumnRuleConfiguration));
+        Properties props = new Properties();
+        props.setProperty("aes-key-value", "123");
+        ShardingSphereAlgorithmConfiguration shardingSphereAlgorithmConfiguration = new ShardingSphereAlgorithmConfiguration("aes", props);
+        Map<String, ShardingSphereAlgorithmConfiguration> encryptors = new HashMap<>();
+        encryptors.put("test", shardingSphereAlgorithmConfiguration);
+        EncryptRuleConfiguration result =
+                new EncryptRuleConfiguration(Collections.singleton(encryptTableRuleConfiguration), encryptors);
+        return result;
+    }
+}