You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2023/06/13 02:47:40 UTC

[shardingsphere] branch master updated: Add DistSQL for sql federation. (#26312)

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

duanzhengqiang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new 67f3a418d00 Add DistSQL for sql federation. (#26312)
67f3a418d00 is described below

commit 67f3a418d00c31971011eef0388f14c37e7a41ae
Author: Raigor <ra...@gmail.com>
AuthorDate: Tue Jun 13 10:47:32 2023 +0800

    Add DistSQL for sql federation. (#26312)
    
    * Add DistSQL for sql federation.
    
    * change test method name.
    
    * Update comment.
---
 .../sql-federation/{ => distsql/handler}/pom.xml   | 31 ++++++--
 .../query/ShowSQLFederationRuleExecutor.java       | 52 +++++++++++++
 .../AlterSQLFederationRuleStatementUpdater.java    | 59 ++++++++++++++
 ....distsql.handler.ral.query.QueryableRALExecutor | 18 +++++
 ...distsql.handler.ral.update.GlobalRuleRALUpdater | 18 +++++
 .../query/ShowSQLFederationRuleExecutorTest.java   | 70 +++++++++++++++++
 ...AlterSQLFederationRuleStatementUpdaterTest.java | 45 +++++++++++
 kernel/sql-federation/distsql/parser/pom.xml       | 91 ++++++++++++++++++++++
 .../antlr4/imports/sql-federation/Alphabet.g4}     | 74 ++++++++----------
 .../main/antlr4/imports/sql-federation/Keyword.g4} | 48 ++++++------
 .../antlr4/imports/sql-federation/Literals.g4}     | 46 +++--------
 .../antlr4/imports/sql-federation}/RALStatement.g4 | 26 +++----
 .../main/antlr4/imports/sql-federation/Symbol.g4   | 62 +++++++++++++++
 .../autogen/SQLFederationDistSQLStatement.g4}      | 46 ++---------
 .../parser/core/SQLFederationDistSQLLexer.java}    | 54 ++++---------
 .../parser/core/SQLFederationDistSQLParser.java    | 40 ++++++++++
 .../core/SQLFederationDistSQLStatementVisitor.java | 66 ++++++++++++++++
 .../SQLFederationDistSQLStatementParserFacade.java | 53 +++++++++++++
 ...engine.spi.FeaturedDistSQLStatementParserFacade | 18 +++++
 kernel/sql-federation/{ => distsql}/pom.xml        |  9 ++-
 .../sql-federation/{ => distsql/statement}/pom.xml | 16 ++--
 .../distsql/segment/CacheOptionSegment.java}       | 56 ++++---------
 .../queryable/ShowSQLFederationRuleStatement.java} | 48 ++----------
 .../AlterSQLFederationRuleStatement.java}          | 57 ++++----------
 kernel/sql-federation/pom.xml                      |  1 +
 .../update/AlterSQLParserRuleStatementUpdater.java | 13 ++--
 .../src/main/antlr4/imports/parser/RALStatement.g4 |  2 +-
 proxy/backend/core/pom.xml                         |  5 ++
 28 files changed, 779 insertions(+), 345 deletions(-)

diff --git a/kernel/sql-federation/pom.xml b/kernel/sql-federation/distsql/handler/pom.xml
similarity index 54%
copy from kernel/sql-federation/pom.xml
copy to kernel/sql-federation/distsql/handler/pom.xml
index 9cb483d3b7b..abcf208b578 100644
--- a/kernel/sql-federation/pom.xml
+++ b/kernel/sql-federation/distsql/handler/pom.xml
@@ -21,15 +21,32 @@
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache.shardingsphere</groupId>
-        <artifactId>shardingsphere-kernel</artifactId>
+        <artifactId>shardingsphere-sql-federation-distsql</artifactId>
         <version>5.3.3-SNAPSHOT</version>
     </parent>
-    <artifactId>shardingsphere-sql-federation</artifactId>
-    <packaging>pom</packaging>
+    <artifactId>shardingsphere-sql-federation-distsql-handler</artifactId>
     <name>${project.artifactId}</name>
     
-    <modules>
-        <module>api</module>
-        <module>core</module>
-    </modules>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-infra-distsql-handler</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-sql-federation-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-sql-federation-distsql-statement</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-sql-federation-distsql-parser</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
 </project>
