You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2023/02/14 03:32:31 UTC

[shardingsphere] branch master updated: Using streaming query at pipeline inventory dump and data consistency check (#24146)

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

zhonghongsheng 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 94c51c324e8 Using streaming query at pipeline inventory dump and data consistency check (#24146)
94c51c324e8 is described below

commit 94c51c324e89bb230c8e0ec559bfafae1b604486
Author: Xinze Guo <10...@users.noreply.github.com>
AuthorDate: Tue Feb 14 11:32:16 2023 +0800

    Using streaming query at pipeline inventory dump and data consistency check (#24146)
    
    * Using stream query at pipeline inventory dump and data consistency check
    
    * Fix ci error
    
    * Improve
    
    * Remove TODO and MAX_CONNECTIONS_SIZE_PER_QUERY set at pipeline
---
 ...DataMatchDataConsistencyCalculateAlgorithm.java |  9 ++-
 .../core/ingest/dumper/InventoryDumper.java        | 10 +++-
 .../pipeline/core/util/JDBCStreamQueryUtil.java    | 68 ++++++++++++++++++++++
 ...MatchDataConsistencyCalculateAlgorithmTest.java |  2 +-
 4 files changed, 84 insertions(+), 5 deletions(-)

diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/check/consistency/algorithm/DataMatchDataConsistencyCalculateAlgorithm.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/check/consistency/algorithm/DataMatchDataConsistencyCalculateAlgorithm.java
index 7a1ee87528f..c7206ac7a30 100644
--- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/check/consistency/algorithm/DataMatchDataConsistencyCalculateAlgorithm.java
+++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/check/consistency/algorithm/DataMatchDataConsistencyCalculateAlgorithm.java
@@ -31,9 +31,11 @@ import org.apache.shardingsphere.data.pipeline.core.check.consistency.DataConsis
 import org.apache.shardingsphere.data.pipeline.core.exception.PipelineSQLException;
 import org.apache.shardingsphere.data.pipeline.core.exception.data.PipelineTableDataConsistencyCheckLoadingFailedException;
 import org.apache.shardingsphere.data.pipeline.core.util.CloseUtil;
+import org.apache.shardingsphere.data.pipeline.core.util.JDBCStreamQueryUtil;
 import org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.ColumnValueReader;
 import org.apache.shardingsphere.data.pipeline.spi.sqlbuilder.PipelineSQLBuilder;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.util.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.util.spi.annotation.SPIDescription;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
@@ -154,9 +156,12 @@ public final class DataMatchDataConsistencyCalculateAlgorithm extends AbstractSt
     
     private void fulfillCalculationContext(final CalculationContext calculationContext, final DataConsistencyCalculateParameter param) throws SQLException {
         String sql = getQuerySQL(param);
-        PreparedStatement preparedStatement = setCurrentStatement(calculationContext.getConnection().prepareStatement(sql));
+        DatabaseType databaseType = TypedSPILoader.getService(DatabaseType.class, param.getDatabaseType());
+        PreparedStatement preparedStatement = setCurrentStatement(JDBCStreamQueryUtil.generateStreamQueryPreparedStatement(databaseType, calculationContext.getConnection(), sql));
+        if (!(databaseType instanceof MySQLDatabaseType)) {
+            preparedStatement.setFetchSize(chunkSize);
+        }
         calculationContext.setPreparedStatement(preparedStatement);
-        preparedStatement.setFetchSize(chunkSize);
         Object tableCheckPosition = param.getTableCheckPosition();
         if (null != tableCheckPosition) {
             preparedStatement.setObject(1, tableCheckPosition);
diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/InventoryDumper.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/InventoryDumper.java
index 218c293d07c..5b5a1642c93 100644
--- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/InventoryDumper.java
+++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/InventoryDumper.java
@@ -39,9 +39,12 @@ import org.apache.shardingsphere.data.pipeline.api.metadata.loader.PipelineTable
 import org.apache.shardingsphere.data.pipeline.api.metadata.model.PipelineTableMetaData;
 import org.apache.shardingsphere.data.pipeline.core.ingest.IngestDataChangeType;
 import org.apache.shardingsphere.data.pipeline.core.ingest.exception.IngestException;
+import org.apache.shardingsphere.data.pipeline.core.util.JDBCStreamQueryUtil;
 import org.apache.shardingsphere.data.pipeline.core.util.PipelineJdbcUtils;
 import org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.ColumnValueReader;
 import org.apache.shardingsphere.data.pipeline.spi.sqlbuilder.PipelineSQLBuilder;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
 import org.apache.shardingsphere.infra.util.exception.external.sql.type.generic.UnsupportedSQLOperationException;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
@@ -121,9 +124,12 @@ public final class InventoryDumper extends AbstractLifecycleExecutor implements
             dumperConfig.getRateLimitAlgorithm().intercept(JobOperationType.SELECT, 1);
         }
         int batchSize = dumperConfig.getBatchSize();
-        try (PreparedStatement preparedStatement = connection.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY)) {
+        DatabaseType databaseType = dumperConfig.getDataSourceConfig().getDatabaseType();
+        try (PreparedStatement preparedStatement = JDBCStreamQueryUtil.generateStreamQueryPreparedStatement(databaseType, connection, sql)) {
             dumpStatement = preparedStatement;
-            preparedStatement.setFetchSize(batchSize);
+            if (!(databaseType instanceof MySQLDatabaseType)) {
+                preparedStatement.setFetchSize(batchSize);
+            }
             setParameters(preparedStatement, beginUniqueKeyValue);
             try (ResultSet resultSet = preparedStatement.executeQuery()) {
                 ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/util/JDBCStreamQueryUtil.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/util/JDBCStreamQueryUtil.java
new file mode 100644
index 00000000000..36dc07c879d
--- /dev/null
+++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/util/JDBCStreamQueryUtil.java
@@ -0,0 +1,68 @@
+/*
+ * 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.data.pipeline.core.util;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.infra.database.type.dialect.OpenGaussDatabaseType;
+import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+/**
+ * JDBC stream query util.
+ */
+@Slf4j
+public final class JDBCStreamQueryUtil {
+    
+    /**
+     * Generate stream query prepared statement.
+     *
+     * @param connection connection
+     * @param databaseType database type
+     * @param sql sql
+     * @return stream query prepared statement
+     * @throws SQLException SQL exception
+     */
+    public static PreparedStatement generateStreamQueryPreparedStatement(final DatabaseType databaseType, final Connection connection, final String sql) throws SQLException {
+        if (databaseType instanceof MySQLDatabaseType) {
+            return generateMySQLStreamQueryPreparedStatement(connection, sql);
+        }
+        if (databaseType instanceof PostgreSQLDatabaseType || databaseType instanceof OpenGaussDatabaseType) {
+            return generatePostgreSQLStreamQueryPreparedStatement(connection, sql);
+        }
+        log.warn("not support {} streaming query now, pay attention to memory usage", databaseType.getType());
+        return connection.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+    }
+    
+    private static PreparedStatement generateMySQLStreamQueryPreparedStatement(final Connection connection, final String sql) throws SQLException {
+        PreparedStatement result = connection.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+        result.setFetchSize(Integer.MIN_VALUE);
+        return result;
+    }
+    
+    private static PreparedStatement generatePostgreSQLStreamQueryPreparedStatement(final Connection connection, final String sql) throws SQLException {
+        PreparedStatement result = connection.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY, ResultSet.CLOSE_CURSORS_AT_COMMIT);
+        connection.setAutoCommit(false);
+        return result;
+    }
+}
diff --git a/test/it/pipeline/src/test/java/org/apache/shardingsphere/test/it/data/pipeline/core/check/consistency/algorithm/DataMatchDataConsistencyCalculateAlgorithmTest.java b/test/it/pipeline/src/test/java/org/apache/shardingsphere/test/it/data/pipeline/core/check/consistency/algorithm/DataMatchDataConsistencyCalculateAlgorithmTest.java
index de667ecb34a..3bf1e42485e 100644
--- a/test/it/pipeline/src/test/java/org/apache/shardingsphere/test/it/data/pipeline/core/check/consistency/algorithm/DataMatchDataConsistencyCalculateAlgorithmTest.java
+++ b/test/it/pipeline/src/test/java/org/apache/shardingsphere/test/it/data/pipeline/core/check/consistency/algorithm/DataMatchDataConsistencyCalculateAlgorithmTest.java
@@ -123,6 +123,6 @@ public final class DataMatchDataConsistencyCalculateAlgorithmTest {
     private DataConsistencyCalculateParameter generateParameter(final PipelineDataSourceWrapper dataSource, final String logicTableName, final Object dataCheckPosition) {
         PipelineColumnMetaData uniqueKey = new PipelineColumnMetaData(1, "order_id", Types.INTEGER, "integer", false, true, true);
         return new DataConsistencyCalculateParameter(dataSource, null, logicTableName, Collections.emptyList(),
-                "MySQL", "MySQL", uniqueKey, dataCheckPosition);
+                "H2", "H2", uniqueKey, dataCheckPosition);
     }
 }