You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ji...@apache.org on 2022/08/13 07:56:21 UTC

[shardingsphere] branch master updated: Migrate global rule (AuthorityRule) to shardingsphere-authority-distsql module. (#20132)

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

jianglongtao 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 c8665f791b7 Migrate global rule (AuthorityRule) to shardingsphere-authority-distsql module. (#20132)
c8665f791b7 is described below

commit c8665f791b7f19ed32d8172226869cb7e36f0a85
Author: yx9o <ya...@163.com>
AuthorDate: Sat Aug 13 15:56:14 2022 +0800

    Migrate global rule (AuthorityRule) to shardingsphere-authority-distsql module. (#20132)
    
    * Migrate global rule (AuthorityRule) to shardingsphere-authority-distsql module.
    
    * Add AuthorityRuleQueryResultSet.
    
    * Improve authority show rule.
    
    * Update QueryableGlobalRuleRALBackendHandlerFactory.
    
    * Remove useless imports.
---
 .../src/main/antlr4/imports/RALStatement.g4        |  4 --
 .../parser/autogen/KernelDistSQLStatement.g4       |  1 -
 .../core/kernel/KernelDistSQLStatementVisitor.java |  7 --
 ...t.java => QueryableGlobalRuleRALStatement.java} |  8 +--
 .../shardingsphere-authority/pom.xml               |  1 +
 .../{ => shardingsphere-authority-distsql}/pom.xml | 11 +--
 .../pom.xml                                        | 26 +++++--
 .../handler/AuthorityRuleQueryResultSet.java       | 80 ++++++++++++++++++++++
 ...dingsphere.infra.distsql.query.DistSQLResultSet | 18 +++++
 .../handler/AuthorityRuleQueryResultSetTest.java   | 65 ++++++++++++++++++
 .../pom.xml                                        | 69 +++++++++++++++++++
 .../src/main/antlr4/imports/authority/Alphabet.g4  | 36 ++++++++--
 .../src/main/antlr4/imports/authority/Keyword.g4   | 24 +++++--
 .../main/antlr4/imports/authority/RALStatement.g4  | 12 ++--
 .../src/main/antlr4/imports/authority/Symbol.g4    | 62 +++++++++++++++++
 .../parser/autogen/AuthorityDistSQLStatement.g4    | 12 ++--
 .../distsql/parser/core/AuthorityDistSQLLexer.java | 14 ++--
 .../parser/core/AuthorityDistSQLParser.java        | 22 ++++--
 .../core/AuthorityDistSQLStatementVisitor.java     | 35 ++++++++++
 .../AuthorityDistSQLStatementParserFacade.java     | 52 ++++++++++++++
 ...engine.spi.FeaturedDistSQLStatementParserFacade | 18 +++++
 .../pom.xml                                        | 16 +++--
 .../statement}/ShowAuthorityRuleStatement.java     |  6 +-
 .../shardingsphere-proxy-backend/pom.xml           |  5 ++
 .../ral/QueryableGlobalRuleRALBackendHandler.java  | 73 ++++++++++++++++++++
 ...ueryableGlobalRuleRALBackendHandlerFactory.java | 51 ++++++++++++++
 .../distsql/ral/RALBackendHandlerFactory.java      | 13 ++--
 .../ral/queryable/ShowAuthorityRuleHandler.java    | 59 ----------------
 .../queryable/ShowAuthorityRuleHandlerTest.java    | 65 ------------------
 .../shardingsphere-parser-test/pom.xml             |  5 ++
 .../ral/impl/QueryableRALStatementAssert.java      |  2 +-
 .../ShowAuthorityRuleStatementAssert.java          | 10 ++-
 32 files changed, 675 insertions(+), 207 deletions(-)

diff --git a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4 b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4
index d3aacff8f80..36291695732 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4
+++ b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4
@@ -59,10 +59,6 @@ showTableMetadata
     : SHOW TABLE METADATA tableName (COMMA tableName*)? (FROM databaseName)?
     ;
 
-showAuthorityRule
-    : SHOW AUTHORITY RULE
-    ;
-
 showTransactionRule
     : SHOW TRANSACTION RULE
     ;
diff --git a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/KernelDistSQLStatement.g4 b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/KernelDistSQLStatement.g4
index c6a2435e415..6a730e9c1eb 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/KernelDistSQLStatement.g4
+++ b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/KernelDistSQLStatement.g4
@@ -51,7 +51,6 @@ execute
     | showTableMetadata
     | showSQLParserRule
     | alterSQLParserRule
-    | showAuthorityRule
     | showTransactionRule
     | alterTransactionRule
     | showTrafficRules
diff --git a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/java/org/apache/shardingsphere/distsql/parser/core/kernel/KernelDistSQLStatementVisitor.java b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/java/org/apache/shardingsphere/distsql/parser/core/kernel/KernelDistSQLStatementVisitor.java
index 60c89440a43..59bbb4a2da8 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/java/org/apache/shardingsphere/distsql/parser/core/kernel/KernelDistSQLStatementVisitor.java
+++ b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/java/org/apache/shardingsphere/distsql/parser/core/kernel/KernelDistSQLStatementVisitor.java
@@ -58,7 +58,6 @@ import org.apache.shardingsphere.distsql.parser.autogen.KernelDistSQLStatementPa
 import org.apache.shardingsphere.distsql.parser.autogen.KernelDistSQLStatementParser.ResourceDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.KernelDistSQLStatementParser.SetVariableContext;
 import org.apache.shardingsphere.distsql.parser.autogen.KernelDistSQLStatementParser.ShowAllVariablesContext;
-import org.apache.shardingsphere.distsql.parser.autogen.KernelDistSQLStatementParser.ShowAuthorityRuleContext;
 import org.apache.shardingsphere.distsql.parser.autogen.KernelDistSQLStatementParser.ShowInstanceInfoContext;
 import org.apache.shardingsphere.distsql.parser.autogen.KernelDistSQLStatementParser.ShowInstanceListContext;
 import org.apache.shardingsphere.distsql.parser.autogen.KernelDistSQLStatementParser.ShowModeInfoContext;
@@ -88,7 +87,6 @@ import org.apache.shardingsphere.distsql.parser.statement.ral.hint.ClearHintStat
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ConvertYamlConfigurationStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ExportDatabaseConfigurationStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowAllVariableStatement;
-import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowAuthorityRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowInstanceInfoStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowInstanceListStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowModeInfoStatement;
@@ -327,11 +325,6 @@ public final class KernelDistSQLStatementVisitor extends KernelDistSQLStatementB
         return new RefreshTableMetadataStatement(tableName, resourceName, schemaName);
     }
     