diff --git a/kernel/sql-federation/distsql/handler/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/handler/query/ShowSQLFederationRuleExecutor.java b/kernel/sql-federation/distsql/handler/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/handler/query/ShowSQLFederationRuleExecutor.java
new file mode 100644
index 00000000000..ec72d5025da
--- /dev/null
+++ b/kernel/sql-federation/distsql/handler/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/handler/query/ShowSQLFederationRuleExecutor.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.sqlfederation.distsql.handler.query;
+
+import org.apache.shardingsphere.distsql.handler.ral.query.MetaDataRequiredQueryableRALExecutor;
+import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
+import org.apache.shardingsphere.sqlfederation.distsql.statement.queryable.ShowSQLFederationRuleStatement;
+import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Show SQL federation rule executor.
+ */
+public final class ShowSQLFederationRuleExecutor implements MetaDataRequiredQueryableRALExecutor<ShowSQLFederationRuleStatement> {
+    
+    @Override
+    public Collection<LocalDataQueryResultRow> getRows(final ShardingSphereMetaData metaData, final ShowSQLFederationRuleStatement sqlStatement) {
+        SQLFederationRuleConfiguration ruleConfig = metaData.getGlobalRuleMetaData().getSingleRule(SQLFederationRule.class).getConfiguration();
+        return Collections.singleton(new LocalDataQueryResultRow(String.valueOf(ruleConfig.isSqlFederationEnabled()),
+                null != ruleConfig.getExecutionPlanCache() ? ruleConfig.getExecutionPlanCache().toString() : ""));
+    }
+    
+    @Override
+    public Collection<String> getColumnNames() {
+        return Arrays.asList("sql_federation_enabled", "execution_plan_cache");
+    }
+    
+    @Override
+    public String getType() {
+        return ShowSQLFederationRuleStatement.class.getName();
+    }
+}
diff --git a/kernel/sql-federation/distsql/handler/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/handler/update/AlterSQLFederationRuleStatementUpdater.java b/kernel/sql-federation/distsql/handler/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/handler/update/AlterSQLFederationRuleStatementUpdater.java
new file mode 100644
index 00000000000..888e2357f3b
--- /dev/null
+++ b/kernel/sql-federation/distsql/handler/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/handler/update/AlterSQLFederationRuleStatementUpdater.java
@@ -0,0 +1,59 @@
+/*
+ * 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.sqlfederation.distsql.handler.update;
+
+import org.apache.shardingsphere.distsql.handler.ral.update.GlobalRuleRALUpdater;
+import org.apache.shardingsphere.sql.parser.api.CacheOption;
+import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
+import org.apache.shardingsphere.sqlfederation.distsql.segment.CacheOptionSegment;
+import org.apache.shardingsphere.sqlfederation.distsql.statement.updatable.AlterSQLFederationRuleStatement;
+
+/**
+ * Alter SQL federation rule statement updater.
+ */
+public final class AlterSQLFederationRuleStatementUpdater implements GlobalRuleRALUpdater<AlterSQLFederationRuleStatement, SQLFederationRuleConfiguration> {
+    
+    @Override
+    public void checkSQLStatement(final SQLFederationRuleConfiguration currentRuleConfig, final AlterSQLFederationRuleStatement sqlStatement) {
+    }
+    
+    @Override
+    public SQLFederationRuleConfiguration buildAlteredRuleConfiguration(final SQLFederationRuleConfiguration currentRuleConfig, final AlterSQLFederationRuleStatement sqlStatement) {
+        boolean sqlFederationEnabled = null == sqlStatement.getSqlFederationEnabled() ? currentRuleConfig.isSqlFederationEnabled() : sqlStatement.getSqlFederationEnabled();
+        CacheOption executionPlanCache = null == sqlStatement.getExecutionPlanCache()
+                ? currentRuleConfig.getExecutionPlanCache()
+                : createCacheOption(currentRuleConfig.getExecutionPlanCache(), sqlStatement.getExecutionPlanCache());
+        return new SQLFederationRuleConfiguration(sqlFederationEnabled, executionPlanCache);
+    }
+    
+    private CacheOption createCacheOption(final CacheOption cacheOption, final CacheOptionSegment segment) {
+        int initialCapacity = null == segment.getInitialCapacity() ? cacheOption.getInitialCapacity() : segment.getInitialCapacity();
+        long maximumSize = null == segment.getMaximumSize() ? cacheOption.getMaximumSize() : segment.getMaximumSize();
+        return new CacheOption(initialCapacity, maximumSize);
+    }
+    
+    @Override
+    public Class<SQLFederationRuleConfiguration> getRuleConfigurationClass() {
+        return SQLFederationRuleConfiguration.class;
+    }
+    
+    @Override
+    public String getType() {
+        return AlterSQLFederationRuleStatement.class.getName();
+    }
+}
diff --git a/kernel/sql-federation/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.query.QueryableRALExecutor b/kernel/sql-federation/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.query.QueryableRALExecutor
new file mode 100644
index 00000000000..0042c60c9da
--- /dev/null
+++ b/kernel/sql-federation/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.query.QueryableRALExecutor
@@ -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.sqlfederation.distsql.handler.query.ShowSQLFederationRuleExecutor
diff --git a/kernel/sql-federation/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.update.GlobalRuleRALUpdater b/kernel/sql-federation/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.update.GlobalRuleRALUpdater
new file mode 100644
index 00000000000..6b2edd1ce45
--- /dev/null
+++ b/kernel/sql-federation/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.update.GlobalRuleRALUpdater
@@ -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.sqlfederation.distsql.handler.update.AlterSQLFederationRuleStatementUpdater
diff --git a/kernel/sql-federation/distsql/handler/src/test/java/org/apache/shardingsphere/sqlfederation/distsql/handler/query/ShowSQLFederationRuleExecutorTest.java b/kernel/sql-federation/distsql/handler/src/test/java/org/apache/shardingsphere/sqlfederation/distsql/handler/query/ShowSQLFederationRuleExecutorTest.java
new file mode 100644
index 00000000000..83cc1f0986c
--- /dev/null
+++ b/kernel/sql-federation/distsql/handler/src/test/java/org/apache/shardingsphere/sqlfederation/distsql/handler/query/ShowSQLFederationRuleExecutorTest.java
@@ -0,0 +1,70 @@
+/*
+ * 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.sqlfederation.distsql.handler.query;
+
+import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.sql.parser.api.CacheOption;
+import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
+import org.apache.shardingsphere.sqlfederation.distsql.statement.queryable.ShowSQLFederationRuleStatement;
+import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+class ShowSQLFederationRuleExecutorTest {
+    
+    @Test
+    void assertSQLFederationRule() {
+        ShardingSphereMetaData metaData = mockMetaData();
+        ShowSQLFederationRuleExecutor executor = new ShowSQLFederationRuleExecutor();
+        Collection<LocalDataQueryResultRow> actual = executor.getRows(metaData, mock(ShowSQLFederationRuleStatement.class));
+        assertThat(actual.size(), is(1));
+        Iterator<LocalDataQueryResultRow> iterator = actual.iterator();
+        LocalDataQueryResultRow row = iterator.next();
+        assertThat(row.getCell(1), is("true"));
+        assertThat(row.getCell(2), is("initialCapacity: 2000, maximumSize: 65535"));
+    }
+    
+    @Test
+    void assertGetColumnNames() {
+        ShowSQLFederationRuleExecutor executor = new ShowSQLFederationRuleExecutor();
+        Collection<String> columns = executor.getColumnNames();
+        assertThat(columns.size(), is(2));
+        Iterator<String> iterator = columns.iterator();
+        assertThat(iterator.next(), is("sql_federation_enabled"));
+        assertThat(iterator.next(), is("execution_plan_cache"));
+    }
+    
+    private ShardingSphereMetaData mockMetaData() {
+        SQLFederationRule sqlFederationRule = mock(SQLFederationRule.class);
+        when(sqlFederationRule.getConfiguration()).thenReturn(new SQLFederationRuleConfiguration(true, new CacheOption(2000, 65535L)));
+        return new ShardingSphereMetaData(new LinkedHashMap<>(), new ShardingSphereRuleMetaData(Collections.singleton(sqlFederationRule)), new ConfigurationProperties(new Properties()));
+    }
+}
diff --git a/kernel/sql-federation/distsql/handler/src/test/java/org/apache/shardingsphere/sqlfederation/distsql/handler/update/AlterSQLFederationRuleStatementUpdaterTest.java b/kernel/sql-federation/distsql/handler/src/test/java/org/apache/shardingsphere/sqlfederation/distsql/handler/update/AlterSQLFederationRuleStatementUpdaterTest.java
new file mode 100644
index 00000000000..93d092e4195
--- /dev/null
+++ b/kernel/sql-federation/distsql/handler/src/test/java/org/apache/shardingsphere/sqlfederation/distsql/handler/update/AlterSQLFederationRuleStatementUpdaterTest.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sqlfederation.distsql.handler.update;
+
+import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
+import org.apache.shardingsphere.sqlfederation.distsql.segment.CacheOptionSegment;
+import org.apache.shardingsphere.sqlfederation.distsql.statement.updatable.AlterSQLFederationRuleStatement;
+import org.apache.shardingsphere.sqlfederation.rule.builder.DefaultSQLFederationRuleConfigurationBuilder;
+import org.junit.jupiter.api.Test;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class AlterSQLFederationRuleStatementUpdaterTest {
+    
+    @Test
+    void assertExecute() {
+        AlterSQLFederationRuleStatementUpdater updater = new AlterSQLFederationRuleStatementUpdater();
+        AlterSQLFederationRuleStatement sqlStatement = new AlterSQLFederationRuleStatement(true, new CacheOptionSegment(64, 512L));
+        SQLFederationRuleConfiguration actual = updater.buildAlteredRuleConfiguration(getSQLFederationRuleConfiguration(), sqlStatement);
+        assertTrue(actual.isSqlFederationEnabled());
+        assertThat(actual.getExecutionPlanCache().getInitialCapacity(), is(64));
+        assertThat(actual.getExecutionPlanCache().getMaximumSize(), is(512L));
+    }
+    
+    private SQLFederationRuleConfiguration getSQLFederationRuleConfiguration() {
+        return new DefaultSQLFederationRuleConfigurationBuilder().build();
+    }
+}
diff --git a/kernel/sql-federation/distsql/parser/pom.xml b/kernel/sql-federation/distsql/parser/pom.xml
new file mode 100644
index 00000000000..90578a1b122
--- /dev/null
+++ b/kernel/sql-federation/distsql/parser/pom.xml
@@ -0,0 +1,91 @@
+<?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-sql-federation-distsql</artifactId>
+        <version>5.3.3-SNAPSHOT</version>
+    </parent>
+    <artifactId>shardingsphere-sql-federation-distsql-parser</artifactId>
+    <name>${project.artifactId}</name>
+    
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-parser-sql-spi</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-parser-distsql-engine</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-sql-federation-distsql-statement</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+    
+    <profiles>
+        <profile>
+            <id>jdk8</id>
+            <activation>
+                <jdk>1.8</jdk>
+            </activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+        <profile>
+            <id>jdk11+</id>
+            <activation>
+                <jdk>[11,)</jdk>
+            </activation>
+            <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/sql-federation/</libDirectory>
+                                    <listener>false</listener>
+                                    <visitor>true</visitor>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+</project>
diff --git a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4 b/kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/Alphabet.g4
similarity index 51%
copy from kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
copy to kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/Alphabet.g4
index 3fb3f1aa1da..f7603cc7e63 100644
--- a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
+++ b/kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/Alphabet.g4
@@ -15,46 +15,34 @@
  * limitations under the License.
  */
 
