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 2023/03/22 05:18:21 UTC

[shardingsphere] branch master updated: Implement batch execution for ShardingSphereStatement (#24717)

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 959f45c07fc Implement batch execution for ShardingSphereStatement (#24717)
959f45c07fc is described below

commit 959f45c07fc23ad7fd91014c91b0c27d221517b8
Author: 吴伟杰 <wu...@apache.org>
AuthorDate: Wed Mar 22 13:18:04 2023 +0800

    Implement batch execution for ShardingSphereStatement (#24717)
---
 .../executor/batch/BatchStatementExecutor.java     | 67 ++++++++++++++++++++++
 .../core/statement/ShardingSphereStatement.java    | 19 ++++++
 ...tractUnsupportedOperationPreparedStatement.java |  5 ++
 .../AbstractUnsupportedOperationStatement.java     | 15 -----
 .../circuit/statement/CircuitBreakerStatement.java | 14 +++++
 .../executor/batch/BatchStatementExecutorTest.java | 45 +++++++++++++++
 .../statement/ShardingSphereStatementTest.java     | 14 ++++-
 .../UnsupportedOperationStatementTest.java         | 15 -----
 8 files changed, 163 insertions(+), 31 deletions(-)

diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchStatementExecutor.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchStatementExecutor.java
new file mode 100644
index 00000000000..81a703f9ead
--- /dev/null
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchStatementExecutor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.driver.executor.batch;
+
+import lombok.RequiredArgsConstructor;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Batch executor for {@link Statement}.
+ */
+@RequiredArgsConstructor
+public final class BatchStatementExecutor {
+    
+    private final Statement statement;
+    
+    private final List<String> batchedSQLs = new LinkedList<>();
+    
+    /**
+     * Add SQL for batched execution.
+     *
+     * @param sql SQL 
+     */
+    public void addBatch(final String sql) {
+        batchedSQLs.add(sql);
+    }
+    
+    /**
+     * Execute batched SQLs.
+     *
+     * @return execute results
+     * @throws SQLException SQL exception
+     */
+    public int[] executeBatch() throws SQLException {
+        int[] result = new int[batchedSQLs.size()];
+        int index = 0;
+        for (String each : batchedSQLs) {
+            result[index++] = statement.executeUpdate(each);
+        }
+        return result;
+    }
+    
+    /**
+     * Clear batched SQLs.
+     */
+    public void clear() {
+        batchedSQLs.clear();
+    }
+}
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
index 67a1e8a26d6..855a5b107b3 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
@@ -22,6 +22,7 @@ import lombok.AccessLevel;
 import lombok.Getter;
 import org.apache.shardingsphere.dialect.SQLExceptionTransformEngine;
 import org.apache.shardingsphere.driver.executor.DriverExecutor;
+import org.apache.shardingsphere.driver.executor.batch.BatchStatementExecutor;
 import org.apache.shardingsphere.driver.executor.callback.ExecuteCallback;
 import org.apache.shardingsphere.driver.executor.callback.ExecuteUpdateCallback;
 import org.apache.shardingsphere.driver.executor.callback.impl.StatementExecuteQueryCallback;
@@ -126,6 +127,8 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     @Getter(AccessLevel.PROTECTED)
     private final StatementManager statementManager;
     
+    private final BatchStatementExecutor batchStatementExecutor;
+    
     private boolean returnGeneratedKeys;
     
     private ExecutionContext executionContext;
@@ -153,6 +156,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         kernelProcessor = new KernelProcessor();
         trafficRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class);
         statementManager = new StatementManager();
+        batchStatementExecutor = new BatchStatementExecutor(this);
     }
     
     @Override
@@ -507,6 +511,21 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         statements.clear();
     }
     