-    @Override
-    public ASTNode visitShowAuthorityRule(final ShowAuthorityRuleContext ctx) {
-        return new ShowAuthorityRuleStatement();
-    }
-    
     @Override
     public ASTNode visitShowTransactionRule(final ShowTransactionRuleContext ctx) {
         return new ShowTransactionRuleStatement();
diff --git a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java b/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/QueryableGlobalRuleRALStatement.java
similarity index 74%
copy from shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
copy to shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/QueryableGlobalRuleRALStatement.java
index 72c7409fbc3..b1c9cac65fc 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
+++ b/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/QueryableGlobalRuleRALStatement.java
@@ -15,12 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.parser.statement.ral.queryable;
-
-import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
+package org.apache.shardingsphere.distsql.parser.statement.ral;
 
 /**
- * Show authority rule statement.
+ * Queryable RAL statement for global rule.
  */
-public final class ShowAuthorityRuleStatement extends QueryableRALStatement {
+public abstract class QueryableGlobalRuleRALStatement extends QueryableRALStatement {
 }
diff --git a/shardingsphere-kernel/shardingsphere-authority/pom.xml b/shardingsphere-kernel/shardingsphere-authority/pom.xml
index 4c538c90b64..4ed0a509ca8 100644
--- a/shardingsphere-kernel/shardingsphere-authority/pom.xml
+++ b/shardingsphere-kernel/shardingsphere-authority/pom.xml
@@ -31,5 +31,6 @@
     <modules>
         <module>shardingsphere-authority-api</module>
         <module>shardingsphere-authority-core</module>
+        <module>shardingsphere-authority-distsql</module>
     </modules>
 </project>
diff --git a/shardingsphere-kernel/shardingsphere-authority/pom.xml b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/pom.xml
similarity index 80%
copy from shardingsphere-kernel/shardingsphere-authority/pom.xml
copy to shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/pom.xml
index 4c538c90b64..30c019d31e8 100644
--- a/shardingsphere-kernel/shardingsphere-authority/pom.xml
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/pom.xml
@@ -21,15 +21,16 @@
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache.shardingsphere</groupId>
-        <artifactId>shardingsphere-kernel</artifactId>
+        <artifactId>shardingsphere-authority</artifactId>
         <version>5.1.3-SNAPSHOT</version>
     </parent>
-    <artifactId>shardingsphere-authority</artifactId>
+    <artifactId>shardingsphere-authority-distsql</artifactId>
     <packaging>pom</packaging>
     <name>${project.artifactId}</name>
-    
+
     <modules>
-        <module>shardingsphere-authority-api</module>
-        <module>shardingsphere-authority-core</module>
+        <module>shardingsphere-authority-distsql-statement</module>
+        <module>shardingsphere-authority-distsql-parser</module>
+        <module>shardingsphere-authority-distsql-handler</module>
     </modules>
 </project>
diff --git a/shardingsphere-kernel/shardingsphere-authority/pom.xml b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-handler/pom.xml
similarity index 60%
copy from shardingsphere-kernel/shardingsphere-authority/pom.xml
copy to shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-handler/pom.xml
index 4c538c90b64..c508a961b08 100644
--- a/shardingsphere-kernel/shardingsphere-authority/pom.xml
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-handler/pom.xml
@@ -21,15 +21,27 @@
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache.shardingsphere</groupId>
-        <artifactId>shardingsphere-kernel</artifactId>
+        <artifactId>shardingsphere-authority-distsql</artifactId>
         <version>5.1.3-SNAPSHOT</version>
     </parent>
-    <artifactId>shardingsphere-authority</artifactId>
-    <packaging>pom</packaging>
+    <artifactId>shardingsphere-authority-distsql-handler</artifactId>
     <name>${project.artifactId}</name>
     
-    <modules>
-        <module>shardingsphere-authority-api</module>
-        <module>shardingsphere-authority-core</module>
-    </modules>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-authority-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-authority-distsql-statement</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-authority-distsql-parser</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
 </project>
diff --git a/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-handler/src/main/java/org/apache/shardingsphere/authority/distsql/handler/AuthorityRuleQueryResultSet.java b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-handler/src/main/java/org/apache/shardingsphere/authority/distsql/handler/AuthorityRuleQueryResultSet.java
new file mode 100644
index 00000000000..5a5f5c2569e
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-handler/src/main/java/org/apache/shardingsphere/authority/distsql/handler/AuthorityRuleQueryResultSet.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.authority.distsql.handler;
+
+import org.apache.shardingsphere.authority.config.AuthorityRuleConfiguration;
+import org.apache.shardingsphere.authority.distsql.parser.statement.ShowAuthorityRuleStatement;
+import org.apache.shardingsphere.authority.rule.AuthorityRule;
+import org.apache.shardingsphere.infra.distsql.query.GlobalRuleDistSQLResultSet;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Query result set for authority rule.
+ */
+public final class AuthorityRuleQueryResultSet implements GlobalRuleDistSQLResultSet {
+    
+    private static final String USERS = "users";
+    
+    private static final String PROVIDER = "provider";
+    
+    private static final String PROPS = "props";
+    
+    private Iterator<Collection<Object>> data = Collections.emptyIterator();
+    
+    @Override
+    public void init(final ShardingSphereRuleMetaData ruleMetaData, final SQLStatement sqlStatement) {
+        Optional<AuthorityRule> rule = ruleMetaData.findSingleRule(AuthorityRule.class);
+        rule.ifPresent(optional -> data = buildData(optional.getConfiguration()).iterator());
+    }
+    
+    private Collection<Collection<Object>> buildData(final AuthorityRuleConfiguration ruleConfig) {
+        Collection<Collection<Object>> result = new LinkedList<>();
+        result.add(Arrays.asList(ruleConfig.getUsers().stream().map(each -> each.getGrantee().toString()).collect(Collectors.joining("; ")),
+                ruleConfig.getProvider().getType(), ruleConfig.getProvider().getProps().size() == 0 ? "" : ruleConfig.getProvider().getProps()));
+        return result;
+    }
+    
+    @Override
+    public Collection<String> getColumnNames() {
+        return Arrays.asList(USERS, PROVIDER, PROPS);
+    }
+    
+    @Override
+    public boolean next() {
+        return data.hasNext();
+    }
+    
+    @Override
+    public Collection<Object> getRowData() {
+        return data.next();
+    }
+    
+    @Override
+    public String getType() {
+        return ShowAuthorityRuleStatement.class.getName();
+    }
+}
diff --git a/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet
new file mode 100644
index 00000000000..f3e8dae60da
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-handler/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+org.apache.shardingsphere.authority.distsql.handler.AuthorityRuleQueryResultSet
diff --git a/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-handler/src/test/java/org/apache/shardingsphere/authority/distsql/handler/AuthorityRuleQueryResultSetTest.java b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-handler/src/test/java/org/apache/shardingsphere/authority/distsql/handler/AuthorityRuleQueryResultSetTest.java
new file mode 100644
index 00000000000..4e57391ad3d
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-handler/src/test/java/org/apache/shardingsphere/authority/distsql/handler/AuthorityRuleQueryResultSetTest.java
@@ -0,0 +1,65 @@
+/*
+ * 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.authority.distsql.handler;
+
+import org.apache.shardingsphere.authority.config.AuthorityRuleConfiguration;
+import org.apache.shardingsphere.authority.distsql.parser.statement.ShowAuthorityRuleStatement;
+import org.apache.shardingsphere.authority.rule.AuthorityRule;
+import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
+import org.apache.shardingsphere.infra.distsql.query.GlobalRuleDistSQLResultSet;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class AuthorityRuleQueryResultSetTest {
+    
+    @Test
+    public void assertAuthorityRule() {
+        ShardingSphereRuleMetaData ruleMetaData = mockRuleMetaData();
+        GlobalRuleDistSQLResultSet resultSet = new AuthorityRuleQueryResultSet();
+        resultSet.init(ruleMetaData, mock(ShowAuthorityRuleStatement.class));
+        Collection<Object> actual = resultSet.getRowData();
+        assertThat(actual.size(), is(3));
+        assertTrue(actual.contains("root@localhost"));
+        assertTrue(actual.contains("ALL_PERMITTED"));
+        assertTrue(actual.contains(""));
+    }
+    
+    private ShardingSphereRuleMetaData mockRuleMetaData() {
+        AuthorityRule authorityRule = new AuthorityRule(getAuthorityRuleConfiguration(), Collections.emptyMap());
+        ShardingSphereRuleMetaData result = mock(ShardingSphereRuleMetaData.class);
+        when(result.findSingleRule(AuthorityRule.class)).thenReturn(Optional.of(authorityRule));
+        return result;
+    }
+    
+    private AuthorityRuleConfiguration getAuthorityRuleConfiguration() {
+        ShardingSphereUser root = new ShardingSphereUser("root", "", "localhost");
+        return new AuthorityRuleConfiguration(Collections.singleton(root), new AlgorithmConfiguration("ALL_PERMITTED", new Properties()));
+    }
+}
diff --git a/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/pom.xml b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/pom.xml
new file mode 100644
index 00000000000..e6dc8f7eb9c
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/pom.xml
@@ -0,0 +1,69 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.shardingsphere</groupId>
+        <artifactId>shardingsphere-authority-distsql</artifactId>
+        <version>5.1.3-SNAPSHOT</version>
+    </parent>
+    <artifactId>shardingsphere-authority-distsql-parser</artifactId>
+    <name>${project.artifactId}</name>
+    
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-sql-parser-spi</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-distsql-parser</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-authority-distsql-statement</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+    
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.antlr</groupId>
+                <artifactId>antlr4-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>antlr</id>
+                        <goals>
+                            <goal>antlr4</goal>
+                        </goals>
+                        <configuration>
+                            <libDirectory>src/main/antlr4/imports/authority/</libDirectory>
+                            <listener>false</listener>
+                            <visitor>true</visitor>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>
diff --git a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/imports/authority/Alphabet.g4
similarity index 56%
copy from shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
copy to shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/imports/authority/Alphabet.g4
index 72c7409fbc3..f7603cc7e63 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/imports/authority/Alphabet.g4
@@ -15,12 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.parser.statement.ral.queryable;
+lexer grammar Alphabet;
 
-import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
+FOR_GENERATOR: 'DO NOT MATCH ANY THING, JUST FOR GENERATOR';
 
-/**
- * Show authority rule statement.
- */
-public final class ShowAuthorityRuleStatement extends QueryableRALStatement {
-}
+fragment A:   [Aa];
+fragment B:   [Bb];
+fragment C:   [Cc];
+fragment D:   [Dd];
+fragment E:   [Ee];
+fragment F:   [Ff];
+fragment G:   [Gg];
+fragment H:   [Hh];
+fragment I:   [Ii];
+fragment J:   [Jj];
+fragment K:   [Kk];
+fragment L:   [Ll];
+fragment M:   [Mm];
+fragment N:   [Nn];
+fragment O:   [Oo];
+fragment P:   [Pp];
+fragment Q:   [Qq];
+fragment R:   [Rr];
+fragment S:   [Ss];
+fragment T:   [Tt];
+fragment U:   [Uu];
+fragment V:   [Vv];
+fragment W:   [Ww];
+fragment X:   [Xx];
+fragment Y:   [Yy];
+fragment Z:   [Zz];
+fragment UL_: '_';
diff --git a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/imports/authority/Keyword.g4
similarity index 74%
copy from shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
copy to shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/imports/authority/Keyword.g4
index 72c7409fbc3..66eff15984c 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/imports/authority/Keyword.g4
@@ -15,12 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.parser.statement.ral.queryable;
+lexer grammar Keyword;
 
-import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
+import Alphabet;
 
-/**
- * Show authority rule statement.
- */
-public final class ShowAuthorityRuleStatement extends QueryableRALStatement {
-}
+WS
+    : [ \t\r\n] + ->skip
+    ;
+
+SHOW
+    : S H O W
+    ;
+
+RULE
+    : R U L E
+    ;
+
+AUTHORITY
+    : A U T H O R I T Y
+    ;
diff --git a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/imports/authority/RALStatement.g4
similarity index 74%
copy from shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
copy to shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/imports/authority/RALStatement.g4
index 72c7409fbc3..e973b92d3e7 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/imports/authority/RALStatement.g4
@@ -15,12 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.parser.statement.ral.queryable;
+grammar RALStatement;
 
-import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
+import Keyword;
 
-/**
- * Show authority rule statement.
- */
-public final class ShowAuthorityRuleStatement extends QueryableRALStatement {
-}
+showAuthorityRule
+    : SHOW AUTHORITY RULE
+    ;
diff --git a/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/imports/authority/Symbol.g4 b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/imports/authority/Symbol.g4
new file mode 100644
index 00000000000..3322b09bb3a
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/imports/authority/Symbol.g4
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+
+lexer grammar Symbol;
+
+AND:                '&&';
+OR:                 '||';
+NOT:                '!';
+TILDE:              '~';
+VERTICALBAR:       '|';
+AMPERSAND:          '&';
+SIGNEDLEFTSHIFT:  '<<';
+SIGNEDRIGHTSHIFT: '>>';
+CARET:              '^';
+MOD:                '%';
+COLON:              ':';
+PLUS:               '+';
+MINUS:              '-';
+ASTERISK:           '*';
+SLASH:              '/';
+BACKSLASH:          '\\';
+DOT:                '.';
+DOTASTERISK:       '.*';
+SAFEEQ:            '<=>';
+DEQ:                '==';
+EQ:                 '=';
+NEQ:                '<>' | '!=';
+GT:                 '>';
+GTE:                '>=';
+LT:                 '<';
+LTE:                '<=';
+POUND:              '#';
+LP:                 '(';
+RP:                 ')';
+LBE:                '{';
+RBE:                '}';
+LBT:                '[';
+RBT:                ']';
+COMMA:              ',';
+DQ:                 '"';
+SQ:                 '\'';
+BQ:                 '`';
+QUESTION:           '?';
+AT:                 '@';
+SEMI:               ';';
+JSONSEPARATOR:      '->>';
+UL:                 '_';
+DL:                 '$';
diff --git a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/AuthorityDistSQLStatement.g4
similarity index 74%
copy from shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
copy to shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/AuthorityDistSQLStatement.g4
index 72c7409fbc3..fcfd6053e6c 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/AuthorityDistSQLStatement.g4
@@ -15,12 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.parser.statement.ral.queryable;
+grammar AuthorityDistSQLStatement;
 
-import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
+import Symbol, RALStatement;
 
-/**
- * Show authority rule statement.
- */
-public final class ShowAuthorityRuleStatement extends QueryableRALStatement {
-}
+execute
+    : (showAuthorityRule) SEMI?
+    ;
diff --git a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/java/org/apache/shardingsphere/authority/distsql/parser/core/AuthorityDistSQLLexer.java
similarity index 62%
copy from shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
copy to shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/java/org/apache/shardingsphere/authority/distsql/parser/core/AuthorityDistSQLLexer.java
index 72c7409fbc3..a982ee01bf4 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/java/org/apache/shardingsphere/authority/distsql/parser/core/AuthorityDistSQLLexer.java
@@ -15,12 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.parser.statement.ral.queryable;
+package org.apache.shardingsphere.authority.distsql.parser.core;
 
-import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
+import org.antlr.v4.runtime.CharStream;
+import org.apache.shardingsphere.distsql.parser.autogen.AuthorityDistSQLStatementLexer;
+import org.apache.shardingsphere.sql.parser.api.parser.SQLLexer;
 
 /**
- * Show authority rule statement.
+ * SQL lexer for authority dist SQL.
  */
-public final class ShowAuthorityRuleStatement extends QueryableRALStatement {
+public final class AuthorityDistSQLLexer extends AuthorityDistSQLStatementLexer implements SQLLexer {
+    
+    public AuthorityDistSQLLexer(final CharStream input) {
+        super(input);
+    }
 }
diff --git a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/java/org/apache/shardingsphere/authority/distsql/parser/core/AuthorityDistSQLParser.java
similarity index 50%
copy from shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
copy to shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/java/org/apache/shardingsphere/authority/distsql/parser/core/AuthorityDistSQLParser.java
index 72c7409fbc3..d0e1516b939 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/java/org/apache/shardingsphere/authority/distsql/parser/core/AuthorityDistSQLParser.java
@@ -15,12 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.parser.statement.ral.queryable;
+package org.apache.shardingsphere.authority.distsql.parser.core;
 
-import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.TokenStream;
+import org.apache.shardingsphere.distsql.parser.autogen.AuthorityDistSQLStatementParser;
+import org.apache.shardingsphere.sql.parser.api.parser.SQLParser;
+import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
+import org.apache.shardingsphere.sql.parser.core.ParseASTNode;
 
 /**
- * Show authority rule statement.
+ * SQL parser for authority dist SQL.
  */
-public final class ShowAuthorityRuleStatement extends QueryableRALStatement {
+public final class AuthorityDistSQLParser extends AuthorityDistSQLStatementParser implements SQLParser {
+    
+    public AuthorityDistSQLParser(final TokenStream input) {
+        super(input);
+    }
+    
+    @Override
+    public ASTNode parse() {
+        return new ParseASTNode(execute(), (CommonTokenStream) getTokenStream());
+    }
 }
diff --git a/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/java/org/apache/shardingsphere/authority/distsql/parser/core/AuthorityDistSQLStatementVisitor.java b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/java/org/apache/shardingsphere/authority/distsql/parser/core/AuthorityDistSQLStatementVisitor.java
new file mode 100644
index 00000000000..66fd1d87e75
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/java/org/apache/shardingsphere/authority/distsql/parser/core/AuthorityDistSQLStatementVisitor.java
@@ -0,0 +1,35 @@
+/*
+ * 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.authority.distsql.parser.core;
+
+import org.apache.shardingsphere.authority.distsql.parser.statement.ShowAuthorityRuleStatement;
+import org.apache.shardingsphere.distsql.parser.autogen.AuthorityDistSQLStatementBaseVisitor;
+import org.apache.shardingsphere.distsql.parser.autogen.AuthorityDistSQLStatementParser.ShowAuthorityRuleContext;
+import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
+import org.apache.shardingsphere.sql.parser.api.visitor.SQLVisitor;
+
+/**
+ * SQL statement visitor for authority dist SQL.
+ */
+public final class AuthorityDistSQLStatementVisitor extends AuthorityDistSQLStatementBaseVisitor<ASTNode> implements SQLVisitor {
+    
+    @Override
+    public ASTNode visitShowAuthorityRule(final ShowAuthorityRuleContext ctx) {
+        return new ShowAuthorityRuleStatement();
+    }
+}
diff --git a/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/java/org/apache/shardingsphere/authority/distsql/parser/facade/AuthorityDistSQLStatementParserFacade.java b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/java/org/apache/shardingsphere/authority/distsql/parser/facade/AuthorityDistSQLStatementParserFacade.java
new file mode 100644
index 00000000000..8e89d8223c0
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/java/org/apache/shardingsphere/authority/distsql/parser/facade/AuthorityDistSQLStatementParserFacade.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.authority.distsql.parser.facade;
+
+import org.apache.shardingsphere.authority.distsql.parser.core.AuthorityDistSQLLexer;
+import org.apache.shardingsphere.authority.distsql.parser.core.AuthorityDistSQLParser;
+import org.apache.shardingsphere.authority.distsql.parser.core.AuthorityDistSQLStatementVisitor;
+import org.apache.shardingsphere.distsql.parser.engine.spi.FeaturedDistSQLStatementParserFacade;
+import org.apache.shardingsphere.sql.parser.api.parser.SQLLexer;
+import org.apache.shardingsphere.sql.parser.api.parser.SQLParser;
+import org.apache.shardingsphere.sql.parser.api.visitor.SQLVisitor;
+
+/**
+ * SQL parser facade for authority dist SQL statement.
+ */
+public final class AuthorityDistSQLStatementParserFacade implements FeaturedDistSQLStatementParserFacade {
+    
+    @Override
+    public Class<? extends SQLLexer> getLexerClass() {
+        return AuthorityDistSQLLexer.class;
+    }
+    
+    @Override
+    public Class<? extends SQLParser> getParserClass() {
+        return AuthorityDistSQLParser.class;
+    }
+    
+    @Override
+    public Class<? extends SQLVisitor> getVisitorClass() {
+        return AuthorityDistSQLStatementVisitor.class;
+    }
+    
+    @Override
+    public String getType() {
+        return "authority";
+    }
+}
diff --git a/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.parser.engine.spi.FeaturedDistSQLStatementParserFacade b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.parser.engine.spi.FeaturedDistSQLStatemen [...]
new file mode 100644
index 00000000000..67647877138
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-parser/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.parser.engine.spi.FeaturedDistSQLStatementParserFacade
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+org.apache.shardingsphere.authority.distsql.parser.facade.AuthorityDistSQLStatementParserFacade
diff --git a/shardingsphere-kernel/shardingsphere-authority/pom.xml b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-statement/pom.xml
similarity index 76%
copy from shardingsphere-kernel/shardingsphere-authority/pom.xml
copy to shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-statement/pom.xml
index 4c538c90b64..022a67fe88a 100644
--- a/shardingsphere-kernel/shardingsphere-authority/pom.xml
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-statement/pom.xml
@@ -21,15 +21,17 @@
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache.shardingsphere</groupId>
-        <artifactId>shardingsphere-kernel</artifactId>
+        <artifactId>shardingsphere-authority-distsql</artifactId>
         <version>5.1.3-SNAPSHOT</version>
     </parent>
-    <artifactId>shardingsphere-authority</artifactId>
-    <packaging>pom</packaging>
+    <artifactId>shardingsphere-authority-distsql-statement</artifactId>
     <name>${project.artifactId}</name>
     
-    <modules>
-        <module>shardingsphere-authority-api</module>
-        <module>shardingsphere-authority-core</module>
-    </modules>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-distsql-statement</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
 </project>
diff --git a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-statement/src/main/java/org/apache/shardingsphere/authority/distsql/parser/statement/ShowAuthorityRuleStatement.java
similarity index 82%
rename from shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
rename to shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-statement/src/main/java/org/apache/shardingsphere/authority/distsql/parser/statement/ShowAuthorityRuleStatement.java
index 72c7409fbc3..eb1fe8d79e2 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/queryable/ShowAuthorityRuleStatement.java
+++ b/shardingsphere-kernel/shardingsphere-authority/shardingsphere-authority-distsql/shardingsphere-authority-distsql-statement/src/main/java/org/apache/shardingsphere/authority/distsql/parser/statement/ShowAuthorityRuleStatement.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.parser.statement.ral.queryable;
+package org.apache.shardingsphere.authority.distsql.parser.statement;
 
-import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
+import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableGlobalRuleRALStatement;
 
 /**
  * Show authority rule statement.
  */
-public final class ShowAuthorityRuleStatement extends QueryableRALStatement {
+public final class ShowAuthorityRuleStatement extends QueryableGlobalRuleRALStatement {
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/pom.xml b/shardingsphere-proxy/shardingsphere-proxy-backend/pom.xml
index feb25d70356..28e3a15fb7e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/pom.xml
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/pom.xml
@@ -133,6 +133,11 @@
             <artifactId>shardingsphere-shadow-distsql-handler</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-authority-distsql-handler</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
             <artifactId>shardingsphere-authority-core</artifactId>
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/QueryableGlobalRuleRALBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/QueryableGlobalRuleRALBackendHandler.java
new file mode 100644
index 00000000000..614e311c989
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/QueryableGlobalRuleRALBackendHandler.java
@@ -0,0 +1,73 @@
+/*
+ * 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.handler.distsql.ral;
+
+import org.apache.shardingsphere.distsql.parser.statement.ral.RALStatement;
+import org.apache.shardingsphere.infra.distsql.query.GlobalRuleDistSQLResultSet;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.apache.shardingsphere.proxy.backend.handler.ProxyBackendHandler;
+import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
+import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseRow;
+import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
+
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Queryable RAL backend handler for global rule.
+ */
+public final class QueryableGlobalRuleRALBackendHandler implements ProxyBackendHandler {
+    
+    private final RALStatement sqlStatement;
+    
+    private final GlobalRuleDistSQLResultSet resultSet;
+    
+    public QueryableGlobalRuleRALBackendHandler(final RALStatement sqlStatement, final GlobalRuleDistSQLResultSet resultSet) {
+        this.sqlStatement = sqlStatement;
+        this.resultSet = resultSet;
+    }
+    
+    @Override
+    public ResponseHeader execute() {
+        resultSet.init(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData(), sqlStatement);
+        List<QueryHeader> queryHeaders = new ArrayList<>();
+        for (String each : resultSet.getColumnNames()) {
+            queryHeaders.add(new QueryHeader("", "", each, each, Types.CHAR, "CHAR", 255, 0, false, false, false, false));
+        }
+        return new QueryResponseHeader(queryHeaders);
+    }
+    
+    @Override
+    public boolean next() {
+        return resultSet.next();
+    }
+    
+    @Override
+    public QueryResponseRow getRowData() {
+        Collection<Object> rowData = resultSet.getRowData();
+        List<QueryResponseCell> result = new ArrayList<>(rowData.size());
+        for (Object each : rowData) {
+            result.add(new QueryResponseCell(Types.CHAR, each));
+        }
+        return new QueryResponseRow(result);
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/QueryableGlobalRuleRALBackendHandlerFactory.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/QueryableGlobalRuleRALBackendHandlerFactory.java
new file mode 100644
index 00000000000..20e802f6b8b
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/QueryableGlobalRuleRALBackendHandlerFactory.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.proxy.backend.handler.distsql.ral;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableGlobalRuleRALStatement;
+import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
+import org.apache.shardingsphere.infra.distsql.query.GlobalRuleDistSQLResultSet;
+import org.apache.shardingsphere.infra.util.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPIRegistry;
+import org.apache.shardingsphere.proxy.backend.handler.ProxyBackendHandler;
+
+import java.util.Properties;
+
+/**
+ * Queryable RAL backend handler factory for global rule.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class QueryableGlobalRuleRALBackendHandlerFactory {
+    
+    static {
+        ShardingSphereServiceLoader.register(DistSQLResultSet.class);
+    }
+    
+    /**
+     * Create new instance of queryable RAL backend handler for global rule.
+     *
+     * @param sqlStatement queryable RAL statement for global rule
+     * @return created instance
+     */
+    public static ProxyBackendHandler newInstance(final QueryableGlobalRuleRALStatement sqlStatement) {
+        DistSQLResultSet resultSet = TypedSPIRegistry.getRegisteredService(DistSQLResultSet.class, sqlStatement.getClass().getCanonicalName(), new Properties());
+        return new QueryableGlobalRuleRALBackendHandler(sqlStatement, (GlobalRuleDistSQLResultSet) resultSet);
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/RALBackendHandlerFactory.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/RALBackendHandlerFactory.java
index 769e74352ac..1f5f5684820 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/RALBackendHandlerFactory.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/RALBackendHandlerFactory.java
@@ -20,11 +20,11 @@ package org.apache.shardingsphere.proxy.backend.handler.distsql.ral;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.HintRALStatement;
+import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableGlobalRuleRALStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.RALStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ConvertYamlConfigurationStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ExportDatabaseConfigurationStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowAllVariableStatement;
-import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowAuthorityRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowInstanceInfoStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowInstanceListStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowModeInfoStatement;
@@ -54,10 +54,12 @@ import org.apache.shardingsphere.infra.distsql.query.DatabaseDistSQLResultSet;
 import org.apache.shardingsphere.infra.distsql.query.DistSQLResultSet;
 import org.apache.shardingsphere.proxy.backend.handler.ProxyBackendHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.hint.HintRALBackendHandler;
+import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.migration.query.QueryableMigrationRALBackendHandler;
+import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.migration.query.QueryableMigrationRALBackendHandlerFactory;
+import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.migration.update.UpdatableMigrationRALBackendHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ConvertYamlConfigurationHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ExportDatabaseConfigurationHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowAllVariableHandler;
-import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowAuthorityRuleHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowInstanceInfoHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowInstanceListHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowModeInfoHandler;
@@ -68,9 +70,6 @@ import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.Sho
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowTrafficRulesHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowTransactionRuleHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowVariableHandler;
-import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.migration.query.QueryableMigrationRALBackendHandler;
-import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.migration.query.QueryableMigrationRALBackendHandlerFactory;
-import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.migration.update.UpdatableMigrationRALBackendHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.AlterSQLParserRuleHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.AlterTrafficRuleHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.AlterTransactionRuleHandler;
@@ -122,7 +121,6 @@ public final class RALBackendHandlerFactory {
         HANDLERS.put(ShowVariableStatement.class, ShowVariableHandler.class);
         HANDLERS.put(ShowAllVariableStatement.class, ShowAllVariableHandler.class);
         HANDLERS.put(ShowReadwriteSplittingReadResourcesStatement.class, ShowReadwriteSplittingReadResourcesHandler.class);
-        HANDLERS.put(ShowAuthorityRuleStatement.class, ShowAuthorityRuleHandler.class);
         HANDLERS.put(ShowSQLParserRuleStatement.class, ShowSQLParserRuleHandler.class);
         HANDLERS.put(ShowTableMetadataStatement.class, ShowTableMetadataHandler.class);
         HANDLERS.put(ShowTrafficRulesStatement.class, ShowTrafficRulesHandler.class);
@@ -153,6 +151,9 @@ public final class RALBackendHandlerFactory {
         if (sqlStatement instanceof UpdatableScalingRALStatement) {
             return new UpdatableMigrationRALBackendHandler((UpdatableScalingRALStatement) sqlStatement);
         }
+        if (sqlStatement instanceof QueryableGlobalRuleRALStatement) {
+            return QueryableGlobalRuleRALBackendHandlerFactory.newInstance((QueryableGlobalRuleRALStatement) sqlStatement);
+        }
         return createRALBackendHandler(sqlStatement, connectionSession);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowAuthorityRuleHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowAuthorityRuleHandler.java
deleted file mode 100644
index b4c73e03eac..00000000000
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowAuthorityRuleHandler.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.handler.distsql.ral.queryable;
-
-import org.apache.shardingsphere.authority.config.AuthorityRuleConfiguration;
-import org.apache.shardingsphere.authority.rule.AuthorityRule;
-import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowAuthorityRuleStatement;
-import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
-import org.apache.shardingsphere.mode.manager.ContextManager;
-import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.QueryableRALBackendHandler;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.stream.Collectors;
-
-/**
- * Show authority rule handler.
- */
-public final class ShowAuthorityRuleHandler extends QueryableRALBackendHandler<ShowAuthorityRuleStatement> {
-    
-    private static final String USERS = "users";
-    
-    private static final String PROVIDER = "provider";
-    
-    private static final String PROPS = "props";
-    
-    @Override
-    protected Collection<String> getColumnNames() {
-        return Arrays.asList(USERS, PROVIDER, PROPS);
-    }
-    
-    @Override
-    protected Collection<LocalDataQueryResultRow> getRows(final ContextManager contextManager) {
-        AuthorityRule rule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData().getGlobalRuleMetaData().getSingleRule(AuthorityRule.class);
-        return Collections.singleton(getRow(rule.getConfiguration()));
-    }
-    
-    private LocalDataQueryResultRow getRow(final AuthorityRuleConfiguration authorityRuleConfig) {
-        return new LocalDataQueryResultRow(authorityRuleConfig.getUsers().stream().map(each -> each.getGrantee().toString()).collect(Collectors.joining("; ")),
-                authorityRuleConfig.getProvider().getType(), authorityRuleConfig.getProvider().getProps().size() == 0 ? "" : authorityRuleConfig.getProvider().getProps());
-    }
-}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowAuthorityRuleHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowAuthorityRuleHandlerTest.java
deleted file mode 100644
index 3ff1f3782b8..00000000000
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowAuthorityRuleHandlerTest.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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.handler.distsql.ral.queryable;
-
-import org.apache.shardingsphere.authority.config.AuthorityRuleConfiguration;
-import org.apache.shardingsphere.authority.rule.AuthorityRule;
-import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowAuthorityRuleStatement;
-import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
-import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
-import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
-import org.apache.shardingsphere.mode.manager.ContextManager;
-import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.proxy.backend.util.ProxyContextRestorer;
-import org.junit.Test;
-
-import java.sql.SQLException;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public final class ShowAuthorityRuleHandlerTest extends ProxyContextRestorer {
-    
-    @Test
-    public void assertAuthorityRule() throws SQLException {
-        ShowAuthorityRuleHandler handler = new ShowAuthorityRuleHandler();
-        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
-        when(contextManager.getMetaDataContexts().getMetaData().getGlobalRuleMetaData()).thenReturn(createGlobalRuleMetaData());
-        ProxyContext.init(contextManager);
-        handler.init(new ShowAuthorityRuleStatement(), null);
-        handler.execute();
-        handler.next();
-        List<Object> data = handler.getRowData().getData();
-        assertThat(data.size(), is(3));
-        assertThat(data.get(0), is("root@localhost"));
-        assertThat(data.get(1), is("ALL_PERMITTED"));
-        assertThat(data.get(2), is(""));
-    }
-    
-    private ShardingSphereRuleMetaData createGlobalRuleMetaData() {
-        ShardingSphereUser root = new ShardingSphereUser("root", "", "localhost");
-        AuthorityRuleConfiguration ruleConfig = new AuthorityRuleConfiguration(Collections.singleton(root), new AlgorithmConfiguration("ALL_PERMITTED", new Properties()));
-        return new ShardingSphereRuleMetaData(Collections.singleton(new AuthorityRule(ruleConfig, Collections.emptyMap())));
-    }
-}
diff --git a/shardingsphere-test/shardingsphere-parser-test/pom.xml b/shardingsphere-test/shardingsphere-parser-test/pom.xml
index 1e64f9830f4..1b71bb31fa0 100644
--- a/shardingsphere-test/shardingsphere-parser-test/pom.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/pom.xml
@@ -72,6 +72,11 @@
             <artifactId>shardingsphere-encrypt-distsql-parser</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-authority-distsql-parser</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
             <artifactId>shardingsphere-shadow-distsql-parser</artifactId>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/QueryableRALStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/QueryableRALStatementAssert.java
index b5bb1b8ca26..241bc7633f9 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/QueryableRALStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/QueryableRALStatementAssert.java
@@ -19,10 +19,10 @@ package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statemen
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.authority.distsql.parser.statement.ShowAuthorityRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ExportDatabaseConfigurationStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowAllVariableStatement;
-import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowAuthorityRuleStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowInstanceInfoStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowInstanceListStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowModeInfoStatement;
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/queryable/ShowAuthorityRuleStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/queryable/ShowAuthorityRuleStatementAssert.java
index 167a33a1957..28b39a97c9b 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/queryable/ShowAuthorityRuleStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/queryable/ShowAuthorityRuleStatementAssert.java
@@ -17,10 +17,13 @@
 
 package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.queryable;
 
-import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowAuthorityRuleStatement;
+import org.apache.shardingsphere.authority.distsql.parser.statement.ShowAuthorityRuleStatement;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ShowAuthorityRuleStatementTestCase;
 
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
 /**
  * Show authority rule statement assert.
  */
@@ -34,5 +37,10 @@ public final class ShowAuthorityRuleStatementAssert {
      * @param expected expected show authority rule statement test case
      */
     public static void assertIs(final SQLCaseAssertContext assertContext, final ShowAuthorityRuleStatement actual, final ShowAuthorityRuleStatementTestCase expected) {
+        if (null == expected) {
+            assertNull(assertContext.getText("Actual statement should not exist."), actual);
+        } else {
+            assertNotNull(assertContext.getText("Actual statement should exist."), actual);
+        }
     }
 }