-grammar RALStatement;
-
-import Keyword, Literals;
-
-showSQLParserRule
-    : SHOW SQL_PARSER RULE
-    ;
-
-alterSQLParserRule
-    : ALTER SQL_PARSER RULE sqlParserRuleDefinition
-    ;
-
-sqlParserRuleDefinition
-    : LP_ commentDefinition? (COMMA_ parseTreeCacheDefinition)? (COMMA_ sqlStatementCacheDefinition)? RP_
-    ;
-
-commentDefinition
-    : SQL_COMMENT_PARSE_ENABLED EQ_ sqlCommentParseEnabled
-    ;
-
-parseTreeCacheDefinition
-    : PARSE_TREE_CACHE LP_ cacheOption RP_
-    ;
-
-sqlStatementCacheDefinition
-    : SQL_STATEMENT_CACHE LP_ cacheOption RP_
-    ;
-
-sqlCommentParseEnabled
-    : TRUE | FALSE
-    ;
-
-cacheOption
-    : (INITIAL_CAPACITY EQ_ initialCapacity)? (COMMA_? MAXIMUM_SIZE EQ_ maximumSize)?
-    ;
-
-initialCapacity
-    : INT_
-    ;
-
-maximumSize
-    : INT_
-    ;
+lexer grammar Alphabet;
+
+FOR_GENERATOR: 'DO NOT MATCH ANY THING, JUST FOR GENERATOR';
+
+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/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4 b/kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/Keyword.g4
similarity index 55%
copy from kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
copy to kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/Keyword.g4
index 3fb3f1aa1da..3c99fc9a5d0 100644
--- a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
+++ b/kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/Keyword.g4
@@ -15,46 +15,50 @@
  * limitations under the License.
  */
 
-grammar RALStatement;
+lexer grammar Keyword;
 
-import Keyword, Literals;
+import Alphabet;
 
-showSQLParserRule
-    : SHOW SQL_PARSER RULE
+WS
+    : [ \t\r\n] + ->skip
     ;
 
-alterSQLParserRule
-    : ALTER SQL_PARSER RULE sqlParserRuleDefinition
+SHOW
+    : S H O W
     ;
 
