You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2022/06/09 00:17:59 UTC

[shardingsphere] branch master updated: Refine federation.executor package structure (#18251)

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 dec5581af37 Refine federation.executor package structure (#18251)
dec5581af37 is described below

commit dec5581af372e1e7daa487800867265ef99bb07c
Author: Liang Zhang <zh...@apache.org>
AuthorDate: Thu Jun 9 08:17:54 2022 +0800

    Refine federation.executor package structure (#18251)
    
    * Rename FilterableFederationExecutor
    
    * Rename TranslatableFederationExecutor
    
    * Refactor FilterableTableScanExecutor
    
    * Rename package
    
    * Rename TranslatableExecuteDataContext
    
    * Add translatable.resultset package
    
    * Rename back AdvancedFederationExecutor
    
    * Rename back OriginalFederationExecutor
    
    * Refactor FederationTableStatistic
    
    * Move packages
    
    * Add SQLDialectFactory
    
    * Refactor FilterableTableScanExecutor
    
    * Fix compile error
    
    * For code format
    
    * For code format
---
 .../rule/EncryptColumnRuleConfiguration.java       |  2 +-
 .../encrypt/rule/EncryptRuleTest.java              |  8 +--
 .../AdvancedExecuteDataContext.java}               |  6 +-
 .../AdvancedFederationExecutor.java                |  5 +-
 .../AbstractUnsupportedOperationResultSet.java     |  2 +-
 ...bstractUnsupportedUpdateOperationResultSet.java |  4 +-
 .../resultset}/FederationResultSet.java            |  5 +-
 .../resultset}/WrapperAdapter.java                 |  2 +-
 .../original/OriginalFederationExecutor.java       |  7 +--
 .../executor/original/SQLDialectFactory.java       | 59 +++++++++++++++++++
 .../{ => database}/FilterableDatabase.java         |  3 +-
 .../original/{ => schema}/FilterableSchema.java    |  3 +-
 .../{ => table}/FederationTableStatistic.java      | 23 ++------
 .../executor/original/table/FilterableTable.java   |  1 -
 .../table/FilterableTableScanExecutor.java         | 68 ++++++----------------
 .../AdvancedFederationExecutorTest.java            |  2 +-
 .../sql/parser/sql/common/util/SQLUtil.java        |  8 +--
 17 files changed, 111 insertions(+), 97 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-api/src/main/java/org/apache/shardingsphere/encrypt/api/config/rule/EncryptColumnRuleConfiguration.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-api/src/main/java/org/apache/shardingsphere/encrypt/api/config/rule/EncryptColumnRuleConfiguration.java
