You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2021/06/25 08:57:24 UTC

[shardingsphere] branch master updated: Show logic SQL in execution process list Info column (#11006)

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

panjuan 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 2fa8d9d  Show logic SQL in execution process list Info column (#11006)
2fa8d9d is described below

commit 2fa8d9de63960f5a160b64306418d9007a7ed528
Author: sandynz <42...@users.noreply.github.com>
AuthorDate: Fri Jun 25 16:56:56 2021 +0800

    Show logic SQL in execution process list Info column (#11006)
    
    * Prepare LogicSQL for ExecuteProcessReporter
    
    * Persist logic SQL and show in Info column
---
 .../process/GovernanceExecuteProcessReporter.java  |  6 +++---
 .../infra/context/kernel/KernelProcessor.java      |  2 +-
 .../executor/sql/context/ExecutionContext.java     | 13 ++++++++----
 .../sql/execute/engine/raw/RawExecutor.java        |  8 ++++----
 .../federate/schema/row/FederateRowExecutor.java   |  2 +-
 .../FederateExecutionContextGenerator.java         |  2 +-
 .../executor/sql/process/ExecuteProcessEngine.java |  8 +++++---
 .../sql/process/model/ExecuteProcessContext.java   |  5 ++++-
 .../model/yaml/YamlExecuteProcessContext.java      |  3 +++
 .../sql/process/spi/ExecuteProcessReporter.java    |  6 +++---
 .../infra/executor/sql/log/SQLLoggerTest.java      |  6 +++---
 .../driver/executor/DriverJDBCExecutor.java        | 24 ++++++++++++----------
 .../statement/ShardingSpherePreparedStatement.java | 12 +++++------
 .../core/statement/ShardingSphereStatement.java    | 18 ++++++++--------
 .../backend/communication/ProxySQLExecutor.java    |  4 ++--
 .../jdbc/executor/ProxyJDBCExecutor.java           |  8 +++++---
 .../mysql/executor/ShowProcessListExecutor.java    |  4 ++--
 17 files changed, 74 insertions(+), 57 deletions(-)

diff --git a/shardingsphere-governance/shardingsphere-governance-context/src/main/java/org/apache/shardingsphere/governance/context/process/GovernanceExecuteProcessReporter.java b/shardingsphere-governance/shardingsphere-governance-context/src/main/java/org/apache/shardingsphere/governance/context/process/GovernanceExecuteProcessReporter.java
index eaff29a..5538457 100644
--- a/shardingsphere-governance/shardingsphere-governance-context/src/main/java/org/apache/shardingsphere/governance/context/process/GovernanceExecuteProcessReporter.java
+++ b/shardingsphere-governance/shardingsphere-governance-context/src/main/java/org/apache/shardingsphere/governance/context/process/GovernanceExecuteProcessReporter.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.governance.context.process;
 import org.apache.shardingsphere.governance.core.registry.process.event.ExecuteProcessReportEvent;
 import org.apache.shardingsphere.governance.core.registry.process.event.ExecuteProcessSummaryReportEvent;
 import org.apache.shardingsphere.governance.core.registry.process.event.ExecuteProcessUnitReportEvent;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroupContext;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.SQLExecutionUnit;
@@ -35,8 +35,8 @@ import org.apache.shardingsphere.infra.executor.sql.process.spi.ExecuteProcessRe
 public final class GovernanceExecuteProcessReporter implements ExecuteProcessReporter {
     
     @Override
-    public void report(final SQLStatementContext<?> context, final ExecutionGroupContext<? extends SQLExecutionUnit> executionGroupContext, final ExecuteProcessConstants constants) {
-        ExecuteProcessContext executeProcessContext = new ExecuteProcessContext(executionGroupContext, constants);
+    public void report(final LogicSQL logicSQL, final ExecutionGroupContext<? extends SQLExecutionUnit> executionGroupContext, final ExecuteProcessConstants constants) {
+        ExecuteProcessContext executeProcessContext = new ExecuteProcessContext(logicSQL.getSql(), executionGroupContext, constants);
         ShardingSphereEventBus.getInstance().post(new ExecuteProcessSummaryReportEvent(executeProcessContext));
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/kernel/KernelProcessor.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/kernel/KernelProcessor.java
index 3a67487..00e50e7 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/kernel/KernelProcessor.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/kernel/KernelProcessor.java
@@ -60,7 +60,7 @@ public final class KernelProcessor {
     }
     
     private ExecutionContext createExecutionContext(final LogicSQL logicSQL, final ShardingSphereMetaData metaData, final RouteContext routeContext, final SQLRewriteResult rewriteResult) {
-        return new ExecutionContext(logicSQL.getSqlStatementContext(), ExecutionContextBuilder.build(metaData, rewriteResult, logicSQL.getSqlStatementContext()), routeContext);
+        return new ExecutionContext(logicSQL, ExecutionContextBuilder.build(metaData, rewriteResult, logicSQL.getSqlStatementContext()), routeContext);
     }
     
     private void logSQL(final LogicSQL logicSQL, final ConfigurationProperties props, final ExecutionContext executionContext) {
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContext.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContext.java
index b855745..5ba5515 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContext.java
@@ -19,11 +19,11 @@ package org.apache.shardingsphere.infra.executor.sql.context;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 
 import java.util.Collection;
-import java.util.Collections;
 
 /**
  * Execution context.
@@ -32,13 +32,18 @@ import java.util.Collections;
 @Getter
 public final class ExecutionContext {
     
-    private final SQLStatementContext<?> sqlStatementContext;
+    private final LogicSQL logicSQL;
     
     private final Collection<ExecutionUnit> executionUnits;
     
     private final RouteContext routeContext;
     
-    public ExecutionContext(final SQLStatementContext<?> sqlStatementContext, final ExecutionUnit executionUnit, final RouteContext routeContext) {
-        this(sqlStatementContext, Collections.singletonList(executionUnit), routeContext);
+    /**
+     * Get SQL statement context.
+     *
+     * @return SQL statement context.
+     */
+    public SQLStatementContext<?> getSqlStatementContext() {
+        return logicSQL.getSqlStatementContext();
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/raw/RawExecutor.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/raw/RawExecutor.java
index c6dd020..8923810 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/raw/RawExecutor.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/raw/RawExecutor.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.infra.executor.sql.execute.engine.raw;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.commons.collections4.CollectionUtils;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroupContext;
@@ -51,16 +51,16 @@ public final class RawExecutor {
      * Execute.
      *
      * @param executionGroupContext execution group context
-     * @param sqlStatementContext SQL statement context
+     * @param logicSQL logic SQL
      * @param callback raw SQL executor callback
      * @return execute results
      * @throws SQLException SQL exception
      */
     public Collection<ExecuteResult> execute(final ExecutionGroupContext<RawSQLExecutionUnit> executionGroupContext,
-                                             final SQLStatementContext<?> sqlStatementContext,
+                                             final LogicSQL logicSQL,
                                              final RawSQLExecutorCallback callback) throws SQLException {
         try {
-            ExecuteProcessEngine.initialize(sqlStatementContext, executionGroupContext, props);
+            ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, props);
             // TODO Load query header for first query
             List<ExecuteResult> results = execute(executionGroupContext, (RawSQLExecutorCallback) null, callback);
             ExecuteProcessEngine.finish(executionGroupContext.getExecutionID());
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/row/FederateRowExecutor.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/row/FederateRowExecutor.java
index 1761a82..b6d96a6 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/row/FederateRowExecutor.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/row/FederateRowExecutor.java
@@ -75,7 +75,7 @@ public final class FederateRowExecutor {
     private Collection<QueryResult> execute(final ExecutionContext context) {
         try {
             ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext = prepareEngine.prepare(context.getRouteContext(), context.getExecutionUnits());
-            ExecuteProcessEngine.initialize(context.getSqlStatementContext(), executionGroupContext, props);
+            ExecuteProcessEngine.initialize(context.getLogicSQL(), executionGroupContext, props);
             Collection<QueryResult> result = jdbcExecutor.execute(executionGroupContext, callback).stream().map(each -> (QueryResult) each).collect(Collectors.toList());
             ExecuteProcessEngine.finish(executionGroupContext.getExecutionID());
             return result;
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecutionContextGenerator.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecutionContextGenerator.java
index df3d0e9..ba869c4 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecutionContextGenerator.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/federate/schema/table/generator/FederateExecutionContextGenerator.java
@@ -48,7 +48,7 @@ public final class FederateExecutionContextGenerator {
      */
     public ExecutionContext generate() {
         RouteContext routeContext = getRouteContext(routeExecutionContext.getRouteContext());
-        return new ExecutionContext(routeExecutionContext.getSqlStatementContext(),
+        return new ExecutionContext(routeExecutionContext.getLogicSQL(),
                 getExecutionUnits(routeContext.getRouteUnits(), generator), routeContext);
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/ExecuteProcessEngine.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/ExecuteProcessEngine.java
index 7c90045..cc6eccc 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/ExecuteProcessEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/ExecuteProcessEngine.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.infra.executor.sql.process;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroupContext;
@@ -46,14 +47,15 @@ public final class ExecuteProcessEngine {
     /**
      * Initialize.
      *
-     * @param context context
+     * @param logicSQL logic SQL
      * @param executionGroupContext execution group context
      * @param props configuration properties
      */
-    public static void initialize(final SQLStatementContext<?> context, final ExecutionGroupContext<? extends SQLExecutionUnit> executionGroupContext, final ConfigurationProperties props) {
+    public static void initialize(final LogicSQL logicSQL, final ExecutionGroupContext<? extends SQLExecutionUnit> executionGroupContext, final ConfigurationProperties props) {
+        SQLStatementContext<?> context = logicSQL.getSqlStatementContext();
         if (!HANDLERS.isEmpty() && ExecuteProcessStrategyEvaluator.evaluate(context, executionGroupContext, props)) {
             ExecutorDataMap.getValue().put(ExecuteProcessConstants.EXECUTE_ID.name(), executionGroupContext.getExecutionID());
-            HANDLERS.iterator().next().report(context, executionGroupContext, ExecuteProcessConstants.EXECUTE_STATUS_START);
+            HANDLERS.iterator().next().report(logicSQL, executionGroupContext, ExecuteProcessConstants.EXECUTE_STATUS_START);
         }
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/model/ExecuteProcessContext.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/model/ExecuteProcessContext.java
index 4495e49..ff150cb 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/model/ExecuteProcessContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/model/ExecuteProcessContext.java
@@ -33,12 +33,15 @@ public final class ExecuteProcessContext {
     
     private final String executionID;
     
+    private final String sql;
+    
     private final Collection<ExecuteProcessUnit> unitStatuses;
     
     private final long startTimeMillis = System.currentTimeMillis();
     
-    public ExecuteProcessContext(final ExecutionGroupContext<? extends SQLExecutionUnit> executionGroupContext, final ExecuteProcessConstants constants) {
+    public ExecuteProcessContext(final String sql, final ExecutionGroupContext<? extends SQLExecutionUnit> executionGroupContext, final ExecuteProcessConstants constants) {
         this.executionID = executionGroupContext.getExecutionID();
+        this.sql = sql;
         unitStatuses = createExecutionUnitStatuses(executionGroupContext, constants);
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/model/yaml/YamlExecuteProcessContext.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/model/yaml/YamlExecuteProcessContext.java
index 1cab3c7..ea64dab 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/model/yaml/YamlExecuteProcessContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/model/yaml/YamlExecuteProcessContext.java
@@ -36,12 +36,15 @@ public final class YamlExecuteProcessContext {
     
     private String executionID;
     
+    private String sql;
+    
     private Collection<YamlExecuteProcessUnit> unitStatuses;
     
     private Long startTimeMillis;
     
     public YamlExecuteProcessContext(final ExecuteProcessContext executeProcessContext) {
         executionID = executeProcessContext.getExecutionID();
+        sql = executeProcessContext.getSql();
         unitStatuses = executeProcessContext.getUnitStatuses().stream().map(YamlExecuteProcessUnit::new).collect(Collectors.toList());
         startTimeMillis = executeProcessContext.getStartTimeMillis();
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/spi/ExecuteProcessReporter.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/spi/ExecuteProcessReporter.java
index 135e222..93c5c54 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/spi/ExecuteProcessReporter.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/process/spi/ExecuteProcessReporter.java
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.infra.executor.sql.process.spi;
 
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroupContext;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.SQLExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.process.model.ExecuteProcessConstants;
@@ -29,11 +29,11 @@ public interface ExecuteProcessReporter {
     
     /**
      * Report the summary of this task.
-     * @param context context
+     * @param logicSQL logic SQL
      * @param executionGroupContext execution group context
      * @param constants constants
      */
-    void report(SQLStatementContext<?> context, ExecutionGroupContext<? extends SQLExecutionUnit> executionGroupContext, ExecuteProcessConstants constants);
+    void report(LogicSQL logicSQL, ExecutionGroupContext<? extends SQLExecutionUnit> executionGroupContext, ExecuteProcessConstants constants);
     
     /**
      * Report a unit of this task.
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/log/SQLLoggerTest.java b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/log/SQLLoggerTest.java
index e861d65..9c03b30 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/log/SQLLoggerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/log/SQLLoggerTest.java
@@ -68,7 +68,7 @@ public final class SQLLoggerTest {
     
     @Test
     public void assertLogNormalSQLWithoutParameter() {
-        SQLLogger.logSQL(logicSQL, false, new ExecutionContext(null, executionUnits, mock(RouteContext.class)));
+        SQLLogger.logSQL(logicSQL, false, new ExecutionContext(logicSQL, executionUnits, mock(RouteContext.class)));
         InOrder inOrder = inOrder(logger);
         inOrder.verify(logger).info("Logic SQL: {}", new Object[]{SQL});
         inOrder.verify(logger).info("SQLStatement: {}", new Object[]{null});
@@ -81,7 +81,7 @@ public final class SQLLoggerTest {
     public void assertLogNormalSQLWithParameters() {
         List<Object> parameters = executionUnits.iterator().next().getSqlUnit().getParameters();
         parameters.add("parameter");
-        SQLLogger.logSQL(logicSQL, false, new ExecutionContext(null, executionUnits, mock(RouteContext.class)));
+        SQLLogger.logSQL(logicSQL, false, new ExecutionContext(logicSQL, executionUnits, mock(RouteContext.class)));
         InOrder inOrder = inOrder(logger);
         inOrder.verify(logger).info("Logic SQL: {}", new Object[]{SQL});
         inOrder.verify(logger).info("SQLStatement: {}", new Object[]{null});
@@ -92,7 +92,7 @@ public final class SQLLoggerTest {
     
     @Test
     public void assertLogSimpleSQL() {
-        SQLLogger.logSQL(logicSQL, true, new ExecutionContext(null, executionUnits, mock(RouteContext.class)));
+        SQLLogger.logSQL(logicSQL, true, new ExecutionContext(logicSQL, executionUnits, mock(RouteContext.class)));
         InOrder inOrder = inOrder(logger);
         inOrder.verify(logger).info("Logic SQL: {}", new Object[]{SQL});
         inOrder.verify(logger).info("SQLStatement: {}", new Object[]{null});
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
index 47f31c9..be53a7d 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.driver.executor;
 
 import lombok.Getter;
 import org.apache.shardingsphere.driver.executor.callback.ExecuteQueryCallback;
+import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroupContext;
@@ -57,15 +58,15 @@ public final class DriverJDBCExecutor {
      * Execute query.
      *
      * @param executionGroupContext execution group context
-     * @param sqlStatementContext SQL statement context
+     * @param logicSQL logic SQL
      * @param callback execute query callback
      * @return query results
      * @throws SQLException SQL exception
      */
-    public List<QueryResult> executeQuery(final ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext, 
-                                          final SQLStatementContext<?> sqlStatementContext, final ExecuteQueryCallback callback) throws SQLException {
+    public List<QueryResult> executeQuery(final ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext,
+                                          final LogicSQL logicSQL, final ExecuteQueryCallback callback) throws SQLException {
         try {
-            ExecuteProcessEngine.initialize(sqlStatementContext, executionGroupContext, metaDataContexts.getProps());
+            ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getProps());
             List<QueryResult> result = jdbcExecutor.execute(executionGroupContext, callback);
             ExecuteProcessEngine.finish(executionGroupContext.getExecutionID());
             return result;
@@ -78,16 +79,17 @@ public final class DriverJDBCExecutor {
      * Execute update.
      *
      * @param executionGroupContext execution group context
-     * @param sqlStatementContext SQL statement context
+     * @param logicSQL logic SQL
      * @param routeUnits route units
      * @param callback JDBC executor callback
      * @return effected records count
      * @throws SQLException SQL exception
      */
     public int executeUpdate(final ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext,
-                             final SQLStatementContext<?> sqlStatementContext, final Collection<RouteUnit> routeUnits, final JDBCExecutorCallback<Integer> callback) throws SQLException {
+                             final LogicSQL logicSQL, final Collection<RouteUnit> routeUnits, final JDBCExecutorCallback<Integer> callback) throws SQLException {
         try {
-            ExecuteProcessEngine.initialize(sqlStatementContext, executionGroupContext, metaDataContexts.getProps());
+            ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getProps());
+            SQLStatementContext<?> sqlStatementContext = logicSQL.getSqlStatementContext();
             List<Integer> results = jdbcLockEngine.execute(executionGroupContext, sqlStatementContext, routeUnits, callback);
             int result = isNeedAccumulate(metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules(), sqlStatementContext) ? accumulate(results) : results.get(0);
             ExecuteProcessEngine.finish(executionGroupContext.getExecutionID());
@@ -109,17 +111,17 @@ public final class DriverJDBCExecutor {
      * Execute SQL.
      *
      * @param executionGroupContext execution group context
-     * @param sqlStatementContext SQL statement context
+     * @param logicSQL logic SQL
      * @param routeUnits route units
      * @param callback JDBC executor callback
      * @return return true if is DQL, false if is DML
      * @throws SQLException SQL exception
      */
-    public boolean execute(final ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext, final SQLStatementContext<?> sqlStatementContext,
+    public boolean execute(final ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext, final LogicSQL logicSQL,
                            final Collection<RouteUnit> routeUnits, final JDBCExecutorCallback<Boolean> callback) throws SQLException {
         try {
-            ExecuteProcessEngine.initialize(sqlStatementContext, executionGroupContext, metaDataContexts.getProps());
-            List<Boolean> results = jdbcLockEngine.execute(executionGroupContext, sqlStatementContext, routeUnits, callback);
+            ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getProps());
+            List<Boolean> results = jdbcLockEngine.execute(executionGroupContext, logicSQL.getSqlStatementContext(), routeUnits, callback);
             boolean result = null != results && !results.isEmpty() && null != results.get(0) && results.get(0);
             ExecuteProcessEngine.finish(executionGroupContext.getExecutionID());
             return result;
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
index 82af5b5..c481f13 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
@@ -194,7 +194,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     
     private List<QueryResult> executeQuery0() throws SQLException {
         if (metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules().stream().anyMatch(each -> each instanceof RawExecutionRule)) {
-            return rawExecutor.execute(createRawExecutionGroupContext(), executionContext.getSqlStatementContext(),
+            return rawExecutor.execute(createRawExecutionGroupContext(), executionContext.getLogicSQL(),
                     new RawSQLExecutorCallback()).stream().map(each -> (QueryResult) each).collect(Collectors.toList());
         }
         if (executionContext.getRouteContext().isFederated()) {
@@ -202,7 +202,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         }
         ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext = createExecutionGroupContext();
         cacheStatements(executionGroupContext.getInputGroups());
-        return driverJDBCExecutor.executeQuery(executionGroupContext, executionContext.getSqlStatementContext(), 
+        return driverJDBCExecutor.executeQuery(executionGroupContext, executionContext.getLogicSQL(),
                 new PreparedStatementExecuteQueryCallback(metaDataContexts.getDefaultMetaData().getResource().getDatabaseType(), sqlStatement, SQLExecutorExceptionHandler.isExceptionThrown()));
     }
     
@@ -227,13 +227,13 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
             clearPrevious();
             executionContext = createExecutionContext();
             if (metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules().stream().anyMatch(each -> each instanceof RawExecutionRule)) {
-                Collection<ExecuteResult> executeResults = rawExecutor.execute(createRawExecutionGroupContext(), executionContext.getSqlStatementContext(), new RawSQLExecutorCallback());
+                Collection<ExecuteResult> executeResults = rawExecutor.execute(createRawExecutionGroupContext(), executionContext.getLogicSQL(), new RawSQLExecutorCallback());
                 accumulate(executeResults);
             }
             ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext = createExecutionGroupContext();
             cacheStatements(executionGroupContext.getInputGroups());
             return driverJDBCExecutor.executeUpdate(executionGroupContext,
-                    executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits(), createExecuteUpdateCallback());
+                    executionContext.getLogicSQL(), executionContext.getRouteContext().getRouteUnits(), createExecuteUpdateCallback());
         } finally {
             clearBatch();
         }
@@ -270,7 +270,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
             executionContext = createExecutionContext();
             if (metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules().stream().anyMatch(each -> each instanceof RawExecutionRule)) {
                 // TODO process getStatement
-                Collection<ExecuteResult> executeResults = rawExecutor.execute(createRawExecutionGroupContext(), executionContext.getSqlStatementContext(), new RawSQLExecutorCallback());
+                Collection<ExecuteResult> executeResults = rawExecutor.execute(createRawExecutionGroupContext(), executionContext.getLogicSQL(), new RawSQLExecutorCallback());
                 return executeResults.iterator().next() instanceof QueryResult;
             }
             if (executionContext.getRouteContext().isFederated()) {
@@ -280,7 +280,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
             ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext = createExecutionGroupContext();
             cacheStatements(executionGroupContext.getInputGroups());
             return driverJDBCExecutor.execute(executionGroupContext,
-                    executionContext.getSqlStatementContext(), executionContext.getRouteContext().getRouteUnits(), createExecuteCallback());
+                    executionContext.getLogicSQL(), executionContext.getRouteContext().getRouteUnits(), createExecuteCallback());
         } finally {
             clearBatch();
         }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
index 358830f..78e40cf 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
@@ -164,7 +164,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     
     private List<QueryResult> executeQuery0() throws SQLException {
         if (metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules().stream().anyMatch(each -> each instanceof RawExecutionRule)) {
-            return rawExecutor.execute(createRawExecutionContext(), executionContext.getSqlStatementContext(),
+            return rawExecutor.execute(createRawExecutionContext(), executionContext.getLogicSQL(),
                     new RawSQLExecutorCallback()).stream().map(each -> (QueryResult) each).collect(Collectors.toList());
         }
         if (executionContext.getRouteContext().isFederated()) {
@@ -174,7 +174,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         cacheStatements(executionGroupContext.getInputGroups());
         StatementExecuteQueryCallback callback = new StatementExecuteQueryCallback(metaDataContexts.getDefaultMetaData().getResource().getDatabaseType(),
                 executionContext.getSqlStatementContext().getSqlStatement(), SQLExecutorExceptionHandler.isExceptionThrown());
-        return driverJDBCExecutor.executeQuery(executionGroupContext, executionContext.getSqlStatementContext(), callback);
+        return driverJDBCExecutor.executeQuery(executionGroupContext, executionContext.getLogicSQL(), callback);
     }
     
     private List<QueryResult> executeFederatedQuery() throws SQLException {
@@ -197,7 +197,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         try {
             executionContext = createExecutionContext(sql);
             if (metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules().stream().anyMatch(each -> each instanceof RawExecutionRule)) {
-                return accumulate(rawExecutor.execute(createRawExecutionContext(), executionContext.getSqlStatementContext(), new RawSQLExecutorCallback()));
+                return accumulate(rawExecutor.execute(createRawExecutionContext(), executionContext.getLogicSQL(), new RawSQLExecutorCallback()));
             }
             ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext = createExecutionContext();
             cacheStatements(executionGroupContext.getInputGroups());
@@ -216,7 +216,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         try {
             executionContext = createExecutionContext(sql);
             if (metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules().stream().anyMatch(each -> each instanceof RawExecutionRule)) {
-                return accumulate(rawExecutor.execute(createRawExecutionContext(), executionContext.getSqlStatementContext(), new RawSQLExecutorCallback()));
+                return accumulate(rawExecutor.execute(createRawExecutionContext(), executionContext.getLogicSQL(), new RawSQLExecutorCallback()));
             }
             ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext = createExecutionContext();
             cacheStatements(executionGroupContext.getInputGroups());
@@ -233,7 +233,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         try {
             executionContext = createExecutionContext(sql);
             if (metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules().stream().anyMatch(each -> each instanceof RawExecutionRule)) {
-                return accumulate(rawExecutor.execute(createRawExecutionContext(), executionContext.getSqlStatementContext(), new RawSQLExecutorCallback()));
+                return accumulate(rawExecutor.execute(createRawExecutionContext(), executionContext.getLogicSQL(), new RawSQLExecutorCallback()));
             }
             ExecutionGroupContext<JDBCExecutionUnit> executionGroups = createExecutionContext();
             cacheStatements(executionGroups.getInputGroups());
@@ -250,7 +250,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         try {
             executionContext = createExecutionContext(sql);
             if (metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules().stream().anyMatch(each -> each instanceof RawExecutionRule)) {
-                return accumulate(rawExecutor.execute(createRawExecutionContext(), executionContext.getSqlStatementContext(), new RawSQLExecutorCallback()));
+                return accumulate(rawExecutor.execute(createRawExecutionContext(), executionContext.getLogicSQL(), new RawSQLExecutorCallback()));
             }
             ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext = createExecutionContext();
             cacheStatements(executionGroupContext.getInputGroups());
@@ -277,7 +277,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
                 return Optional.empty();
             }
         };
-        return driverJDBCExecutor.executeUpdate(executionGroupContext, sqlStatementContext, routeUnits, callback);
+        return driverJDBCExecutor.executeUpdate(executionGroupContext, executionContext.getLogicSQL(), routeUnits, callback);
     }
     
     private int accumulate(final Collection<ExecuteResult> results) {
@@ -329,7 +329,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
                 return Optional.empty();
             }
         };
-        return driverJDBCExecutor.execute(executionGroupContext, executionContext.getSqlStatementContext(), routeUnits, jdbcExecutorCallback);
+        return driverJDBCExecutor.execute(executionGroupContext, executionContext.getLogicSQL(), routeUnits, jdbcExecutorCallback);
     }
     
     private boolean execute0(final String sql, final ExecuteCallback callback) throws SQLException {
@@ -337,7 +337,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
             executionContext = createExecutionContext(sql);
             if (metaDataContexts.getDefaultMetaData().getRuleMetaData().getRules().stream().anyMatch(each -> each instanceof RawExecutionRule)) {
                 // TODO process getStatement
-                Collection<ExecuteResult> results = rawExecutor.execute(createRawExecutionContext(), executionContext.getSqlStatementContext(), new RawSQLExecutorCallback());
+                Collection<ExecuteResult> results = rawExecutor.execute(createRawExecutionContext(), executionContext.getLogicSQL(), new RawSQLExecutorCallback());
                 return results.iterator().next() instanceof QueryResult;
             }
             if (executionContext.getRouteContext().isFederated()) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java
index 0df7432..f2bbf85 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java
@@ -139,7 +139,7 @@ public final class ProxySQLExecutor {
             return getSaneExecuteResults(executionContext, ex);
         }
         // TODO handle query header
-        return rawExecutor.execute(executionGroupContext, executionContext.getSqlStatementContext(), new RawSQLExecutorCallback());
+        return rawExecutor.execute(executionGroupContext, executionContext.getLogicSQL(), new RawSQLExecutorCallback());
     }
     
     private Collection<ExecuteResult> federateExecute(final ExecutionContext executionContext, final boolean isReturnGeneratedKeys, final boolean isExceptionThrown) throws SQLException {
@@ -170,7 +170,7 @@ public final class ProxySQLExecutor {
         } catch (final SQLException ex) {
             return getSaneExecuteResults(executionContext, ex);
         }
-        return jdbcExecutor.execute(executionContext.getSqlStatementContext(), executionGroupContext, isReturnGeneratedKeys, isExceptionThrown);
+        return jdbcExecutor.execute(executionContext.getLogicSQL(), executionGroupContext, isReturnGeneratedKeys, isExceptionThrown);
     }
     
     private Collection<ExecuteResult> getSaneExecuteResults(final ExecutionContext executionContext, final SQLException originalException) throws SQLException {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutor.java
index d6e12c6..0676a72 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutor.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.proxy.backend.communication.jdbc.executor;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
@@ -53,19 +54,20 @@ public final class ProxyJDBCExecutor {
     /**
      * Execute.
      * 
-     * @param context SQL statement context
+     * @param logicSQL logic SQL
      * @param executionGroupContext execution group context
      * @param isReturnGeneratedKeys is return generated keys
      * @param isExceptionThrown is exception thrown
      * @return execute results
      * @throws SQLException SQL exception
      */
-    public Collection<ExecuteResult> execute(final SQLStatementContext<?> context, final ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext,
+    public Collection<ExecuteResult> execute(final LogicSQL logicSQL, final ExecutionGroupContext<JDBCExecutionUnit> executionGroupContext,
                                              final boolean isReturnGeneratedKeys, final boolean isExceptionThrown) throws SQLException {
         try {
             MetaDataContexts metaDataContexts = ProxyContext.getInstance().getMetaDataContexts();
             DatabaseType databaseType = metaDataContexts.getMetaData(backendConnection.getSchemaName()).getResource().getDatabaseType();
-            ExecuteProcessEngine.initialize(context, executionGroupContext, metaDataContexts.getProps());
+            ExecuteProcessEngine.initialize(logicSQL, executionGroupContext, metaDataContexts.getProps());
+            SQLStatementContext<?> context = logicSQL.getSqlStatementContext();
             Collection<ExecuteResult> result = jdbcExecutor.execute(executionGroupContext,
                     ProxyJDBCExecutorCallbackFactory.newInstance(type, databaseType, context.getSqlStatement(), databaseCommunicationEngine, isReturnGeneratedKeys, isExceptionThrown, true),
                     ProxyJDBCExecutorCallbackFactory.newInstance(type, databaseType, context.getSqlStatement(), databaseCommunicationEngine, isReturnGeneratedKeys, isExceptionThrown, false));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowProcessListExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowProcessListExecutor.java
index 1f98c10..49bcb88 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowProcessListExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/mysql/executor/ShowProcessListExecutor.java
@@ -100,8 +100,8 @@ public final class ShowProcessListExecutor implements DatabaseAdminQueryExecutor
             int processDoneCount = processContext.getUnitStatuses().stream().map(processUnit -> ExecuteProcessConstants.EXECUTE_STATUS_DONE == processUnit.getStatus() ? 1 : 0).reduce(0, Integer::sum);
             String statePrefix = "Executing ";
             rowValues.add(statePrefix + processDoneCount + "/" + processContext.getUnitStatuses().size());
-            // TODO Show original SQL
-            rowValues.add("");
+            String sql = processContext.getSql();
+            rowValues.add(sql != null ? sql : "");
             return new MemoryQueryResultDataRow(rowValues);
         }).collect(Collectors.toList());
         return new RawMemoryQueryResult(queryResultMetaData, rows);