-sqlParserRuleDefinition
-    : LP_ commentDefinition? (COMMA_ parseTreeCacheDefinition)? (COMMA_ sqlStatementCacheDefinition)? RP_
+ALTER
+    : A L T E R
     ;
 
-commentDefinition
-    : SQL_COMMENT_PARSE_ENABLED EQ_ sqlCommentParseEnabled
+RULE
+    : R U L E
     ;
 
-parseTreeCacheDefinition
-    : PARSE_TREE_CACHE LP_ cacheOption RP_
+SQL_FEDERATION
+    : S Q L UL_ F E D E R A T I O N
     ;
 
-sqlStatementCacheDefinition
-    : SQL_STATEMENT_CACHE LP_ cacheOption RP_
+SQL_FEDERATION_ENABLED
+    : S Q L UL_ F E D E R A T I O N UL_ E N A B L E D
     ;
 
-sqlCommentParseEnabled
-    : TRUE | FALSE
+EXECUTION_PLAN_CACHE
+    : E X E C U T I O N UL_ P L A N UL_ C A C H E
     ;
 
-cacheOption
-    : (INITIAL_CAPACITY EQ_ initialCapacity)? (COMMA_? MAXIMUM_SIZE EQ_ maximumSize)?
+INITIAL_CAPACITY
+    : I N I T I A L UL_ C A P A C I T Y
     ;
 
-initialCapacity
-    : INT_
+MAXIMUM_SIZE
+    : M A X I M U M UL_ S I Z E
     ;
 
-maximumSize
-    : INT_
+TRUE
+    : T R U E
+    ;
+
+FALSE
+    : F A L S E
     ;
diff --git a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4 b/kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/Literals.g4
similarity index 52%
copy from kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
copy to kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/Literals.g4
index 3fb3f1aa1da..68f6a1b07ed 100644
--- a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
+++ b/kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/Literals.g4
@@ -15,46 +15,20 @@
  * limitations under the License.
  */
 
-grammar RALStatement;
+lexer grammar Literals;
 
-import Keyword, Literals;
+import Alphabet, Symbol;
 
-showSQLParserRule
-    : SHOW SQL_PARSER RULE
+IDENTIFIER_
+    : [A-Za-z_$0-9]*?[A-Za-z_$]+?[A-Za-z_$0-9]*
+    | BQ_ ~'`'+ BQ_
     ;
 
-alterSQLParserRule
-    : ALTER SQL_PARSER RULE sqlParserRuleDefinition
+STRING_
+    : (DQ_ ('\\'. | '""' | ~('"' | '\\'))* DQ_)
+    | (SQ_ ('\\'. | '\'\'' | ~('\'' | '\\'))* SQ_)
     ;
 
-sqlParserRuleDefinition
-    : LP_ commentDefinition? (COMMA_ parseTreeCacheDefinition)? (COMMA_ sqlStatementCacheDefinition)? RP_
-    ;
-
-commentDefinition
-    : SQL_COMMENT_PARSE_ENABLED EQ_ sqlCommentParseEnabled
-    ;
-
-parseTreeCacheDefinition
-    : PARSE_TREE_CACHE LP_ cacheOption RP_
-    ;
-
-sqlStatementCacheDefinition
-    : SQL_STATEMENT_CACHE LP_ cacheOption RP_
-    ;
-
-sqlCommentParseEnabled
-    : TRUE | FALSE
-    ;
-
-cacheOption
-    : (INITIAL_CAPACITY EQ_ initialCapacity)? (COMMA_? MAXIMUM_SIZE EQ_ maximumSize)?
-    ;
-
-initialCapacity
-    : INT_
-    ;
-
-maximumSize
-    : INT_
+INT_
+    : [0-9]+
     ;
diff --git a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4 b/kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/RALStatement.g4
similarity index 68%
copy from kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
copy to kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/RALStatement.g4
index 3fb3f1aa1da..14c7cc617a0 100644
--- a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
+++ b/kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/RALStatement.g4
@@ -19,31 +19,27 @@ grammar RALStatement;
 
 import Keyword, Literals;
 
-showSQLParserRule
-    : SHOW SQL_PARSER RULE
+showSQLFederationRule
+    : SHOW SQL_FEDERATION RULE
     ;
 
-alterSQLParserRule
-    : ALTER SQL_PARSER RULE sqlParserRuleDefinition
+alterSQLFederationRule
+    : ALTER SQL_FEDERATION RULE sqlFederationRuleDefinition
     ;
 
-sqlParserRuleDefinition
-    : LP_ commentDefinition? (COMMA_ parseTreeCacheDefinition)? (COMMA_ sqlStatementCacheDefinition)? RP_
+sqlFederationRuleDefinition
+    : LP_ sqlFederationEnabled? COMMA_ executionPlanCache RP_
     ;
 
-commentDefinition
-    : SQL_COMMENT_PARSE_ENABLED EQ_ sqlCommentParseEnabled
+sqlFederationEnabled
+    : SQL_FEDERATION_ENABLED EQ_ boolean_
     ;
 
-parseTreeCacheDefinition
-    : PARSE_TREE_CACHE LP_ cacheOption RP_
+executionPlanCache
+    : EXECUTION_PLAN_CACHE LP_ cacheOption RP_
     ;
 
-sqlStatementCacheDefinition
-    : SQL_STATEMENT_CACHE LP_ cacheOption RP_
-    ;
-
-sqlCommentParseEnabled
+boolean_
     : TRUE | FALSE
     ;
 
diff --git a/kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/Symbol.g4 b/kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/Symbol.g4
new file mode 100644
index 00000000000..61031e590dd
--- /dev/null
+++ b/kernel/sql-federation/distsql/parser/src/main/antlr4/imports/sql-federation/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/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4 b/kernel/sql-federation/distsql/parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/SQLFederationDistSQLStatement.g4
similarity index 51%
copy from kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
copy to kernel/sql-federation/distsql/parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/SQLFederationDistSQLStatement.g4
index 3fb3f1aa1da..d33eaeb60ed 100644
--- a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
+++ b/kernel/sql-federation/distsql/parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/SQLFederationDistSQLStatement.g4
@@ -15,46 +15,12 @@
  * limitations under the License.
  */
 