index e9ff4f2f135..740c102764e 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-api/src/main/java/org/apache/shardingsphere/encrypt/api/config/rule/EncryptColumnRuleConfiguration.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-api/src/main/java/org/apache/shardingsphere/encrypt/api/config/rule/EncryptColumnRuleConfiguration.java
@@ -41,7 +41,7 @@ public final class EncryptColumnRuleConfiguration {
     
     private final Boolean queryWithCipherColumn;
     
-    public EncryptColumnRuleConfiguration(final String logicColumn, final String cipherColumn, final String assistedQueryColumn, final String plainColumn, 
+    public EncryptColumnRuleConfiguration(final String logicColumn, final String cipherColumn, final String assistedQueryColumn, final String plainColumn,
                                           final String encryptorName, final Boolean queryWithCipherColumn) {
         this(logicColumn, cipherColumn, assistedQueryColumn, plainColumn, encryptorName, null, queryWithCipherColumn);
     }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
index d478285ef3a..763f665d8ba 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
@@ -149,19 +149,19 @@ public final class EncryptRuleTest {
                 Collections.singleton(tableConfig), Collections.singletonMap("test_encryptor", new CoreEncryptAlgorithmFixture()), true);
         EncryptRule actual = new EncryptRule(ruleConfig);
         assertTrue(actual.isQueryWithCipherColumn("t_encrypt", "encrypt_column"));
-    
+        
         encryptColumnConfig = new EncryptColumnRuleConfiguration("encrypt_column", "encrypt_cipher", "", "", "test_encryptor", null);
         tableConfig = new EncryptTableRuleConfiguration("t_encrypt", Collections.singletonList(encryptColumnConfig), false);
         ruleConfig = new AlgorithmProvidedEncryptRuleConfiguration(Collections.singleton(tableConfig), Collections.singletonMap("test_encryptor", new CoreEncryptAlgorithmFixture()), true);
         actual = new EncryptRule(ruleConfig);
         assertFalse(actual.isQueryWithCipherColumn("t_encrypt", "encrypt_column"));
-    
+        
         encryptColumnConfig = new EncryptColumnRuleConfiguration("encrypt_column", "encrypt_cipher", "", "", "test_encryptor", true);
         tableConfig = new EncryptTableRuleConfiguration("t_encrypt", Collections.singletonList(encryptColumnConfig), false);
         ruleConfig = new AlgorithmProvidedEncryptRuleConfiguration(Collections.singleton(tableConfig), Collections.singletonMap("test_encryptor", new CoreEncryptAlgorithmFixture()), true);
         actual = new EncryptRule(ruleConfig);
         assertTrue(actual.isQueryWithCipherColumn("t_encrypt", "encrypt_column"));
-    
+        
         encryptColumnConfig = new EncryptColumnRuleConfiguration("encrypt_column", "encrypt_cipher", "", "", "test_encryptor", false);
         tableConfig = new EncryptTableRuleConfiguration("t_encrypt", Collections.singletonList(encryptColumnConfig), null);
         ruleConfig = new AlgorithmProvidedEncryptRuleConfiguration(Collections.singleton(tableConfig), Collections.singletonMap("test_encryptor", new CoreEncryptAlgorithmFixture()), true);
@@ -213,7 +213,7 @@ public final class EncryptRuleTest {
     @Test
     public void assertAssistedQueryEncryptorNameSpecified() {
         EncryptColumnRuleConfiguration pwdColumnConfig = new EncryptColumnRuleConfiguration("pwd", "pwd_cipher", "pwd_assist", "pwd_plain", "test_encryptor", "assisted_query_test_encryptor", null);
-        assertTrue(pwdColumnConfig.getAssistedQueryEncryptorName().equals("assisted_query_test_encryptor"));
+        assertThat(pwdColumnConfig.getAssistedQueryEncryptorName(), is("assisted_query_test_encryptor"));
     }
     
     private EncryptRuleConfiguration createEncryptRuleConfigurationWithUpperCaseLogicTable() {
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/CustomizedFilterableExecuteDataContext.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/AdvancedExecuteDataContext.java
similarity index 89%
rename from shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/CustomizedFilterableExecuteDataContext.java
rename to shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/AdvancedExecuteDataContext.java
index e920cd3f05d..f0e879f275a 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/CustomizedFilterableExecuteDataContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/AdvancedExecuteDataContext.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.federation.executor.customized;
+package org.apache.shardingsphere.infra.federation.executor.advanced;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.calcite.DataContext;
@@ -26,10 +26,10 @@ import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 
 /**
- * Customized filterable execute data context.
+ * Advanced execute data context.
  */
 @RequiredArgsConstructor
-public final class CustomizedFilterableExecuteDataContext implements DataContext {
+public final class AdvancedExecuteDataContext implements DataContext {
     
     private final SqlValidator validator;
     
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/AdvancedFederationExecutor.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/AdvancedFederationExecutor.java
similarity index 95%
rename from shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/AdvancedFederationExecutor.java
rename to shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/AdvancedFederationExecutor.java
index 7f835dc389e..07fe5b291e1 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/AdvancedFederationExecutor.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/AdvancedFederationExecutor.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.federation.executor.customized;
+package org.apache.shardingsphere.infra.federation.executor.advanced;
 
 import org.apache.calcite.interpreter.InterpretableConvention;
 import org.apache.calcite.interpreter.InterpretableConverter;
@@ -31,6 +31,7 @@ import org.apache.shardingsphere.infra.executor.sql.execute.result.ExecuteResult
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.DriverExecutionPrepareEngine;
 import org.apache.shardingsphere.infra.federation.executor.FederationContext;
 import org.apache.shardingsphere.infra.federation.executor.FederationExecutor;
+import org.apache.shardingsphere.infra.federation.executor.advanced.resultset.FederationResultSet;
 import org.apache.shardingsphere.infra.federation.optimizer.ShardingSphereOptimizer;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
@@ -88,7 +89,7 @@ public final class AdvancedFederationExecutor implements FederationExecutor {
         SqlValidator validator = optimizerContext.getPlannerContexts().get(databaseName).getValidators().get(schemaName);
         SqlToRelConverter converter = optimizerContext.getPlannerContexts().get(databaseName).getConverters().get(schemaName);
         return new FederateInterpretableConverter(
-                cluster, cluster.traitSetOf(InterpretableConvention.INSTANCE), bestPlan).bind(new CustomizedFilterableExecuteDataContext(validator, converter));
+                cluster, cluster.traitSetOf(InterpretableConvention.INSTANCE), bestPlan).bind(new AdvancedExecuteDataContext(validator, converter));
     }
     
     @Override
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/AbstractUnsupportedOperationResultSet.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/resultset/AbstractUnsupportedOperationResultSet.java
similarity index 98%
rename from shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/AbstractUnsupportedOperationResultSet.java
rename to shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/resultset/AbstractUnsupportedOperationResultSet.java
index c4ad246a279..24854ebd5f6 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/AbstractUnsupportedOperationResultSet.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/resultset/AbstractUnsupportedOperationResultSet.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.federation.executor.customized;
+package org.apache.shardingsphere.infra.federation.executor.advanced.resultset;
 
 import java.io.Reader;
 import java.sql.NClob;
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/AbstractUnsupportedUpdateOperationResultSet.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/resultset/AbstractUnsupportedUpdateOperationResultSet.java
similarity index 99%
rename from shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/AbstractUnsupportedUpdateOperationResultSet.java
rename to shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/resultset/AbstractUnsupportedUpdateOperationResultSet.java
index 7784d57c8ae..9374ef0e7ec 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/AbstractUnsupportedUpdateOperationResultSet.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/resultset/AbstractUnsupportedUpdateOperationResultSet.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.federation.executor.customized;
+package org.apache.shardingsphere.infra.federation.executor.advanced.resultset;
 
 import java.io.InputStream;
 import java.io.Reader;
@@ -35,7 +35,7 @@ import java.sql.Time;
 import java.sql.Timestamp;
 
 /**
- * Unsupported {@code ResultSet} methods.
+ * Unsupported {@code ResultSet} update methods.
  */
 public abstract class AbstractUnsupportedUpdateOperationResultSet extends WrapperAdapter implements ResultSet {
     
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/FederationResultSet.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/resultset/FederationResultSet.java
similarity index 98%
rename from shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/FederationResultSet.java
rename to shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/resultset/FederationResultSet.java
index 3be6a183310..8e00355d316 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/FederationResultSet.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/resultset/FederationResultSet.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.federation.executor.customized;
+package org.apache.shardingsphere.infra.federation.executor.advanced.resultset;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
@@ -38,6 +38,9 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.Calendar;
 
+/**
+ * Federation result set.
+ */
 @RequiredArgsConstructor
 public final class FederationResultSet extends AbstractUnsupportedOperationResultSet {
     
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/WrapperAdapter.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/resultset/WrapperAdapter.java
similarity index 94%
rename from shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/WrapperAdapter.java
rename to shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/resultset/WrapperAdapter.java
index c3eb60f2a54..f20c0ab82d7 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/customized/WrapperAdapter.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/advanced/resultset/WrapperAdapter.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.federation.executor.customized;
+package org.apache.shardingsphere.infra.federation.executor.advanced.resultset;
 
 import java.sql.SQLException;
 import java.sql.Wrapper;
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/OriginalFederationExecutor.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/OriginalFederationExecutor.java
index 1a9cda32256..4296b92ed90 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/OriginalFederationExecutor.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/OriginalFederationExecutor.java
@@ -19,7 +19,6 @@ package org.apache.shardingsphere.infra.federation.executor.original;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.calcite.jdbc.CalciteConnection;
-import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutor;
@@ -28,6 +27,7 @@ import org.apache.shardingsphere.infra.executor.sql.execute.result.ExecuteResult
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.DriverExecutionPrepareEngine;
 import org.apache.shardingsphere.infra.federation.executor.FederationContext;
 import org.apache.shardingsphere.infra.federation.executor.FederationExecutor;
+import org.apache.shardingsphere.infra.federation.executor.original.database.FilterableDatabase;
 import org.apache.shardingsphere.infra.federation.executor.original.table.FilterableTableScanExecutor;
 import org.apache.shardingsphere.infra.federation.executor.original.table.FilterableTableScanExecutorContext;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
@@ -74,10 +74,9 @@ public final class OriginalFederationExecutor implements FederationExecutor {
     @Override
     public ResultSet executeQuery(final DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine,
                                   final JDBCExecutorCallback<? extends ExecuteResult> callback, final FederationContext federationContext) throws SQLException {
-        LogicSQL logicSQL = federationContext.getLogicSQL();
         Connection connection = createConnection(prepareEngine, callback, federationContext);
-        PreparedStatement preparedStatement = connection.prepareStatement(SQLUtil.trimSemicolon(logicSQL.getSql()));
-        setParameters(preparedStatement, logicSQL.getParameters());
+        PreparedStatement preparedStatement = connection.prepareStatement(SQLUtil.trimSemicolon(federationContext.getLogicSQL().getSql()));
+        setParameters(preparedStatement, federationContext.getLogicSQL().getParameters());
         this.statement = preparedStatement;
         return preparedStatement.executeQuery();
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/SQLDialectFactory.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/SQLDialectFactory.java
new file mode 100644
index 00000000000..31f4d3a63c9
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/SQLDialectFactory.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.infra.federation.executor.original;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.dialect.MssqlSqlDialect;
+import org.apache.calcite.sql.dialect.MysqlSqlDialect;
+import org.apache.calcite.sql.dialect.OracleSqlDialect;
+import org.apache.calcite.sql.dialect.PostgresqlSqlDialect;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * SQL dialect factory.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class SQLDialectFactory {
+    
+    private static final Map<String, SqlDialect> SQL_DIALECTS_REGISTRY = new HashMap<>();
+    
+    static {
+        SQL_DIALECTS_REGISTRY.put("H2", MysqlSqlDialect.DEFAULT);
+        SQL_DIALECTS_REGISTRY.put("MySQL", MysqlSqlDialect.DEFAULT);
+        SQL_DIALECTS_REGISTRY.put("MariaDB", MysqlSqlDialect.DEFAULT);
+        SQL_DIALECTS_REGISTRY.put("Oracle", OracleSqlDialect.DEFAULT);
+        SQL_DIALECTS_REGISTRY.put("SQLServer", MssqlSqlDialect.DEFAULT);
+        SQL_DIALECTS_REGISTRY.put("PostgreSQL", PostgresqlSqlDialect.DEFAULT);
+        SQL_DIALECTS_REGISTRY.put("openGauss", PostgresqlSqlDialect.DEFAULT);
+    }
+    
+    /**
+     * Get SQL dialect.
+     * 
+     * @param databaseType database type
+     * @return SQL dialect
+     */
+    public static SqlDialect getSQLDialect(final DatabaseType databaseType) {
+        return SQL_DIALECTS_REGISTRY.getOrDefault(databaseType.getType(), MysqlSqlDialect.DEFAULT);
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/FilterableDatabase.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/database/FilterableDatabase.java
similarity index 95%
rename from shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/FilterableDatabase.java
rename to shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/database/FilterableDatabase.java
index 803c68a8bd6..5d51705a881 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/FilterableDatabase.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/database/FilterableDatabase.java
@@ -15,11 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.federation.executor.original;
+package org.apache.shardingsphere.infra.federation.executor.original.database;
 
 import lombok.Getter;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.shardingsphere.infra.federation.executor.original.schema.FilterableSchema;
 import org.apache.shardingsphere.infra.federation.executor.original.table.FilterableTableScanExecutor;
 import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationDatabaseMetaData;
 import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationSchemaMetaData;
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/FilterableSchema.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/schema/FilterableSchema.java
similarity index 95%
rename from shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/FilterableSchema.java
rename to shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/schema/FilterableSchema.java
index 80a3c540785..318823b777b 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/FilterableSchema.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/schema/FilterableSchema.java
@@ -15,11 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.federation.executor.original;
+package org.apache.shardingsphere.infra.federation.executor.original.schema;
 
 import lombok.Getter;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.shardingsphere.infra.federation.executor.original.table.FederationTableStatistic;
 import org.apache.shardingsphere.infra.federation.executor.original.table.FilterableTable;
 import org.apache.shardingsphere.infra.federation.executor.original.table.FilterableTableScanExecutor;
 import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationSchemaMetaData;
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/FederationTableStatistic.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/table/FederationTableStatistic.java
similarity index 80%
rename from shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/FederationTableStatistic.java
rename to shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/table/FederationTableStatistic.java
index 651165b6c01..b80fab8bb54 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/FederationTableStatistic.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/table/FederationTableStatistic.java
@@ -15,37 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.federation.executor.original;
+package org.apache.shardingsphere.infra.federation.executor.original.table;
 
-import java.util.ArrayList;
-import java.util.List;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelDistributionTraitDef;
 import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.schema.Statistic;
-import org.apache.calcite.util.ImmutableBitSet;
+
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * Statistic of federation table.
  */
 public final class FederationTableStatistic implements Statistic {
     
-    @Override
-    public Double getRowCount() {
-        return Statistic.super.getRowCount();
-    }
-    
-    @Override
-    public boolean isKey(final ImmutableBitSet columns) {
-        return Statistic.super.isKey(columns);
-    }
-    
-    @Override
-    public List<ImmutableBitSet> getKeys() {
-        return Statistic.super.getKeys();
-    }
-    
     @Override
     public List<RelReferentialConstraint> getReferentialConstraints() {
         return new ArrayList<>();
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/table/FilterableTable.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/table/FilterableTable.java
index a1e165c1c56..aa0fb7b0048 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/table/FilterableTable.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/table/FilterableTable.java
@@ -26,7 +26,6 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.schema.ProjectableFilterableTable;
 import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.impl.AbstractTable;
-import org.apache.shardingsphere.infra.federation.executor.original.FederationTableStatistic;
 import org.apache.shardingsphere.infra.federation.optimizer.metadata.FederationTableMetaData;
 
 import java.util.List;
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/table/FilterableTableScanExecutor.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/table/FilterableTableScanExecutor.java
index cc38d865386..8e37252040c 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/table/FilterableTableScanExecutor.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/main/java/org/apache/shardingsphere/infra/federation/executor/original/table/FilterableTableScanExecutor.java
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.infra.federation.executor.original.table;
 
-import com.google.common.collect.ImmutableList;
+import lombok.RequiredArgsConstructor;
 import lombok.SneakyThrows;
 import org.apache.calcite.linq4j.AbstractEnumerable;
 import org.apache.calcite.linq4j.Enumerable;
@@ -29,10 +29,6 @@ import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.rel2sql.RelToSqlConverter;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlDialect;
-import org.apache.calcite.sql.dialect.MssqlSqlDialect;
-import org.apache.calcite.sql.dialect.MysqlSqlDialect;
-import org.apache.calcite.sql.dialect.OracleSqlDialect;
-import org.apache.calcite.sql.dialect.PostgresqlSqlDialect;
 import org.apache.calcite.sql.util.SqlString;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.shardingsphere.infra.binder.LogicSQL;
@@ -41,13 +37,6 @@ import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.context.kernel.KernelProcessor;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
-import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
-import org.apache.shardingsphere.infra.database.type.dialect.MariaDBDatabaseType;
-import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
-import org.apache.shardingsphere.infra.database.type.dialect.OpenGaussDatabaseType;
-import org.apache.shardingsphere.infra.database.type.dialect.OracleDatabaseType;
-import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
-import org.apache.shardingsphere.infra.database.type.dialect.SQLServerDatabaseType;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroupContext;
@@ -63,6 +52,7 @@ import org.apache.shardingsphere.infra.executor.sql.execute.result.query.impl.dr
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.DriverExecutionPrepareEngine;
 import org.apache.shardingsphere.infra.executor.sql.process.ExecuteProcessEngine;
 import org.apache.shardingsphere.infra.federation.executor.FederationContext;
+import org.apache.shardingsphere.infra.federation.executor.original.SQLDialectFactory;
 import org.apache.shardingsphere.infra.federation.executor.original.row.EmptyRowEnumerator;
 import org.apache.shardingsphere.infra.federation.executor.original.row.FilterableRowEnumerator;
 import org.apache.shardingsphere.infra.federation.optimizer.context.OptimizerContext;
@@ -80,7 +70,6 @@ import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -89,20 +78,9 @@ import java.util.stream.Collectors;
 /**
  * Filterable table scan executor.
  */
+@RequiredArgsConstructor
 public final class FilterableTableScanExecutor {
     
-    private static final Map<Class<? extends DatabaseType>, SqlDialect> SQL_DIALECTS = new HashMap<>();
-    
-    static {
-        SQL_DIALECTS.put(H2DatabaseType.class, MysqlSqlDialect.DEFAULT);
-        SQL_DIALECTS.put(MySQLDatabaseType.class, MysqlSqlDialect.DEFAULT);
-        SQL_DIALECTS.put(MariaDBDatabaseType.class, MysqlSqlDialect.DEFAULT);
-        SQL_DIALECTS.put(OracleDatabaseType.class, OracleSqlDialect.DEFAULT);
-        SQL_DIALECTS.put(SQLServerDatabaseType.class, MssqlSqlDialect.DEFAULT);
-        SQL_DIALECTS.put(PostgreSQLDatabaseType.class, PostgresqlSqlDialect.DEFAULT);
-        SQL_DIALECTS.put(OpenGaussDatabaseType.class, PostgresqlSqlDialect.DEFAULT);
-    }
-    
     private final DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine;
     
     private final JDBCExecutor jdbcExecutor;
@@ -113,16 +91,6 @@ public final class FilterableTableScanExecutor {
     
     private final FilterableTableScanExecutorContext executorContext;
     
-    public FilterableTableScanExecutor(final DriverExecutionPrepareEngine<JDBCExecutionUnit, Connection> prepareEngine,
-                                       final JDBCExecutor jdbcExecutor, final JDBCExecutorCallback<? extends ExecuteResult> callback,
-                                       final OptimizerContext optimizerContext, final FilterableTableScanExecutorContext executorContext) {
-        this.jdbcExecutor = jdbcExecutor;
-        this.callback = callback;
-        this.prepareEngine = prepareEngine;
-        this.optimizerContext = optimizerContext;
-        this.executorContext = executorContext;
-    }
-    
     /**
      * Execute.
      *
@@ -134,7 +102,7 @@ public final class FilterableTableScanExecutor {
         String databaseName = executorContext.getDatabaseName();
         String schemaName = executorContext.getSchemaName();
         DatabaseType databaseType = DatabaseTypeEngine.getTrunkDatabaseType(optimizerContext.getParserContexts().get(databaseName).getDatabaseType().getType());
-        SqlString sqlString = createSQLString(tableMetaData, scanContext, databaseType);
+        SqlString sqlString = createSQLString(tableMetaData, scanContext, SQLDialectFactory.getSQLDialect(databaseType));
         // TODO replace sql parse with sql convert
         FederationContext federationContext = executorContext.getFederationContext();
         LogicSQL logicSQL = createLogicSQL(federationContext.getDatabases(), sqlString, databaseType);
@@ -187,12 +155,10 @@ public final class FilterableTableScanExecutor {
         return result;
     }
     
-    private SqlString createSQLString(final FederationTableMetaData tableMetaData, final FilterableTableScanContext scanContext, final DatabaseType databaseType) {
-        SqlDialect sqlDialect = SQL_DIALECTS.getOrDefault(databaseType.getClass(), MysqlSqlDialect.DEFAULT);
+    private SqlString createSQLString(final FederationTableMetaData tableMetaData, final FilterableTableScanContext scanContext, final SqlDialect sqlDialect) {
         return new RelToSqlConverter(sqlDialect).visitRoot(createRelNode(tableMetaData, scanContext)).asStatement().toSqlString(sqlDialect);
     }
     
-    @SneakyThrows
     private void setParameters(final Collection<ExecutionGroup<JDBCExecutionUnit>> inputGroups) {
         for (ExecutionGroup<JDBCExecutionUnit> each : inputGroups) {
             for (JDBCExecutionUnit executionUnit : each.getInputs()) {
@@ -204,7 +170,7 @@ public final class FilterableTableScanExecutor {
         }
     }
     
-    @SneakyThrows
+    @SneakyThrows(SQLException.class)
     private void setParameters(final PreparedStatement preparedStatement, final List<Object> parameters) {
         for (int i = 0; i < parameters.size(); i++) {
             Object parameter = parameters.get(i);
@@ -212,17 +178,6 @@ public final class FilterableTableScanExecutor {
         }
     }
     
-    private List<Object> getParameters(final ImmutableList<Integer> parameterIndices) {
-        if (null == parameterIndices) {
-            return Collections.emptyList();
-        }
-        List<Object> result = new ArrayList<>();
-        for (Integer each : parameterIndices) {
-            result.add(executorContext.getFederationContext().getLogicSQL().getParameters().get(each));
-        }
-        return result;
-    }
-    
     private RelNode createRelNode(final FederationTableMetaData tableMetaData, final FilterableTableScanContext scanContext) {
         String databaseName = executorContext.getDatabaseName();
         String schemaName = executorContext.getSchemaName();
@@ -263,6 +218,17 @@ public final class FilterableTableScanExecutor {
         return new LogicSQL(sqlStatementContext, sql, parameters);
     }
     
+    private List<Object> getParameters(final List<Integer> parameterIndexes) {
+        if (null == parameterIndexes) {
+            return Collections.emptyList();
+        }
+        List<Object> result = new ArrayList<>();
+        for (Integer each : parameterIndexes) {
+            result.add(executorContext.getFederationContext().getLogicSQL().getParameters().get(each));
+        }
+        return result;
+    }
+    
     private AbstractEnumerable<Object[]> createEmptyEnumerable() {
         return new AbstractEnumerable<Object[]>() {
             
diff --git a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/customized/AdvancedFederationExecutorTest.java b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/advanced/AdvancedFederationExecutorTest.java
similarity index 98%
rename from shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/customized/AdvancedFederationExecutorTest.java
rename to shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/advanced/AdvancedFederationExecutorTest.java
index a1fadfe84cd..a5ed9387718 100644
--- a/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/customized/AdvancedFederationExecutorTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-federation/shardingsphere-infra-federation-executor/src/test/java/org/apache/shardingsphere/infra/federation/executor/advanced/AdvancedFederationExecutorTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.federation.executor.customized;
+package org.apache.shardingsphere.infra.federation.executor.advanced;
 
 import lombok.SneakyThrows;
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/SQLUtil.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/SQLUtil.java
index 9d553613ae0..81ce9e66cb5 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/SQLUtil.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/SQLUtil.java
@@ -254,7 +254,7 @@ public final class SQLUtil {
     /**
      * Create literal expression.
      * 
-     * @param astNode ast node
+     * @param astNode AST node
      * @param startIndex start index
      * @param stopIndex stop index
      * @param text text
@@ -277,17 +277,17 @@ public final class SQLUtil {
     }
     
     /**
-     * Trim the semicolon of sql.
+     * Trim the semicolon of SQL.
      *
      * @param sql SQL to be trim
-     * @return sql without semicolon
+     * @return SQL without semicolon
      */
     public static String trimSemicolon(final String sql) {
         return sql.endsWith(SQL_END) ? sql.substring(0, sql.length() - 1) : sql;
     }
     
     /**
-     * Trim the comment of sql.
+     * Trim the comment of SQL.
      *
      * @param sql SQL to be trim
      * @return remove comment from SQL