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 2020/11/29 12:46:47 UTC

[shardingsphere] branch master updated: Merge ProxySQLExecutor and JDBCExecutor (#8409)

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 cd0381b  Merge ProxySQLExecutor and JDBCExecutor (#8409)
cd0381b is described below

commit cd0381b82edde27773bf0147df06862e626b4828
Author: Liang Zhang <te...@163.com>
AuthorDate: Sun Nov 29 20:46:20 2020 +0800

    Merge ProxySQLExecutor and JDBCExecutor (#8409)
    
    * Rename ProxySQLExecutor
    
    * Move ProxySQLExecutor to communication root package
    
    * Rename jdbc.executor
    
    * For code format
    
    * Refactor ProxySQLExecutor
---
 .../DatabaseCommunicationEngineFactory.java        |  9 ++---
 ...DBCExecuteEngine.java => ProxySQLExecutor.java} | 36 +++++++++++------
 .../jdbc/JDBCDatabaseCommunicationEngine.java      | 12 +++---
 .../jdbc/execute/SQLExecuteEngine.java             | 45 ----------------------
 .../ProxyJDBCExecutorCallback.java                 |  2 +-
 .../engine/jdbc => executor}/RawProxyExecutor.java |  2 +-
 6 files changed, 37 insertions(+), 69 deletions(-)

diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactory.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactory.java
index 5fd1212..59b0962 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactory.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactory.java
@@ -25,7 +25,6 @@ import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.JDBCDatabaseCommunicationEngine;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.execute.engine.jdbc.JDBCExecuteEngine;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor.impl.PreparedStatementAccessor;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor.impl.StatementAccessor;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
@@ -63,8 +62,8 @@ public final class DatabaseCommunicationEngineFactory {
     public DatabaseCommunicationEngine newTextProtocolInstance(final SQLStatement sqlStatement, final String sql, final BackendConnection backendConnection) {
         ShardingSphereMetaData metaData = ProxyContext.getInstance().getMetaData(backendConnection.getSchemaName());
         LogicSQL logicSQL = createLogicSQL(sqlStatement, sql, Collections.emptyList(), metaData);
-        JDBCExecuteEngine jdbcExecuteEngine = new JDBCExecuteEngine(backendConnection, new StatementAccessor());
-        return new JDBCDatabaseCommunicationEngine(logicSQL, metaData, jdbcExecuteEngine);
+        ProxySQLExecutor proxySQLExecutor = new ProxySQLExecutor(backendConnection, new StatementAccessor());
+        return new JDBCDatabaseCommunicationEngine(logicSQL, metaData, proxySQLExecutor);
     }
     
     /**
@@ -79,8 +78,8 @@ public final class DatabaseCommunicationEngineFactory {
     public DatabaseCommunicationEngine newBinaryProtocolInstance(final SQLStatement sqlStatement, final String sql, final List<Object> parameters, final BackendConnection backendConnection) {
         ShardingSphereMetaData metaData = ProxyContext.getInstance().getMetaData(backendConnection.getSchemaName());
         LogicSQL logicSQL = createLogicSQL(sqlStatement, sql, new ArrayList<>(parameters), metaData);
-        JDBCExecuteEngine jdbcExecuteEngine = new JDBCExecuteEngine(backendConnection, new PreparedStatementAccessor());
-        return new JDBCDatabaseCommunicationEngine(logicSQL, metaData, jdbcExecuteEngine);
+        ProxySQLExecutor proxySQLExecutor = new ProxySQLExecutor(backendConnection, new PreparedStatementAccessor());
+        return new JDBCDatabaseCommunicationEngine(logicSQL, metaData, proxySQLExecutor);
     }
     
     private LogicSQL createLogicSQL(final SQLStatement sqlStatement, final String sql, final List<Object> parameters, final ShardingSphereMetaData metaData) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/JDBCExecuteEngine.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java
similarity index 91%
rename from shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/JDBCExecuteEngine.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java
index b11c1bb..58f2721 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/JDBCExecuteEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/ProxySQLExecutor.java
@@ -15,13 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.execute.engine.jdbc;
+package org.apache.shardingsphere.proxy.backend.communication;
 
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
@@ -31,7 +32,6 @@ import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.J
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.raw.RawSQLExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.raw.callback.RawSQLExecutorCallback;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.ExecuteResult;
-import org.apache.shardingsphere.proxy.backend.response.query.QueryHeader;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.DriverExecutionPrepareEngine;
 import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.StatementOption;
@@ -41,13 +41,15 @@ import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.type.RawExecutionRule;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.execute.SQLExecuteEngine;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.executor.ProxyJDBCExecutorCallback;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.executor.RawProxyExecutor;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.accessor.JDBCAccessor;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.transaction.TransactionStatus;
 import org.apache.shardingsphere.proxy.backend.context.BackendExecutorContext;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.exception.TableModifyInTransactionException;
 import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
+import org.apache.shardingsphere.proxy.backend.response.query.QueryHeader;
 import org.apache.shardingsphere.proxy.backend.response.query.QueryHeaderBuilder;
 import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
 import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
@@ -65,9 +67,9 @@ import java.util.Collection;
 import java.util.List;
 
 /**
- * SQL Execute engine for JDBC.
+ * Proxy SQL Executor.
  */
-public final class JDBCExecuteEngine implements SQLExecuteEngine {
+public final class ProxySQLExecutor {
     
     private final BackendConnection backendConnection;
     
@@ -77,14 +79,20 @@ public final class JDBCExecuteEngine implements SQLExecuteEngine {
     
     private final RawProxyExecutor rawExecutor;
     
-    public JDBCExecuteEngine(final BackendConnection backendConnection, final JDBCAccessor accessor) {
+    public ProxySQLExecutor(final BackendConnection backendConnection, final JDBCAccessor accessor) {
         this.backendConnection = backendConnection;
         this.accessor = accessor;
-        jdbcExecutor = new JDBCExecutor(BackendExecutorContext.getInstance().getExecutorEngine(), backendConnection.isSerialExecute());
-        rawExecutor = new RawProxyExecutor(BackendExecutorContext.getInstance().getExecutorEngine(), backendConnection.isSerialExecute());
+        ExecutorEngine executorEngine = BackendExecutorContext.getInstance().getExecutorEngine();
+        boolean isSerialExecute = backendConnection.isSerialExecute();
+        jdbcExecutor = new JDBCExecutor(executorEngine, isSerialExecute);
+        rawExecutor = new RawProxyExecutor(executorEngine, isSerialExecute);
     }
     
-    @Override
+    /**
+     * Check execute prerequisites.
+     *
+     * @param executionContext execution context
+     */
     public void checkExecutePrerequisites(final ExecutionContext executionContext) {
         if (isExecuteDDLInXATransaction(executionContext.getSqlStatementContext().getSqlStatement())) {
             throw new TableModifyInTransactionException(getTableName(executionContext.getSqlStatementContext()));
@@ -103,7 +111,13 @@ public final class JDBCExecuteEngine implements SQLExecuteEngine {
         return "unknown_table";
     }
     
-    @Override
+    /**
+     * Execute SQL.
+     *
+     * @param executionContext execution context
+     * @return execute response
+     * @throws SQLException SQL exception
+     */
     public BackendResponse execute(final ExecutionContext executionContext) throws SQLException {
         Collection<ExecuteResult> executeResults = execute(executionContext,
                 executionContext.getSqlStatementContext().getSqlStatement() instanceof InsertStatement, SQLExecutorExceptionHandler.isExceptionThrown());
@@ -173,7 +187,7 @@ public final class JDBCExecuteEngine implements SQLExecuteEngine {
         }
         return result;
     }
-
+    
     private boolean hasSelectExpandProjections(final SQLStatementContext<?> sqlStatementContext) {
         return sqlStatementContext instanceof SelectStatementContext && !((SelectStatementContext) sqlStatementContext).getProjectionsContext().getExpandProjections().isEmpty();
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
index a3dab98..7b286f7 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
@@ -22,10 +22,9 @@ import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.context.kernel.KernelProcessor;
-import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
+import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
 import org.apache.shardingsphere.infra.executor.sql.log.SQLLogger;
-import org.apache.shardingsphere.proxy.backend.response.query.QueryHeader;
 import org.apache.shardingsphere.infra.merge.MergeEngine;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
@@ -37,10 +36,11 @@ import org.apache.shardingsphere.infra.metadata.schema.refresher.spi.SchemaChang
 import org.apache.shardingsphere.infra.rule.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngine;
-import org.apache.shardingsphere.proxy.backend.communication.jdbc.execute.SQLExecuteEngine;
+import org.apache.shardingsphere.proxy.backend.communication.ProxySQLExecutor;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
 import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
+import org.apache.shardingsphere.proxy.backend.response.query.QueryHeader;
 import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
 import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
@@ -63,7 +63,7 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
     
     private final ShardingSphereMetaData metaData;
     
-    private final SQLExecuteEngine sqlExecuteEngine;
+    private final ProxySQLExecutor proxySQLExecutor;
     
     private final KernelProcessor kernelProcessor = new KernelProcessor();
     
@@ -88,8 +88,8 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
         if (executionContext.getExecutionUnits().isEmpty()) {
             return new UpdateResponse();
         }
-        sqlExecuteEngine.checkExecutePrerequisites(executionContext);
-        response = sqlExecuteEngine.execute(executionContext);
+        proxySQLExecutor.checkExecutePrerequisites(executionContext);
+        response = proxySQLExecutor.execute(executionContext);
         refreshSchema(executionContext);
         return merge(executionContext.getSqlStatementContext());
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/SQLExecuteEngine.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/SQLExecuteEngine.java
deleted file mode 100644
index a68cc47..0000000
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/SQLExecuteEngine.java
+++ /dev/null
@@ -1,45 +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.communication.jdbc.execute;
-
-import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
-import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
-
-import java.sql.SQLException;
-
-/**
- * SQL Execute engine.
- */
-public interface SQLExecuteEngine {
-    
-    /**
-     * Check execute prerequisites.
-     * 
-     * @param executionContext execution context
-     */
-    void checkExecutePrerequisites(ExecutionContext executionContext);
-    
-    /**
-     * Execute SQL.
-     *
-     * @param executionContext execution context
-     * @return execute response
-     * @throws SQLException SQL exception
-     */
-    BackendResponse execute(ExecutionContext executionContext) throws SQLException;
-}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/ProxyJDBCExecutorCallback.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutorCallback.java
similarity index 99%
rename from shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/ProxyJDBCExecutorCallback.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutorCallback.java
index 1dde71a..703858a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/ProxyJDBCExecutorCallback.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/ProxyJDBCExecutorCallback.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.execute.engine.jdbc;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.executor;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.executor.sql.execute.engine.ConnectionMode;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/RawProxyExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/RawProxyExecutor.java
similarity index 99%
rename from shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/RawProxyExecutor.java
rename to shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/RawProxyExecutor.java
index 8d688cc..c1dcb98 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/execute/engine/jdbc/RawProxyExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/executor/RawProxyExecutor.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy.backend.communication.jdbc.execute.engine.jdbc;
+package org.apache.shardingsphere.proxy.backend.communication.jdbc.executor;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;