-grammar RALStatement;
+grammar SQLFederationDistSQLStatement;
 
-import Keyword, Literals;
+import Symbol, RALStatement;
 
-showSQLParserRule
-    : SHOW SQL_PARSER RULE
-    ;
-
-alterSQLParserRule
-    : ALTER SQL_PARSER RULE sqlParserRuleDefinition
-    ;
-
-sqlParserRuleDefinition
-    : LP_ commentDefinition? (COMMA_ parseTreeCacheDefinition)? (COMMA_ sqlStatementCacheDefinition)? RP_
-    ;
-
-commentDefinition
-    : SQL_COMMENT_PARSE_ENABLED EQ_ sqlCommentParseEnabled
-    ;
-
-parseTreeCacheDefinition
-    : PARSE_TREE_CACHE LP_ cacheOption RP_
-    ;
-
-sqlStatementCacheDefinition
-    : SQL_STATEMENT_CACHE LP_ cacheOption RP_
-    ;
-
-sqlCommentParseEnabled
-    : TRUE | FALSE
-    ;
-
-cacheOption
-    : (INITIAL_CAPACITY EQ_ initialCapacity)? (COMMA_? MAXIMUM_SIZE EQ_ maximumSize)?
-    ;
-
-initialCapacity
-    : INT_
-    ;
-
-maximumSize
-    : INT_
+execute
+    : (showSQLFederationRule
+    | alterSQLFederationRule
+    ) SEMI_? EOF
     ;
diff --git a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4 b/kernel/sql-federation/distsql/parser/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/parser/core/SQLFederationDistSQLLexer.java
similarity index 51%
copy from kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
copy to kernel/sql-federation/distsql/parser/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/parser/core/SQLFederationDistSQLLexer.java
index 3fb3f1aa1da..237551c2c36 100644
--- a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
+++ b/kernel/sql-federation/distsql/parser/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/parser/core/SQLFederationDistSQLLexer.java
@@ -15,46 +15,18 @@
  * limitations under the License.
  */
 
-grammar RALStatement;
+package org.apache.shardingsphere.sqlfederation.distsql.parser.core;
 
-import Keyword, Literals;
+import org.antlr.v4.runtime.CharStream;
+import org.apache.shardingsphere.distsql.parser.autogen.SQLFederationDistSQLStatementLexer;
+import org.apache.shardingsphere.sql.parser.api.parser.SQLLexer;
 
