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

[shardingsphere] branch master updated: Refactor advance related handlers. (#15513)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new c625de0  Refactor advance related handlers. (#15513)
c625de0 is described below

commit c625de04d933bd34d02932684ee528230e313554
Author: lanchengx <52...@users.noreply.github.com>
AuthorDate: Mon Feb 21 11:32:14 2022 +0800

    Refactor advance related handlers. (#15513)
    
    * Refactor advance related handlers.
    
    * Adjust the package path.
    
    * Update integration tests.
---
 .../distsql/ral/QueryableRALBackendHandler.java    |  4 +-
 .../text/distsql/ral/RALBackendHandler.java        |  3 +-
 .../text/distsql/ral/RALBackendHandlerFactory.java | 11 ++--
 .../AdvancedDistSQLBackendHandlerFactory.java      | 55 -----------------
 .../ral/advanced/ParseDistSQLBackendHandler.java   | 58 ++++++++----------
 .../ral/advanced/PreviewDistSQLBackendHandler.java | 69 +++++++++-------------
 .../advance/ParseDistSQLBackendHandlerTest.java    | 21 ++++---
 .../mysql/preview_federation_select.xml            |  2 +-
 .../postgresql/preview_federation_select.xml       |  2 +-
 .../cases/ral/dataset/empty_rules/preview_sql.xml  |  2 +-
 10 files changed, 78 insertions(+), 149 deletions(-)

diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/QueryableRALBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/QueryableRALBackendHandler.java
index 5c2dfdb..db4565b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/QueryableRALBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/QueryableRALBackendHandler.java
@@ -44,7 +44,7 @@ public abstract class QueryableRALBackendHandler<E extends RALStatement, R exten
     private MultipleLocalDataMergedResult mergedResult;
     
     @Override
-    protected final ResponseHeader handle(final ContextManager contextManager, final E sqlStatement) {
+    protected final ResponseHeader handle(final ContextManager contextManager, final E sqlStatement) throws SQLException {
         queryHeaders = createQueryHeader(getColumnNames());
         mergedResult = createMergedResult(getRows(contextManager));
         return new QueryResponseHeader(queryHeaders);
@@ -62,7 +62,7 @@ public abstract class QueryableRALBackendHandler<E extends RALStatement, R exten
     
     protected abstract Collection<String> getColumnNames();
     
-    protected abstract Collection<List<Object>> getRows(ContextManager contextManager);
+    protected abstract Collection<List<Object>> getRows(ContextManager contextManager) throws SQLException;
     
     private MultipleLocalDataMergedResult createMergedResult(final Collection<List<Object>> rows) {
         return new MultipleLocalDataMergedResult(rows);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/RALBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/RALBackendHandler.java
index dc8bc29..75c55da 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/RALBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/RALBackendHandler.java
@@ -25,7 +25,6 @@ import lombok.Setter;
 import lombok.experimental.Accessors;
 import org.apache.shardingsphere.distsql.parser.statement.ral.RALStatement;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
@@ -50,7 +49,7 @@ public abstract class RALBackendHandler<E extends RALStatement, R extends RALBac
         return handle(contextManager, sqlStatement);
     }
     
-    protected abstract ResponseHeader handle(ContextManager contextManager, E sqlStatement) throws DistSQLException;
+    protected abstract ResponseHeader handle(ContextManager contextManager, E sqlStatement) throws SQLException;
     
     /**
      * Method to initialize handler, this method needs to be rewritten when the handler has properties other than sql statement.
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/RALBackendHandlerFactory.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/RALBackendHandlerFactory.java
index 0c549c8..5b3bfd3 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/RALBackendHandlerFactory.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/RALBackendHandlerFactory.java
@@ -19,10 +19,11 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.ral;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.distsql.parser.statement.ral.AdvancedDistSQLStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.RALStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableRALStatement;
+import org.apache.shardingsphere.distsql.parser.statement.ral.advanced.parse.ParseStatement;
+import org.apache.shardingsphere.distsql.parser.statement.ral.advanced.preview.PreviewStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.ExportSchemaConfigurationStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.HintDistSQLStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.RefreshTableMetadataStatement;
@@ -48,7 +49,8 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
 import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.RALBackendHandler.HandlerParameter;
-import org.apache.shardingsphere.proxy.backend.text.distsql.ral.advanced.AdvancedDistSQLBackendHandlerFactory;
+import org.apache.shardingsphere.proxy.backend.text.distsql.ral.advanced.ParseDistSQLBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.ral.advanced.PreviewDistSQLBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.HintDistSQLBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.queryable.ExportSchemaConfigurationHandler;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.queryable.ShowAllVariablesHandler;
@@ -122,6 +124,8 @@ public final class RALBackendHandlerFactory {
         QUERYABLE_HANDLER_MAP.put(ShowTrafficRulesStatement.class.getName(), ShowTrafficRulesHandler.class);
         QUERYABLE_HANDLER_MAP.put(ShowTransactionRuleStatement.class.getName(), ShowTransactionRuleHandler.class);
         QUERYABLE_HANDLER_MAP.put(ExportSchemaConfigurationStatement.class.getName(), ExportSchemaConfigurationHandler.class);
+        QUERYABLE_HANDLER_MAP.put(ParseStatement.class.getName(), ParseDistSQLBackendHandler.class);
+        QUERYABLE_HANDLER_MAP.put(PreviewStatement.class.getName(), PreviewDistSQLBackendHandler.class);
     }
     
     /**
@@ -144,9 +148,6 @@ public final class RALBackendHandlerFactory {
         if (sqlStatement instanceof UpdatableRALStatement) {
             result = UpdatableScalingRALBackendHandlerFactory.newInstance((UpdatableRALStatement) sqlStatement);
         }
-        if (sqlStatement instanceof AdvancedDistSQLStatement) {
-            result = AdvancedDistSQLBackendHandlerFactory.newInstance(databaseType, (AdvancedDistSQLStatement) sqlStatement, connectionSession);
-        }
         if (result == null) {
             HandlerParameter parameter = new HandlerParameter(sqlStatement, databaseType, connectionSession);
             result = getHandler(sqlStatement, parameter);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/AdvancedDistSQLBackendHandlerFactory.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/AdvancedDistSQLBackendHandlerFactory.java
deleted file mode 100644
index 1996bf8..0000000
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/AdvancedDistSQLBackendHandlerFactory.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.proxy.backend.text.distsql.ral.advanced;
-
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.distsql.parser.statement.ral.AdvancedDistSQLStatement;
-import org.apache.shardingsphere.distsql.parser.statement.ral.advanced.parse.ParseStatement;
-import org.apache.shardingsphere.distsql.parser.statement.ral.advanced.preview.PreviewStatement;
-import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
-import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
-
-import java.sql.SQLException;
-
-/**
- * Advanced dist sql backend handler factory.
- */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class AdvancedDistSQLBackendHandlerFactory {
-    
-    /**
-     * Create new instance of advanced dist sql backend handler.
-     *
-     * @param databaseType database type
-     * @param sqlStatement advanced dist sql statement
-     * @param connectionSession connection session
-     * @return advanced dist sql backend handler
-     * @throws SQLException SQL exception
-     */
-    public static TextProtocolBackendHandler newInstance(final DatabaseType databaseType, final AdvancedDistSQLStatement sqlStatement,
-                                                         final ConnectionSession connectionSession) throws SQLException {
-        if (sqlStatement instanceof PreviewStatement) {
-            return new PreviewDistSQLBackendHandler((PreviewStatement) sqlStatement, connectionSession);
-        } else if (sqlStatement instanceof ParseStatement) {
-            return new ParseDistSQLBackendHandler(databaseType, (ParseStatement) sqlStatement, connectionSession);
-        }
-        throw new UnsupportedOperationException(sqlStatement.getClass().getCanonicalName());
-    }
-}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/ParseDistSQLBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/ParseDistSQLBackendHandler.java
index 933f43d..b6c7db9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/ParseDistSQLBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/ParseDistSQLBackendHandler.java
@@ -19,58 +19,58 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.ral.advanced;
 
 import com.google.common.base.Strings;
 import com.google.gson.Gson;
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.advanced.parse.ParseStatement;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
+import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
-import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
-import org.apache.shardingsphere.proxy.backend.response.header.query.impl.QueryHeader;
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
-import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.ral.QueryableRALBackendHandler;
 import org.apache.shardingsphere.sql.parser.exception.SQLParsingException;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
-import java.sql.Types;
-import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Optional;
 
 /**
  * Parse dist sql backend handler.
  */
-@RequiredArgsConstructor
-@Getter
-public final class ParseDistSQLBackendHandler implements TextProtocolBackendHandler {
+public final class ParseDistSQLBackendHandler extends QueryableRALBackendHandler<ParseStatement, ParseDistSQLBackendHandler> {
     
-    private final DatabaseType databaseType;
+    private static final String PARSED_STATEMENT = "parsed_statement";
     
-    private final ParseStatement previewStatement;
+    private static final String PARSED_STATEMENT_DETAIL = "parsed_statement_detail";
     
-    private final ConnectionSession connectionSession;
+    private DatabaseType databaseType;
     
-    private final List<QueryHeader> queryHeaders = new ArrayList<>(1);
+    private ConnectionSession connectionSession;
     
-    private Iterator<String> data = Collections.emptyIterator();
+    @Override
+    public ParseDistSQLBackendHandler init(final HandlerParameter<ParseStatement> parameter) {
+        databaseType = parameter.getDatabaseType();
+        connectionSession = parameter.getConnectionSession();
+        return super.init(parameter);
+    }
+    
+    @Override
+    protected Collection<String> getColumnNames() {
+        return Arrays.asList(PARSED_STATEMENT, PARSED_STATEMENT_DETAIL);
+    }
     
     @Override
-    public ResponseHeader execute() {
-        Optional<SQLParserRule> sqlParserRule = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        SQLStatement sqlStatement;
+    protected Collection<List<Object>> getRows(final ContextManager contextManager) {
+        Optional<SQLParserRule> sqlParserRule = contextManager.getMetaDataContexts().getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
+        SQLStatement parsedSqlStatement;
         try {
-            sqlStatement = new ShardingSphereSQLParserEngine(getBackendDatabaseType(databaseType, connectionSession).getName(), sqlParserRule.orElse(null)).parse(previewStatement.getSql(), false);
+            parsedSqlStatement = new ShardingSphereSQLParserEngine(getBackendDatabaseType(databaseType, connectionSession).getName(), sqlParserRule.orElse(null)).parse(sqlStatement.getSql(), false);
         } catch (SQLParsingException ex) {
             throw new SQLParsingException("You have an error in your SQL syntax that you are parsed");
         }
-        data = Collections.singleton(new Gson().toJson(sqlStatement)).iterator();
-        queryHeaders.add(new QueryHeader("", "", sqlStatement.getClass().getSimpleName(), "", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
-        return new QueryResponseHeader(queryHeaders);
+        return Collections.singleton(Arrays.asList(parsedSqlStatement.getClass().getSimpleName(), new Gson().toJson(parsedSqlStatement)));
     }
     
     private static DatabaseType getBackendDatabaseType(final DatabaseType defaultDatabaseType, final ConnectionSession connectionSession) {
@@ -78,14 +78,4 @@ public final class ParseDistSQLBackendHandler implements TextProtocolBackendHand
         return Strings.isNullOrEmpty(schemaName) || !ProxyContext.getInstance().schemaExists(schemaName)
                 ? defaultDatabaseType : ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData(schemaName).getResource().getDatabaseType();
     }
-    
-    @Override
-    public boolean next() {
-        return data.hasNext();
-    }
-    
-    @Override
-    public Collection<Object> getRowData() {
-        return Collections.singleton(data.next());
-    }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/PreviewDistSQLBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/PreviewDistSQLBackendHandler.java
index 30810e4..9712c72 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/PreviewDistSQLBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advanced/PreviewDistSQLBackendHandler.java
@@ -18,8 +18,6 @@
 package org.apache.shardingsphere.proxy.backend.text.distsql.ral.advanced;
 
 import com.google.common.base.Strings;
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.advanced.preview.PreviewStatement;
 import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
@@ -47,6 +45,7 @@ import org.apache.shardingsphere.infra.federation.executor.FederationExecutor;
 import org.apache.shardingsphere.infra.federation.executor.FederationExecutorFactory;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
+import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.proxy.backend.communication.SQLStatementSchemaHolder;
@@ -55,51 +54,54 @@ import org.apache.shardingsphere.proxy.backend.context.BackendExecutorContext;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.exception.NoDatabaseSelectedException;
 import org.apache.shardingsphere.proxy.backend.exception.RuleNotExistedException;
-import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
-import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
-import org.apache.shardingsphere.proxy.backend.response.header.query.impl.QueryHeader;
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
-import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
+import org.apache.shardingsphere.proxy.backend.text.distsql.ral.QueryableRALBackendHandler;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLInsertStatement;
 
 import java.sql.Connection;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.sql.Types;
-import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Optional;
+import java.util.stream.Collectors;
 
 /**
  * Preview dist sql backend handler.
  */
-@RequiredArgsConstructor
-@Getter
-public final class PreviewDistSQLBackendHandler implements TextProtocolBackendHandler {
+public final class PreviewDistSQLBackendHandler extends QueryableRALBackendHandler<PreviewStatement, PreviewDistSQLBackendHandler> {
     
-    private final PreviewStatement previewStatement;
+    private static final String DATA_SOURCE_NAME = "data_source_name";
     
-    private final ConnectionSession connectionSession;
+    private static final String ACTUAL_SQL = "actual_sql";
+    
+    private ConnectionSession connectionSession;
     
     private final KernelProcessor kernelProcessor = new KernelProcessor();
     
-    private List<QueryHeader> queryHeaders;
+    @Override
+    public PreviewDistSQLBackendHandler init(final HandlerParameter<PreviewStatement> parameter) {
+        connectionSession = parameter.getConnectionSession();
+        return super.init(parameter);
+    }
     
-    private Iterator<ExecutionUnit> executionUnits;
+    @Override
+    protected Collection<String> getColumnNames() {
+        return Arrays.asList(DATA_SOURCE_NAME, ACTUAL_SQL);
+    }
     
     @Override
-    public ResponseHeader execute() throws SQLException {
+    protected Collection<List<Object>> getRows(final ContextManager contextManager) throws SQLException {
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
         String schemaName = getSchemaName();
         String databaseType = DatabaseTypeRegistry.getTrunkDatabaseTypeName(metaDataContexts.getMetaData(schemaName).getResource().getDatabaseType());
         Optional<SQLParserRule> sqlParserRule = metaDataContexts.getGlobalRuleMetaData().findSingleRule(SQLParserRule.class);
-        SQLStatement sqlStatement = new ShardingSphereSQLParserEngine(databaseType, sqlParserRule.get()).parse(previewStatement.getSql(), false);
-        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaDataMap(), Collections.emptyList(), sqlStatement, schemaName);
+        SQLStatement previewedStatement = new ShardingSphereSQLParserEngine(databaseType, sqlParserRule.get()).parse(sqlStatement.getSql(), false);
+        SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaDataContexts.getMetaDataMap(), Collections.emptyList(), previewedStatement, schemaName);
         // TODO optimize SQLStatementSchemaHolder
         if (sqlStatementContext instanceof TableAvailable) {
             ((TableAvailable) sqlStatementContext).getTablesContext().getSchemaName().ifPresent(SQLStatementSchemaHolder::set);
@@ -108,14 +110,15 @@ public final class PreviewDistSQLBackendHandler implements TextProtocolBackendHa
         if (!metaData.isComplete()) {
             throw new RuleNotExistedException();
         }
-        LogicSQL logicSQL = new LogicSQL(sqlStatementContext, previewStatement.getSql(), Collections.emptyList());
+        LogicSQL logicSQL = new LogicSQL(sqlStatementContext, sqlStatement.getSql(), Collections.emptyList());
         ExecutionContext executionContext = kernelProcessor.generateExecutionContext(logicSQL, metaData, metaDataContexts.getProps());
-        executionUnits = executionContext.getRouteContext().isFederated() 
-                ? getFederationExecutionUnits(logicSQL, schemaName, metaDataContexts).iterator() : executionContext.getExecutionUnits().iterator();
-        queryHeaders = new ArrayList<>(2);
-        queryHeaders.add(new QueryHeader("", "", "data_source_name", "", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
-        queryHeaders.add(new QueryHeader("", "", "sql", "", Types.CHAR, "CHAR", 255, 0, false, false, false, false));
-        return new QueryResponseHeader(queryHeaders);
+        Collection<ExecutionUnit> executionUnits = executionContext.getRouteContext().isFederated()
+                ? getFederationExecutionUnits(logicSQL, schemaName, metaDataContexts) : executionContext.getExecutionUnits();
+        return executionUnits.stream().map(this::buildRow).collect(Collectors.toCollection(LinkedList::new));
+    }
+    
+    private List<Object> buildRow(final ExecutionUnit unit) {
+        return Arrays.asList(unit.getDataSourceName(), unit.getSqlUnit().getSql());
     }
     
     private Collection<ExecutionUnit> getFederationExecutionUnits(final LogicSQL logicSQL, final String schemaName, final MetaDataContexts metaDataContexts) throws SQLException {
@@ -151,20 +154,6 @@ public final class PreviewDistSQLBackendHandler implements TextProtocolBackendHa
                 new StatementOption(isReturnGeneratedKeys), metaData.getMetaData(getSchemaName()).getRuleMetaData().getRules());
     }
     
-    @Override
-    public boolean next() {
-        return null != executionUnits && executionUnits.hasNext();
-    }
-    
-    @Override
-    public Collection<Object> getRowData() {
-        ExecutionUnit executionUnit = executionUnits.next();
-        Collection<Object> result = new LinkedList<>();
-        result.add(executionUnit.getDataSourceName());
-        result.add(executionUnit.getSqlUnit().getSql());
-        return result;
-    }
-    
     private String getSchemaName() {
         String result = !Strings.isNullOrEmpty(connectionSession.getSchemaName()) ? connectionSession.getSchemaName() : connectionSession.getDefaultSchemaName();
         if (Strings.isNullOrEmpty(result)) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advance/ParseDistSQLBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advance/ParseDistSQLBackendHandlerTest.java
index fa1e217..68fedca 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advance/ParseDistSQLBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/advance/ParseDistSQLBackendHandlerTest.java
@@ -26,6 +26,7 @@ import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.parser.rule.builder.DefaultSQLParserRuleConfigurationBuilder;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
+import org.apache.shardingsphere.proxy.backend.text.distsql.ral.RALBackendHandler.HandlerParameter;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.advanced.ParseDistSQLBackendHandler;
 import org.apache.shardingsphere.sql.parser.exception.SQLParsingException;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
@@ -47,12 +48,12 @@ import static org.mockito.Mockito.when;
 
 @RunWith(MockitoJUnitRunner.class)
 public final class ParseDistSQLBackendHandlerTest {
-
+    
     private final SQLParserRule sqlParserRule = new SQLParserRule(new DefaultSQLParserRuleConfigurationBuilder().build());
-
+    
     @Mock(answer = Answers.RETURNS_DEEP_STUBS)
     private ContextManager contextManager;
-
+    
     @Before
     public void setUp() throws SQLException {
         ProxyContext.getInstance().init(contextManager);
@@ -60,20 +61,24 @@ public final class ParseDistSQLBackendHandlerTest {
     }
     
     @Test
-    public void assertGetRowData() {
+    public void assertGetRowData() throws SQLException {
         String sql = "select * from t_order";
         ParseStatement parseStatement = new ParseStatement(sql);
-        ParseDistSQLBackendHandler parseDistSQLBackendHandler = new ParseDistSQLBackendHandler(new MySQLDatabaseType(), parseStatement, mock(ConnectionSession.class));
+        ParseDistSQLBackendHandler parseDistSQLBackendHandler = new ParseDistSQLBackendHandler()
+                .init(new HandlerParameter<ParseStatement>().setStatement(parseStatement).setConnectionSession(mock(ConnectionSession.class)).setDatabaseType(new MySQLDatabaseType()));
         parseDistSQLBackendHandler.execute();
+        parseDistSQLBackendHandler.next();
         SQLStatement statement = new ShardingSphereSQLParserEngine("MySQL", sqlParserRule).parse(sql, false);
-        assertThat(new LinkedList<>(parseDistSQLBackendHandler.getRowData()).getFirst(), is(new Gson().toJson(statement)));
+        assertThat(new LinkedList<>(parseDistSQLBackendHandler.getRowData()).getFirst(), is("MySQLSelectStatement"));
+        assertThat(new LinkedList<>(parseDistSQLBackendHandler.getRowData()).getLast(), is(new Gson().toJson(statement)));
     }
     
     @Test(expected = SQLParsingException.class)
-    public void assertExecute() {
+    public void assertExecute() throws SQLException {
         String sql = "wrong sql";
         ParseStatement parseStatement = new ParseStatement(sql);
-        ParseDistSQLBackendHandler parseDistSQLBackendHandler = new ParseDistSQLBackendHandler(new MySQLDatabaseType(), parseStatement, mock(ConnectionSession.class));
+        ParseDistSQLBackendHandler parseDistSQLBackendHandler = new ParseDistSQLBackendHandler()
+                .init(new HandlerParameter<ParseStatement>().setStatement(parseStatement).setConnectionSession(mock(ConnectionSession.class)).setDatabaseType(new MySQLDatabaseType()));
         parseDistSQLBackendHandler.execute();
     }
 }
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/resources/cases/ral/dataset/empty_rules/mysql/preview_federation_select.xml b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/resources/cases/ral/dataset/empty_rules/mysql/preview_federation_select.xml
index 1225e83..57caec5 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/resources/cases/ral/dataset/empty_rules/mysql/preview_federation_select.xml
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/resources/cases/ral/dataset/empty_rules/mysql/preview_federation_select.xml
@@ -18,7 +18,7 @@
 <dataset>
     <metadata>
         <column name="data_source_name"/>
-        <column name="sql"/>
+        <column name="actual_sql"/>
     </metadata>
     <row values="encrypt_write_ds_0| SELECT * FROM `t_user_item_0` WHERE CAST(`user_id` AS SIGNED) = 1 UNION ALL SELECT * FROM `t_user_item_10` WHERE CAST(`user_id` AS SIGNED) = 1 UNION ALL SELECT * FROM `t_user_item_20` WHERE CAST(`user_id` AS SIGNED) = 1 UNION ALL SELECT * FROM `t_user_item_30` WHERE CAST(`user_id` AS SIGNED) = 1"/>
     <row values="encrypt_write_ds_1| SELECT * FROM `t_user_item_1` WHERE CAST(`user_id` AS SIGNED) = 1 UNION ALL SELECT * FROM `t_user_item_11` WHERE CAST(`user_id` AS SIGNED) = 1 UNION ALL SELECT * FROM `t_user_item_21` WHERE CAST(`user_id` AS SIGNED) = 1 UNION ALL SELECT * FROM `t_user_item_31` WHERE CAST(`user_id` AS SIGNED) = 1"/>
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/resources/cases/ral/dataset/empty_rules/postgresql/preview_federation_select.xml b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/resources/cases/ral/dataset/empty_rules/postgresql/preview_federation_select.xml
index ee6b07a..aad09e6 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/resources/cases/ral/dataset/empty_rules/postgresql/preview_federation_select.xml
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/resources/cases/ral/dataset/empty_rules/postgresql/preview_federation_select.xml
@@ -18,7 +18,7 @@
 <dataset>
     <metadata>
         <column name="data_source_name"/>
-        <column name="sql"/>
+        <column name="actual_sql"/>
     </metadata>
     <row values="encrypt_write_ds_0| SELECT * FROM &quot;t_user_item_0&quot; WHERE CAST(&quot;user_id&quot; AS INTEGER) = 1 UNION ALL SELECT * FROM &quot;t_user_item_10&quot; WHERE CAST(&quot;user_id&quot; AS INTEGER) = 1 UNION ALL SELECT * FROM &quot;t_user_item_20&quot; WHERE CAST(&quot;user_id&quot; AS INTEGER) = 1 UNION ALL SELECT * FROM &quot;t_user_item_30&quot; WHERE CAST(&quot;user_id&quot; AS INTEGER) = 1"/>
     <row values="encrypt_write_ds_1| SELECT * FROM &quot;t_user_item_1&quot; WHERE CAST(&quot;user_id&quot; AS INTEGER) = 1 UNION ALL SELECT * FROM &quot;t_user_item_11&quot; WHERE CAST(&quot;user_id&quot; AS INTEGER) = 1 UNION ALL SELECT * FROM &quot;t_user_item_21&quot; WHERE CAST(&quot;user_id&quot; AS INTEGER) = 1 UNION ALL SELECT * FROM &quot;t_user_item_31&quot; WHERE CAST(&quot;user_id&quot; AS INTEGER) = 1"/>
diff --git a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/resources/cases/ral/dataset/empty_rules/preview_sql.xml b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/resources/cases/ral/dataset/empty_rules/preview_sql.xml
index f5af2bf..0bf6756 100644
--- a/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/resources/cases/ral/dataset/empty_rules/preview_sql.xml
+++ b/shardingsphere-test/shardingsphere-integration-test/shardingsphere-integration-test-suite/src/test/resources/cases/ral/dataset/empty_rules/preview_sql.xml
@@ -18,7 +18,7 @@
 <dataset>
     <metadata>
         <column name="data_source_name"/>
-        <column name="sql"/>
+        <column name="actual_sql"/>
     </metadata>
     <row values="encrypt_write_ds_0| select * from t_user_item_0 UNION ALL select * from t_user_item_10 UNION ALL select * from t_user_item_20 UNION ALL select * from t_user_item_30"/>
     <row values="encrypt_write_ds_1| select * from t_user_item_1 UNION ALL select * from t_user_item_11 UNION ALL select * from t_user_item_21 UNION ALL select * from t_user_item_31"/>