Remove unused InventoryDumperContext fields (#36753)

* Remove unused InventoryDumperContext.transactionIsolation

* Remove unused InventoryDumperContext.querySQL and queryParams
diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/inventory/InventoryDumper.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/inventory/InventoryDumper.java
index b88bc8b..27b62bb 100644
--- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/inventory/InventoryDumper.java
+++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/inventory/InventoryDumper.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.data.pipeline.core.ingest.dumper.inventory;
 
-import com.google.common.base.Strings;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.data.pipeline.core.channel.PipelineChannel;
 import org.apache.shardingsphere.data.pipeline.core.constant.PipelineSQLOperationType;
@@ -106,7 +105,7 @@
             return;
         }
         try (Connection connection = dataSource.getConnection()) {
-            if (!Strings.isNullOrEmpty(dumperContext.getQuerySQL()) || !dumperContext.hasUniqueKey() || isPrimaryKeyWithoutRange(position)) {
+            if (!dumperContext.hasUniqueKey() || isPrimaryKeyWithoutRange(position)) {
                 dumpWithStreamingQuery(connection);
             } else {
                 dumpByPage(connection);
@@ -123,12 +122,8 @@
         return position instanceof PrimaryKeyIngestPosition && null == ((PrimaryKeyIngestPosition<?>) position).getBeginValue() && null == ((PrimaryKeyIngestPosition<?>) position).getEndValue();
     }
     
-    @SuppressWarnings("MagicConstant")
     private void dumpByPage(final Connection connection) throws SQLException {
         log.info("Start to dump inventory data by page, dataSource={}, actualTable={}", dumperContext.getCommonContext().getDataSourceName(), dumperContext.getActualTableName());
-        if (null != dumperContext.getTransactionIsolation()) {
-            connection.setTransactionIsolation(dumperContext.getTransactionIsolation());
-        }
         boolean firstQuery = true;
         AtomicLong rowCount = new AtomicLong();
         IngestPosition position = dumperContext.getCommonContext().getPosition();
@@ -250,18 +245,10 @@
         return ((DataRecord) dataRecords.get(index)).getUniqueKeyValue().iterator().next();
     }
     
-    @SuppressWarnings("MagicConstant")
     private void dumpWithStreamingQuery(final Connection connection) throws SQLException {
         int batchSize = dumperContext.getBatchSize();
         DatabaseType databaseType = dumperContext.getCommonContext().getDataSourceConfig().getDatabaseType();
-        if (null != dumperContext.getTransactionIsolation()) {
-            connection.setTransactionIsolation(dumperContext.getTransactionIsolation());
-        }
-        if (null == dumperContext.getQuerySQL()) {
-            fetchAllQuery(connection, databaseType, batchSize);
-        } else {
-            designatedParametersQuery(connection, databaseType, batchSize);
-        }
+        fetchAllQuery(connection, databaseType, batchSize);
     }
     
     private void fetchAllQuery(final Connection connection, final DatabaseType databaseType, final int batchSize) throws SQLException {
@@ -277,24 +264,6 @@
         log.info("End to fetch all inventory data with streaming query, dataSource={}, actualTable={}", dumperContext.getCommonContext().getDataSourceName(), dumperContext.getActualTableName());
     }
     
-    private void designatedParametersQuery(final Connection connection, final DatabaseType databaseType, final int batchSize) throws SQLException {
-        log.info("Start to dump inventory data with designated parameters query, dataSource={}, actualTable={}", dumperContext.getCommonContext().getDataSourceName(),
-                dumperContext.getActualTableName());
-        try (PreparedStatement statement = JDBCStreamQueryBuilder.build(databaseType, connection, dumperContext.getQuerySQL(), batchSize)) {
-            runningStatement.set(statement);
-            for (int i = 0; i < dumperContext.getQueryParams().size(); i++) {
-                statement.setObject(i + 1, dumperContext.getQueryParams().get(i));
-            }
-            try (ResultSet resultSet = statement.executeQuery()) {
-                consumeResultSetToChannel(resultSet, batchSize);
-            } finally {
-                runningStatement.set(null);
-            }
-        }
-        log.info("End to dump inventory data with designated parameters query, dataSource={}, actualTable={}", dumperContext.getCommonContext().getDataSourceName(),
-                dumperContext.getActualTableName());
-    }
-    
     private void consumeResultSetToChannel(final ResultSet resultSet, final int batchSize) throws SQLException {
         long rowCount = 0;
         JobRateLimitAlgorithm rateLimitAlgorithm = dumperContext.getRateLimitAlgorithm();
diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/inventory/InventoryDumperContext.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/inventory/InventoryDumperContext.java
index 29cb112..6283815 100644
--- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/inventory/InventoryDumperContext.java
+++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/inventory/InventoryDumperContext.java
@@ -50,12 +50,6 @@
     
     private List<String> insertColumnNames;
     
-    private String querySQL;
-    
-    private List<Object> queryParams;
-    
-    private Integer transactionIsolation;
-    
     private int shardingItem;
     
     private int batchSize = 1000;