-showSQLParserRule
-    : SHOW SQL_PARSER RULE
-    ;
-
-alterSQLParserRule
-    : ALTER SQL_PARSER RULE sqlParserRuleDefinition
-    ;
-
-sqlParserRuleDefinition
-    : LP_ commentDefinition? (COMMA_ parseTreeCacheDefinition)? (COMMA_ sqlStatementCacheDefinition)? RP_
-    ;
-
-commentDefinition
-    : SQL_COMMENT_PARSE_ENABLED EQ_ sqlCommentParseEnabled
-    ;
-
-parseTreeCacheDefinition
-    : PARSE_TREE_CACHE LP_ cacheOption RP_
-    ;
-
-sqlStatementCacheDefinition
-    : SQL_STATEMENT_CACHE LP_ cacheOption RP_
-    ;
-
-sqlCommentParseEnabled
-    : TRUE | FALSE
-    ;
-
-cacheOption
-    : (INITIAL_CAPACITY EQ_ initialCapacity)? (COMMA_? MAXIMUM_SIZE EQ_ maximumSize)?
-    ;
-
-initialCapacity
-    : INT_
-    ;
-
-maximumSize
-    : INT_
-    ;
+/**
+ * SQL lexer for SQL federation DistSQL.
+ */
+public final class SQLFederationDistSQLLexer extends SQLFederationDistSQLStatementLexer implements SQLLexer {
+    
+    public SQLFederationDistSQLLexer(final CharStream input) {
+        super(input);
+    }
+}
diff --git a/kernel/sql-federation/distsql/parser/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/parser/core/SQLFederationDistSQLParser.java b/kernel/sql-federation/distsql/parser/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/parser/core/SQLFederationDistSQLParser.java
new file mode 100644
index 00000000000..5f9c8c41c0f
--- /dev/null
+++ b/kernel/sql-federation/distsql/parser/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/parser/core/SQLFederationDistSQLParser.java
@@ -0,0 +1,40 @@
+/*
+ * 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.sqlfederation.distsql.parser.core;
+
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.TokenStream;
+import org.apache.shardingsphere.distsql.parser.autogen.SQLFederationDistSQLStatementParser;
+import org.apache.shardingsphere.sql.parser.api.parser.SQLParser;
+import org.apache.shardingsphere.sql.parser.api.ASTNode;
+import org.apache.shardingsphere.sql.parser.core.ParseASTNode;
+
+/**
+ * SQL parser for SQL federation DistSQL.
+ */
+public final class SQLFederationDistSQLParser extends SQLFederationDistSQLStatementParser implements SQLParser {
+    
+    public SQLFederationDistSQLParser(final TokenStream input) {
+        super(input);
+    }
+    
+    @Override
+    public ASTNode parse() {
+        return new ParseASTNode(execute(), (CommonTokenStream) getTokenStream());
+    }
+}
diff --git a/kernel/sql-federation/distsql/parser/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/parser/core/SQLFederationDistSQLStatementVisitor.java b/kernel/sql-federation/distsql/parser/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/parser/core/SQLFederationDistSQLStatementVisitor.java
new file mode 100644
index 00000000000..adfd60efab8
--- /dev/null
+++ b/kernel/sql-federation/distsql/parser/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/parser/core/SQLFederationDistSQLStatementVisitor.java
@@ -0,0 +1,66 @@
+/*
+ * 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.sqlfederation.distsql.parser.core;
+
+import org.antlr.v4.runtime.tree.ParseTree;
+import org.apache.shardingsphere.distsql.parser.autogen.SQLFederationDistSQLStatementBaseVisitor;
+import org.apache.shardingsphere.distsql.parser.autogen.SQLFederationDistSQLStatementParser.AlterSQLFederationRuleContext;
+import org.apache.shardingsphere.distsql.parser.autogen.SQLFederationDistSQLStatementParser.CacheOptionContext;
+import org.apache.shardingsphere.distsql.parser.autogen.SQLFederationDistSQLStatementParser.ShowSQLFederationRuleContext;
+import org.apache.shardingsphere.distsql.parser.autogen.SQLFederationDistSQLStatementParser.SqlFederationRuleDefinitionContext;
+import org.apache.shardingsphere.sql.parser.api.ASTNode;
+import org.apache.shardingsphere.sql.parser.api.visitor.SQLVisitor;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+import org.apache.shardingsphere.sqlfederation.distsql.segment.CacheOptionSegment;
+import org.apache.shardingsphere.sqlfederation.distsql.statement.queryable.ShowSQLFederationRuleStatement;
+import org.apache.shardingsphere.sqlfederation.distsql.statement.updatable.AlterSQLFederationRuleStatement;
+
+/**
+ * SQL statement visitor for SQL federation DistSQL.
+ */
+public final class SQLFederationDistSQLStatementVisitor extends SQLFederationDistSQLStatementBaseVisitor<ASTNode> implements SQLVisitor<ASTNode> {
+    
+    @Override
+    public ASTNode visitShowSQLFederationRule(final ShowSQLFederationRuleContext ctx) {
+        return new ShowSQLFederationRuleStatement();
+    }
+    
+    @Override
+    public ASTNode visitAlterSQLFederationRule(final AlterSQLFederationRuleContext ctx) {
+        return super.visit(ctx.sqlFederationRuleDefinition());
+    }
+    
+    @Override
+    public ASTNode visitSqlFederationRuleDefinition(final SqlFederationRuleDefinitionContext ctx) {
+        Boolean sqlFederationEnabled = null == ctx.sqlFederationEnabled() ? null : Boolean.parseBoolean(getIdentifierValue(ctx.sqlFederationEnabled().boolean_()));
+        CacheOptionSegment executionPlanCache = null == ctx.executionPlanCache() ? null : visitCacheOption(ctx.executionPlanCache().cacheOption());
+        return new AlterSQLFederationRuleStatement(sqlFederationEnabled, executionPlanCache);
+    }
+    
+    @Override
+    public CacheOptionSegment visitCacheOption(final CacheOptionContext ctx) {
+        return new CacheOptionSegment(
+                null == ctx.initialCapacity() ? null : Integer.parseInt(getIdentifierValue(ctx.initialCapacity())),
+                null == ctx.maximumSize() ? null : Long.parseLong(getIdentifierValue(ctx.maximumSize())));
+    }
+    
+    private String getIdentifierValue(final ParseTree context) {
+        return null == context ? null : new IdentifierValue(context.getText()).getValue();
+    }
+    
+}
diff --git a/kernel/sql-federation/distsql/parser/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/parser/facade/SQLFederationDistSQLStatementParserFacade.java b/kernel/sql-federation/distsql/parser/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/parser/facade/SQLFederationDistSQLStatementParserFacade.java
new file mode 100644
index 00000000000..3f7ac9c7223
--- /dev/null
+++ b/kernel/sql-federation/distsql/parser/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/parser/facade/SQLFederationDistSQLStatementParserFacade.java
@@ -0,0 +1,53 @@
+/*
+ * 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.sqlfederation.distsql.parser.facade;
+
+import org.apache.shardingsphere.distsql.parser.engine.spi.FeaturedDistSQLStatementParserFacade;
+import org.apache.shardingsphere.sqlfederation.distsql.parser.core.SQLFederationDistSQLLexer;
+import org.apache.shardingsphere.sqlfederation.distsql.parser.core.SQLFederationDistSQLParser;
+import org.apache.shardingsphere.sqlfederation.distsql.parser.core.SQLFederationDistSQLStatementVisitor;
+import org.apache.shardingsphere.sql.parser.api.ASTNode;
+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 SQL federation DistSQL statement.
+ */
+public final class SQLFederationDistSQLStatementParserFacade implements FeaturedDistSQLStatementParserFacade {
+    
+    @Override
+    public Class<? extends SQLLexer> getLexerClass() {
+        return SQLFederationDistSQLLexer.class;
+    }
+    
+    @Override
+    public Class<? extends SQLParser> getParserClass() {
+        return SQLFederationDistSQLParser.class;
+    }
+    
+    @Override
+    public Class<? extends SQLVisitor<ASTNode>> getVisitorClass() {
+        return SQLFederationDistSQLStatementVisitor.class;
+    }
+    
+    @Override
+    public String getType() {
+        return "sql_federation";
+    }
+}
diff --git a/kernel/sql-federation/distsql/parser/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.parser.engine.spi.FeaturedDistSQLStatementParserFacade b/kernel/sql-federation/distsql/parser/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.parser.engine.spi.FeaturedDistSQLStatementParserFacade
new file mode 100644
index 00000000000..8a6b68181b6
--- /dev/null
+++ b/kernel/sql-federation/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.sqlfederation.distsql.parser.facade.SQLFederationDistSQLStatementParserFacade
diff --git a/kernel/sql-federation/pom.xml b/kernel/sql-federation/distsql/pom.xml
similarity index 85%
copy from kernel/sql-federation/pom.xml
copy to kernel/sql-federation/distsql/pom.xml
index 9cb483d3b7b..304eefaa7b2 100644
--- a/kernel/sql-federation/pom.xml
+++ b/kernel/sql-federation/distsql/pom.xml
@@ -21,15 +21,16 @@
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache.shardingsphere</groupId>
-        <artifactId>shardingsphere-kernel</artifactId>
+        <artifactId>shardingsphere-sql-federation</artifactId>
         <version>5.3.3-SNAPSHOT</version>
     </parent>
-    <artifactId>shardingsphere-sql-federation</artifactId>
+    <artifactId>shardingsphere-sql-federation-distsql</artifactId>
     <packaging>pom</packaging>
     <name>${project.artifactId}</name>
     
     <modules>