+    @Override
+    public void addBatch(final String sql) throws SQLException {
+        batchStatementExecutor.addBatch(sql);
+    }
+    
+    @Override
+    public void clearBatch() {
+        batchStatementExecutor.clear();
+    }
+    
+    @Override
+    public int[] executeBatch() throws SQLException {
+        return batchStatementExecutor.executeBatch();
+    }
+    
     private QueryContext createQueryContext(final String originSQL) {
         SQLParserRule sqlParserRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(SQLParserRule.class);
         String sql = sqlParserRule.isSqlCommentParseEnabled() ? originSQL : SQLHintUtils.removeHint(originSQL);
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/unsupported/AbstractUnsupportedOperationPreparedStatement.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/unsupported/AbstractUnsupportedOperationPreparedStatement.java
index 1d14b05e395..b3c76b21ead 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/unsupported/AbstractUnsupportedOperationPreparedStatement.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/unsupported/AbstractUnsupportedOperationPreparedStatement.java
@@ -34,6 +34,11 @@ import java.sql.SQLFeatureNotSupportedException;
  */
 public abstract class AbstractUnsupportedOperationPreparedStatement extends AbstractStatementAdapter implements PreparedStatement {
     
+    @Override
+    public final void addBatch(final String sql) throws SQLException {
+        throw new SQLFeatureNotSupportedException("addBatch sql in PreparedStatement");
+    }
+    
     @Override
     public final ResultSetMetaData getMetaData() throws SQLException {
         throw new SQLFeatureNotSupportedException("getMetaData");
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/unsupported/AbstractUnsupportedOperationStatement.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/unsupported/AbstractUnsupportedOperationStatement.java
index cb763d3dc9f..b6c10685931 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/unsupported/AbstractUnsupportedOperationStatement.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/unsupported/AbstractUnsupportedOperationStatement.java
@@ -28,21 +28,6 @@ import java.sql.Statement;
  */
 public abstract class AbstractUnsupportedOperationStatement extends WrapperAdapter implements Statement {
     
-    @Override
-    public final void addBatch(final String sql) throws SQLException {
-        throw new SQLFeatureNotSupportedException("addBatch sql");
-    }
-    
-    @Override
-    public void clearBatch() throws SQLException {
-        throw new SQLFeatureNotSupportedException("clearBatch");
-    }
-    
-    @Override
-    public int[] executeBatch() throws SQLException {
-        throw new SQLFeatureNotSupportedException("executeBatch");
-    }
-    
     @Override
     public final void closeOnCompletion() throws SQLException {
         throw new SQLFeatureNotSupportedException("closeOnCompletion");
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/state/circuit/statement/CircuitBreakerStatement.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/state/circuit/statement/CircuitBreakerStatement.java
index 3c4909d7f74..31dd09b0067 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/state/circuit/statement/CircuitBreakerStatement.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/state/circuit/statement/CircuitBreakerStatement.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.driver.state.circuit.connection.CircuitBreakerC
 
 import java.sql.Connection;
 import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.sql.SQLWarning;
 
 /**
@@ -115,6 +116,19 @@ public final class CircuitBreakerStatement extends AbstractUnsupportedOperationS
         return ResultSet.TYPE_FORWARD_ONLY;
     }
     
+    @Override
+    public void addBatch(final String sql) {
+    }
+    
+    @Override
+    public void clearBatch() {
+    }
+    
+    @Override
+    public int[] executeBatch() throws SQLException {
+        return new int[0];
+    }
+    
     @Override
     public Connection getConnection() {
         return new CircuitBreakerConnection();
diff --git a/jdbc/core/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchStatementExecutorTest.java b/jdbc/core/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchStatementExecutorTest.java
new file mode 100644
index 00000000000..10d3dc0a2d2
--- /dev/null
+++ b/jdbc/core/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchStatementExecutorTest.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.driver.executor.batch;
+
+import org.junit.jupiter.api.Test;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+final class BatchStatementExecutorTest {
+    
+    @Test
+    void assertExecuteBatchAndClear() throws SQLException {
+        Statement statement = mock(Statement.class);
+        when(statement.executeUpdate(anyString())).thenReturn(1, 2);
+        BatchStatementExecutor executor = new BatchStatementExecutor(statement);
+        executor.addBatch("UPDATE t SET col=1 WHERE id=1");
+        executor.addBatch("UPDATE t SET col=10 WHERE id=2 OR id=3");
+        int[] actual = executor.executeBatch();
+        assertThat(actual, is(new int[]{1, 2}));
+        executor.clear();
+        assertThat(executor.executeBatch(), is(new int[0]));
+    }
+}
diff --git a/jdbc/core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatementTest.java b/jdbc/core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatementTest.java
index 01bd507c7b0..6aa9f5d68c8 100644
--- a/jdbc/core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatementTest.java
+++ b/jdbc/core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatementTest.java
@@ -27,9 +27,9 @@ import java.sql.SQLException;
 import java.sql.Statement;
 
 import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
@@ -126,4 +126,16 @@ public final class ShardingSphereStatementTest extends AbstractShardingSphereDat
             assertThat(resultSet.getString(1), is(DefaultDatabase.LOGIC_NAME));
         }
     }
+    
+    @Test
+    public void assertExecuteBatch() throws SQLException {
+        try (Connection connection = getShardingSphereDataSource().getConnection(); Statement statement = connection.createStatement()) {
+            statement.addBatch("UPDATE t_order SET status = 'closed' WHERE order_id = 1001");
+            statement.addBatch("UPDATE t_order SET status = 'closed' WHERE order_id = 1100 OR order_id = 1101");
+            statement.addBatch("DELETE FROM t_order WHERE order_id = 1000");
+            assertThat(statement.executeBatch(), is(new int[]{1, 2, 1}));
+            statement.clearBatch();
+            assertThat(statement.executeBatch(), is(new int[0]));
+        }
+    }
 }
diff --git a/jdbc/core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationStatementTest.java b/jdbc/core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationStatementTest.java
index 9314bad4902..323c506d89e 100644
--- a/jdbc/core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationStatementTest.java
+++ b/jdbc/core/src/test/java/org/apache/shardingsphere/driver/jdbc/unsupported/UnsupportedOperationStatementTest.java
@@ -55,21 +55,6 @@ public final class UnsupportedOperationStatementTest {
         shardingSphereStatement = new ShardingSphereStatement(connection);
     }
     
-    @Test
-    public void assertAddBatch() {
-        assertThrows(SQLFeatureNotSupportedException.class, () -> shardingSphereStatement.addBatch(""));
-    }
-    
-    @Test
-    public void assertClearBatch() {
-        assertThrows(SQLFeatureNotSupportedException.class, () -> shardingSphereStatement.clearBatch());
-    }
-    
-    @Test
-    public void assertExecuteBatch() {
-        assertThrows(SQLFeatureNotSupportedException.class, () -> shardingSphereStatement.executeBatch());
-    }
-    
     @Test
     public void assertCloseOnCompletion() {
         assertThrows(SQLFeatureNotSupportedException.class, () -> shardingSphereStatement.closeOnCompletion());