-        <module>api</module>
-        <module>core</module>
+        <module>statement</module>
+        <module>parser</module>
+        <module>handler</module>
     </modules>
 </project>
diff --git a/kernel/sql-federation/pom.xml b/kernel/sql-federation/distsql/statement/pom.xml
similarity index 75%
copy from kernel/sql-federation/pom.xml
copy to kernel/sql-federation/distsql/statement/pom.xml
index 9cb483d3b7b..c737690aaec 100644
--- a/kernel/sql-federation/pom.xml
+++ b/kernel/sql-federation/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-sql-federation-distsql</artifactId>
         <version>5.3.3-SNAPSHOT</version>
     </parent>
-    <artifactId>shardingsphere-sql-federation</artifactId>
-    <packaging>pom</packaging>
+    <artifactId>shardingsphere-sql-federation-distsql-statement</artifactId>
     <name>${project.artifactId}</name>
     
-    <modules>
-        <module>api</module>
-        <module>core</module>
-    </modules>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-parser-distsql-statement</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
 </project>
diff --git a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4 b/kernel/sql-federation/distsql/statement/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/segment/CacheOptionSegment.java
similarity index 51%
copy from kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
copy to kernel/sql-federation/distsql/statement/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/segment/CacheOptionSegment.java
index 3fb3f1aa1da..7f3a4ea1b59 100644
--- a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
+++ b/kernel/sql-federation/distsql/statement/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/segment/CacheOptionSegment.java
@@ -15,46 +15,20 @@
  * limitations under the License.
  */
 
-grammar RALStatement;
+package org.apache.shardingsphere.sqlfederation.distsql.segment;
 
-import Keyword, Literals;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.sql.parser.api.ASTNode;
 
-showSQLParserRule
-    : SHOW SQL_PARSER RULE
-    ;
-
-alterSQLParserRule
-    : ALTER SQL_PARSER RULE sqlParserRuleDefinition
-    ;
-
-sqlParserRuleDefinition
-    : LP_ commentDefinition? (COMMA_ parseTreeCacheDefinition)? (COMMA_ sqlStatementCacheDefinition)? RP_
-    ;
-
-commentDefinition
-    : SQL_COMMENT_PARSE_ENABLED EQ_ sqlCommentParseEnabled
-    ;
-
-parseTreeCacheDefinition
-    : PARSE_TREE_CACHE LP_ cacheOption RP_
-    ;
-
-sqlStatementCacheDefinition
-    : SQL_STATEMENT_CACHE LP_ cacheOption RP_
-    ;
-
-sqlCommentParseEnabled
-    : TRUE | FALSE
-    ;
-
-cacheOption
-    : (INITIAL_CAPACITY EQ_ initialCapacity)? (COMMA_? MAXIMUM_SIZE EQ_ maximumSize)?
-    ;
-
-initialCapacity
-    : INT_
-    ;
-
-maximumSize
-    : INT_
-    ;
+/**
+ * Cache option segment.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class CacheOptionSegment implements ASTNode {
+    
+    private final Integer initialCapacity;
+    
+    private final Long maximumSize;
+}
diff --git a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4 b/kernel/sql-federation/distsql/statement/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/statement/queryable/ShowSQLFederationRuleStatement.java
similarity index 51%
copy from kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
copy to kernel/sql-federation/distsql/statement/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/statement/queryable/ShowSQLFederationRuleStatement.java
index 3fb3f1aa1da..e09a1b5c3c3 100644
--- a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
+++ b/kernel/sql-federation/distsql/statement/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/statement/queryable/ShowSQLFederationRuleStatement.java
@@ -15,46 +15,12 @@
  * limitations under the License.
  */
 
-grammar RALStatement;
+package org.apache.shardingsphere.sqlfederation.distsql.statement.queryable;
 
-import Keyword, Literals;
+import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
 
-showSQLParserRule
-    : SHOW SQL_PARSER RULE
-    ;
-
-alterSQLParserRule
-    : ALTER SQL_PARSER RULE sqlParserRuleDefinition
-    ;
-
-sqlParserRuleDefinition
-    : LP_ commentDefinition? (COMMA_ parseTreeCacheDefinition)? (COMMA_ sqlStatementCacheDefinition)? RP_
-    ;
-
-commentDefinition
-    : SQL_COMMENT_PARSE_ENABLED EQ_ sqlCommentParseEnabled
-    ;
-
-parseTreeCacheDefinition
-    : PARSE_TREE_CACHE LP_ cacheOption RP_
-    ;
-
-sqlStatementCacheDefinition
-    : SQL_STATEMENT_CACHE LP_ cacheOption RP_
-    ;
-
-sqlCommentParseEnabled
-    : TRUE | FALSE
-    ;
-
-cacheOption
-    : (INITIAL_CAPACITY EQ_ initialCapacity)? (COMMA_? MAXIMUM_SIZE EQ_ maximumSize)?
-    ;
-
-initialCapacity
-    : INT_
-    ;
-
-maximumSize
-    : INT_
-    ;
+/**
+ * Show SQL federation rule statement.
+ */
+public final class ShowSQLFederationRuleStatement extends QueryableRALStatement {
+}
diff --git a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4 b/kernel/sql-federation/distsql/statement/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/statement/updatable/AlterSQLFederationRuleStatement.java
similarity index 51%
copy from kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
copy to kernel/sql-federation/distsql/statement/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/statement/updatable/AlterSQLFederationRuleStatement.java
index 3fb3f1aa1da..e93154f08d5 100644
--- a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
+++ b/kernel/sql-federation/distsql/statement/src/main/java/org/apache/shardingsphere/sqlfederation/distsql/statement/updatable/AlterSQLFederationRuleStatement.java
@@ -15,46 +15,21 @@
  * limitations under the License.
  */
 
-grammar RALStatement;
+package org.apache.shardingsphere.sqlfederation.distsql.statement.updatable;
 
-import Keyword, Literals;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableGlobalRuleRALStatement;
+import org.apache.shardingsphere.sqlfederation.distsql.segment.CacheOptionSegment;
 
-showSQLParserRule
-    : SHOW SQL_PARSER RULE
-    ;
-
-alterSQLParserRule
-    : ALTER SQL_PARSER RULE sqlParserRuleDefinition
-    ;
-
-sqlParserRuleDefinition
-    : LP_ commentDefinition? (COMMA_ parseTreeCacheDefinition)? (COMMA_ sqlStatementCacheDefinition)? RP_
-    ;
-
-commentDefinition
-    : SQL_COMMENT_PARSE_ENABLED EQ_ sqlCommentParseEnabled
-    ;
-
-parseTreeCacheDefinition
-    : PARSE_TREE_CACHE LP_ cacheOption RP_
-    ;
-
-sqlStatementCacheDefinition
-    : SQL_STATEMENT_CACHE LP_ cacheOption RP_
-    ;
-
-sqlCommentParseEnabled
-    : TRUE | FALSE
-    ;
-
-cacheOption
-    : (INITIAL_CAPACITY EQ_ initialCapacity)? (COMMA_? MAXIMUM_SIZE EQ_ maximumSize)?
-    ;
-
-initialCapacity
-    : INT_
-    ;
-
-maximumSize
-    : INT_
-    ;
+/**
+ * Alter SQL federation rule statement.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterSQLFederationRuleStatement extends UpdatableGlobalRuleRALStatement {
+    
+    private final Boolean sqlFederationEnabled;
+    
+    private final CacheOptionSegment executionPlanCache;
+}
diff --git a/kernel/sql-federation/pom.xml b/kernel/sql-federation/pom.xml
index 9cb483d3b7b..cfb533242e1 100644
--- a/kernel/sql-federation/pom.xml
+++ b/kernel/sql-federation/pom.xml
@@ -31,5 +31,6 @@
     <modules>
         <module>api</module>
         <module>core</module>
+        <module>distsql</module>
     </modules>
 </project>
diff --git a/kernel/sql-parser/distsql/handler/src/main/java/org/apache/shardingsphere/parser/distsql/handler/update/AlterSQLParserRuleStatementUpdater.java b/kernel/sql-parser/distsql/handler/src/main/java/org/apache/shardingsphere/parser/distsql/handler/update/AlterSQLParserRuleStatementUpdater.java
index ed10148293b..86684ac7b24 100644
--- a/kernel/sql-parser/distsql/handler/src/main/java/org/apache/shardingsphere/parser/distsql/handler/update/AlterSQLParserRuleStatementUpdater.java
+++ b/kernel/sql-parser/distsql/handler/src/main/java/org/apache/shardingsphere/parser/distsql/handler/update/AlterSQLParserRuleStatementUpdater.java
@@ -24,7 +24,7 @@ import org.apache.shardingsphere.parser.distsql.parser.statement.updatable.Alter
 import org.apache.shardingsphere.sql.parser.api.CacheOption;
 
 /**
- * Alter SQL parser rule statement handler.
+ * Alter SQL parser rule statement updater.
  */
 public final class AlterSQLParserRuleStatementUpdater implements GlobalRuleRALUpdater<AlterSQLParserRuleStatement, SQLParserRuleConfiguration> {
     
@@ -35,11 +35,12 @@ public final class AlterSQLParserRuleStatementUpdater implements GlobalRuleRALUp
     @Override
     public SQLParserRuleConfiguration buildAlteredRuleConfiguration(final SQLParserRuleConfiguration currentRuleConfig, final AlterSQLParserRuleStatement sqlStatement) {
         boolean sqlCommentParseEnabled = null == sqlStatement.getSqlCommentParseEnabled() ? currentRuleConfig.isSqlCommentParseEnabled() : sqlStatement.getSqlCommentParseEnabled();
-        CacheOption parseTreeCache =
-                null == sqlStatement.getParseTreeCache() ? currentRuleConfig.getParseTreeCache() : createCacheOption(currentRuleConfig.getParseTreeCache(), sqlStatement.getParseTreeCache());
-        CacheOption sqlStatementCache =
-                null == sqlStatement.getSqlStatementCache() ? currentRuleConfig.getSqlStatementCache()
-                        : createCacheOption(currentRuleConfig.getSqlStatementCache(), sqlStatement.getSqlStatementCache());
+        CacheOption parseTreeCache = null == sqlStatement.getParseTreeCache()
+                ? currentRuleConfig.getParseTreeCache()
+                : createCacheOption(currentRuleConfig.getParseTreeCache(), sqlStatement.getParseTreeCache());
+        CacheOption sqlStatementCache = null == sqlStatement.getSqlStatementCache()
+                ? currentRuleConfig.getSqlStatementCache()
+                : createCacheOption(currentRuleConfig.getSqlStatementCache(), sqlStatement.getSqlStatementCache());
         return new SQLParserRuleConfiguration(sqlCommentParseEnabled, parseTreeCache, sqlStatementCache);
     }
     
diff --git a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4 b/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
index 3fb3f1aa1da..b69ea7edb39 100644
--- a/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
+++ b/kernel/sql-parser/distsql/parser/src/main/antlr4/imports/parser/RALStatement.g4
@@ -28,7 +28,7 @@ alterSQLParserRule
     ;
 
 sqlParserRuleDefinition
-    : LP_ commentDefinition? (COMMA_ parseTreeCacheDefinition)? (COMMA_ sqlStatementCacheDefinition)? RP_
+    : LP_ commentDefinition? (COMMA_? parseTreeCacheDefinition)? (COMMA_? sqlStatementCacheDefinition)? RP_
     ;
 
 commentDefinition
diff --git a/proxy/backend/core/pom.xml b/proxy/backend/core/pom.xml
index 37fe331fb75..05c6a9db44f 100644
--- a/proxy/backend/core/pom.xml
+++ b/proxy/backend/core/pom.xml
@@ -128,6 +128,11 @@
             <artifactId>shardingsphere-single-distsql-handler</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-sql-federation-distsql-handler</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
             <artifactId>shardingsphere-sql-parser-distsql-handler</artifactId>