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

zhangliang 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 7c5c88fe3c1 Add InventoryDumperContextSplitter (#32590)
7c5c88fe3c1 is described below

commit 7c5c88fe3c19733584d54d3119209bb35f3569f2
Author: Liang Zhang <[email protected]>
AuthorDate: Mon Aug 19 00:56:36 2024 +0800

    Add InventoryDumperContextSplitter (#32590)
---
 .../InventoryRecordsCountCalculator.java           |  2 +-
 .../InventoryDumperContextSplitter.java}           | 48 ++------------
 .../inventory/splitter/InventoryTaskSplitter.java  | 75 ++++++++++++++++++++++
 .../pipeline/cdc/core/prepare/CDCJobPreparer.java  |  6 +-
 .../migration/preparer/MigrationJobPreparer.java   |  4 +-
 .../core/prepare/InventoryTaskSplitterTest.java    | 26 ++++----
 6 files changed, 100 insertions(+), 61 deletions(-)

diff --git 
a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/InventoryRecordsCountCalculator.java
 
b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/calculator/InventoryRecordsCountCalculator.java
similarity index 99%
rename from 
kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/InventoryRecordsCountCalculator.java
rename to 
kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/calculator/InventoryRecordsCountCalculator.java
index 0544ce606e5..5d48ea0ca0f 100644
--- 
a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/InventoryRecordsCountCalculator.java
+++ 
b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/calculator/InventoryRecordsCountCalculator.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.data.pipeline.core.preparer.inventory;
+package 
org.apache.shardingsphere.data.pipeline.core.preparer.inventory.calculator;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
diff --git 
a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/InventoryTaskSplitter.java
 
b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/splitter/InventoryDumperContextSplitter.java
similarity index 79%
rename from 
kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/InventoryTaskSplitter.java
rename to 
kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/splitter/InventoryDumperContextSplitter.java
index 2d614aa0dc9..4ccdfd82b60 100644
--- 
a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/InventoryTaskSplitter.java
+++ 
b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/splitter/InventoryDumperContextSplitter.java
@@ -15,22 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.data.pipeline.core.preparer.inventory;
+package 
org.apache.shardingsphere.data.pipeline.core.preparer.inventory.splitter;
 
 import lombok.RequiredArgsConstructor;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.commons.lang3.Range;
-import 
org.apache.shardingsphere.data.pipeline.core.channel.InventoryChannelCreator;
-import org.apache.shardingsphere.data.pipeline.core.channel.PipelineChannel;
 import 
org.apache.shardingsphere.data.pipeline.core.context.TransmissionJobItemContext;
 import 
org.apache.shardingsphere.data.pipeline.core.context.TransmissionProcessContext;
 import 
org.apache.shardingsphere.data.pipeline.core.datasource.PipelineDataSourceWrapper;
 import 
org.apache.shardingsphere.data.pipeline.core.exception.job.SplitPipelineJobByUniqueKeyException;
-import org.apache.shardingsphere.data.pipeline.core.importer.Importer;
-import 
org.apache.shardingsphere.data.pipeline.core.importer.ImporterConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.core.importer.SingleChannelConsumerImporter;
-import org.apache.shardingsphere.data.pipeline.core.ingest.dumper.Dumper;
-import 
org.apache.shardingsphere.data.pipeline.core.ingest.dumper.inventory.InventoryDumper;
 import 
org.apache.shardingsphere.data.pipeline.core.ingest.dumper.inventory.InventoryDumperContext;
 import 
org.apache.shardingsphere.data.pipeline.core.ingest.position.IngestPosition;
 import 
org.apache.shardingsphere.data.pipeline.core.ingest.position.type.pk.type.IntegerPrimaryKeyIngestPosition;
@@ -41,10 +34,9 @@ import 
org.apache.shardingsphere.data.pipeline.core.job.progress.TransmissionJob
 import 
org.apache.shardingsphere.data.pipeline.core.job.progress.config.PipelineReadConfiguration;
 import 
org.apache.shardingsphere.data.pipeline.core.metadata.loader.PipelineTableMetaDataUtils;
 import 
org.apache.shardingsphere.data.pipeline.core.metadata.model.PipelineColumnMetaData;
+import 
org.apache.shardingsphere.data.pipeline.core.preparer.inventory.calculator.InventoryRecordsCountCalculator;
 import 
org.apache.shardingsphere.data.pipeline.core.ratelimit.JobRateLimitAlgorithm;
 import 
org.apache.shardingsphere.data.pipeline.core.sqlbuilder.sql.PipelinePrepareSQLBuilder;
-import org.apache.shardingsphere.data.pipeline.core.task.InventoryTask;
-import org.apache.shardingsphere.data.pipeline.core.task.PipelineTaskUtils;
 import 
org.apache.shardingsphere.data.pipeline.core.util.IntervalToRangeIterator;
 import org.apache.shardingsphere.data.pipeline.core.util.PipelineJdbcUtils;
 
@@ -57,55 +49,27 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
 
 /**
  * Inventory task splitter.
  */
 @RequiredArgsConstructor
 @Slf4j
-public final class InventoryTaskSplitter {
+public final class InventoryDumperContextSplitter {
     
     private final PipelineDataSourceWrapper sourceDataSource;
     
     private final InventoryDumperContext dumperContext;
     
-    private final ImporterConfiguration importerConfig;
-    
-    /**
-     * Split inventory data to multi-tasks.
-     *
-     * @param jobItemContext job item context
-     * @return split inventory data task
-     */
-    public List<InventoryTask> splitInventoryData(final 
TransmissionJobItemContext jobItemContext) {
-        List<InventoryTask> result = new LinkedList<>();
-        long startTimeMillis = System.currentTimeMillis();
-        TransmissionProcessContext processContext = 
jobItemContext.getJobProcessContext();
-        for (InventoryDumperContext each : 
splitInventoryDumperContext(jobItemContext)) {
-            AtomicReference<IngestPosition> position = new 
AtomicReference<>(each.getCommonContext().getPosition());
-            PipelineChannel channel = 
InventoryChannelCreator.create(processContext.getProcessConfiguration().getStreamChannel(),
 importerConfig.getBatchSize(), position);
-            Dumper dumper = new InventoryDumper(each, channel, 
sourceDataSource, jobItemContext.getSourceMetaDataLoader());
-            Importer importer = new SingleChannelConsumerImporter(channel, 
importerConfig.getBatchSize(), 3000L, jobItemContext.getSink(), jobItemContext);
-            result.add(new 
InventoryTask(PipelineTaskUtils.generateInventoryTaskId(each), 
processContext.getInventoryDumperExecuteEngine(),
-                    processContext.getInventoryImporterExecuteEngine(), 
dumper, importer, position));
-        }
-        log.info("splitInventoryData cost {} ms", System.currentTimeMillis() - 
startTimeMillis);
-        return result;
-    }
-    
     /**
      * Split inventory dumper context.
      *
      * @param jobItemContext job item context
      * @return inventory dumper contexts
      */
-    public Collection<InventoryDumperContext> 
splitInventoryDumperContext(final TransmissionJobItemContext jobItemContext) {
-        Collection<InventoryDumperContext> result = new LinkedList<>();
-        for (InventoryDumperContext each : splitByTable(dumperContext)) {
-            result.addAll(splitByPrimaryKey(each, jobItemContext, 
sourceDataSource));
-        }
-        return result;
+    public Collection<InventoryDumperContext> split(final 
TransmissionJobItemContext jobItemContext) {
+        return splitByTable(dumperContext).stream().flatMap(each -> 
splitByPrimaryKey(each, jobItemContext, 
sourceDataSource).stream()).collect(Collectors.toList());
     }
     
     private Collection<InventoryDumperContext> splitByTable(final 
InventoryDumperContext dumperContext) {
diff --git 
a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/splitter/InventoryTaskSplitter.java
 
b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/splitter/InventoryTaskSplitter.java
new file mode 100644
index 00000000000..05ec09a93da
--- /dev/null
+++ 
b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/preparer/inventory/splitter/InventoryTaskSplitter.java
@@ -0,0 +1,75 @@
+/*
+ * 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.preparer.inventory.splitter;
+
+import lombok.RequiredArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import 
org.apache.shardingsphere.data.pipeline.core.channel.InventoryChannelCreator;
+import org.apache.shardingsphere.data.pipeline.core.channel.PipelineChannel;
+import 
org.apache.shardingsphere.data.pipeline.core.context.TransmissionJobItemContext;
+import 
org.apache.shardingsphere.data.pipeline.core.context.TransmissionProcessContext;
+import 
org.apache.shardingsphere.data.pipeline.core.datasource.PipelineDataSourceWrapper;
+import org.apache.shardingsphere.data.pipeline.core.importer.Importer;
+import 
org.apache.shardingsphere.data.pipeline.core.importer.ImporterConfiguration;
+import 
org.apache.shardingsphere.data.pipeline.core.importer.SingleChannelConsumerImporter;
+import org.apache.shardingsphere.data.pipeline.core.ingest.dumper.Dumper;
+import 
org.apache.shardingsphere.data.pipeline.core.ingest.dumper.inventory.InventoryDumper;
+import 
org.apache.shardingsphere.data.pipeline.core.ingest.dumper.inventory.InventoryDumperContext;
+import 
org.apache.shardingsphere.data.pipeline.core.ingest.position.IngestPosition;
+import org.apache.shardingsphere.data.pipeline.core.task.InventoryTask;
+import org.apache.shardingsphere.data.pipeline.core.task.PipelineTaskUtils;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Inventory task splitter.
+ */
+@RequiredArgsConstructor
+@Slf4j
+public final class InventoryTaskSplitter {
+    
+    private final PipelineDataSourceWrapper sourceDataSource;
+    
+    private final InventoryDumperContext dumperContext;
+    
+    private final ImporterConfiguration importerConfig;
+    
+    /**
+     * Split inventory data to multi-tasks.
+     *
+     * @param jobItemContext job item context
+     * @return split inventory data task
+     */
+    public List<InventoryTask> split(final TransmissionJobItemContext 
jobItemContext) {
+        List<InventoryTask> result = new LinkedList<>();
+        long startTimeMillis = System.currentTimeMillis();
+        TransmissionProcessContext processContext = 
jobItemContext.getJobProcessContext();
+        for (InventoryDumperContext each : new 
InventoryDumperContextSplitter(sourceDataSource, 
dumperContext).split(jobItemContext)) {
+            AtomicReference<IngestPosition> position = new 
AtomicReference<>(each.getCommonContext().getPosition());
+            PipelineChannel channel = 
InventoryChannelCreator.create(processContext.getProcessConfiguration().getStreamChannel(),
 importerConfig.getBatchSize(), position);
+            Dumper dumper = new InventoryDumper(each, channel, 
sourceDataSource, jobItemContext.getSourceMetaDataLoader());
+            Importer importer = new SingleChannelConsumerImporter(channel, 
importerConfig.getBatchSize(), 3000L, jobItemContext.getSink(), jobItemContext);
+            result.add(new 
InventoryTask(PipelineTaskUtils.generateInventoryTaskId(each),
+                    processContext.getInventoryDumperExecuteEngine(), 
processContext.getInventoryImporterExecuteEngine(), dumper, importer, 
position));
+        }
+        log.info("splitInventoryData cost {} ms", System.currentTimeMillis() - 
startTimeMillis);
+        return result;
+    }
+}
diff --git 
a/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/prepare/CDCJobPreparer.java
 
b/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/prepare/CDCJobPreparer.java
index 57d86dcc8c8..d36d59bf030 100644
--- 
a/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/prepare/CDCJobPreparer.java
+++ 
b/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/core/prepare/CDCJobPreparer.java
@@ -45,7 +45,7 @@ import 
org.apache.shardingsphere.data.pipeline.core.job.progress.JobItemIncremen
 import 
org.apache.shardingsphere.data.pipeline.core.job.progress.TransmissionJobItemProgress;
 import 
org.apache.shardingsphere.data.pipeline.core.job.service.PipelineJobItemManager;
 import 
org.apache.shardingsphere.data.pipeline.core.preparer.incremental.IncrementalTaskPositionManager;
-import 
org.apache.shardingsphere.data.pipeline.core.preparer.inventory.InventoryTaskSplitter;
+import 
org.apache.shardingsphere.data.pipeline.core.preparer.inventory.splitter.InventoryDumperContextSplitter;
 import org.apache.shardingsphere.data.pipeline.core.task.PipelineTask;
 import org.apache.shardingsphere.data.pipeline.core.task.PipelineTaskUtils;
 import 
org.apache.shardingsphere.data.pipeline.core.task.progress.IncrementalTaskProgress;
@@ -117,8 +117,8 @@ public final class CDCJobPreparer {
         CDCTaskConfiguration taskConfig = jobItemContext.getTaskConfig();
         ImporterConfiguration importerConfig = taskConfig.getImporterConfig();
         TransmissionProcessContext processContext = 
jobItemContext.getJobProcessContext();
-        for (InventoryDumperContext each : new 
InventoryTaskSplitter(jobItemContext.getSourceDataSource(), new 
InventoryDumperContext(taskConfig.getDumperContext().getCommonContext()), 
importerConfig)
-                .splitInventoryDumperContext(jobItemContext)) {
+        for (InventoryDumperContext each : new 
InventoryDumperContextSplitter(jobItemContext.getSourceDataSource(), new 
InventoryDumperContext(taskConfig.getDumperContext().getCommonContext()))
+                .split(jobItemContext)) {
             AtomicReference<IngestPosition> position = new 
AtomicReference<>(each.getCommonContext().getPosition());
             PipelineChannel channel = 
InventoryChannelCreator.create(processContext.getProcessConfiguration().getStreamChannel(),
 importerConfig.getBatchSize(), position);
             if (!(position.get() instanceof IngestFinishedPosition)) {
diff --git 
a/kernel/data-pipeline/scenario/migration/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/migration/preparer/MigrationJobPreparer.java
 
b/kernel/data-pipeline/scenario/migration/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/migration/preparer/MigrationJobPreparer.java
index d30ef4d2390..1ef92b4cd5b 100644
--- 
a/kernel/data-pipeline/scenario/migration/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/migration/preparer/MigrationJobPreparer.java
+++ 
b/kernel/data-pipeline/scenario/migration/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/migration/preparer/MigrationJobPreparer.java
@@ -50,7 +50,7 @@ import 
org.apache.shardingsphere.data.pipeline.core.preparer.datasource.param.Cr
 import 
org.apache.shardingsphere.data.pipeline.core.preparer.datasource.param.PrepareTargetSchemasParameter;
 import 
org.apache.shardingsphere.data.pipeline.core.preparer.datasource.param.PrepareTargetTablesParameter;
 import 
org.apache.shardingsphere.data.pipeline.core.preparer.incremental.IncrementalTaskPositionManager;
-import 
org.apache.shardingsphere.data.pipeline.core.preparer.inventory.InventoryTaskSplitter;
+import 
org.apache.shardingsphere.data.pipeline.core.preparer.inventory.splitter.InventoryTaskSplitter;
 import org.apache.shardingsphere.data.pipeline.core.task.IncrementalTask;
 import org.apache.shardingsphere.data.pipeline.core.task.PipelineTask;
 import org.apache.shardingsphere.data.pipeline.core.task.PipelineTaskUtils;
@@ -188,7 +188,7 @@ public final class MigrationJobPreparer {
     private void initInventoryTasks(final MigrationJobItemContext 
jobItemContext) {
         InventoryDumperContext inventoryDumperContext = new 
InventoryDumperContext(jobItemContext.getTaskConfig().getDumperContext().getCommonContext());
         InventoryTaskSplitter inventoryTaskSplitter = new 
InventoryTaskSplitter(jobItemContext.getSourceDataSource(), 
inventoryDumperContext, jobItemContext.getTaskConfig().getImporterConfig());
-        
jobItemContext.getInventoryTasks().addAll(inventoryTaskSplitter.splitInventoryData(jobItemContext));
+        
jobItemContext.getInventoryTasks().addAll(inventoryTaskSplitter.split(jobItemContext));
     }
     
     private void initIncrementalTasks(final MigrationJobItemContext 
jobItemContext) {
diff --git 
a/test/it/pipeline/src/test/java/org/apache/shardingsphere/test/it/data/pipeline/core/prepare/InventoryTaskSplitterTest.java
 
b/test/it/pipeline/src/test/java/org/apache/shardingsphere/test/it/data/pipeline/core/prepare/InventoryTaskSplitterTest.java
index 3b3aca487d8..8b3aef32347 100644
--- 
a/test/it/pipeline/src/test/java/org/apache/shardingsphere/test/it/data/pipeline/core/prepare/InventoryTaskSplitterTest.java
+++ 
b/test/it/pipeline/src/test/java/org/apache/shardingsphere/test/it/data/pipeline/core/prepare/InventoryTaskSplitterTest.java
@@ -25,7 +25,7 @@ import 
org.apache.shardingsphere.data.pipeline.core.datasource.PipelineDataSourc
 import 
org.apache.shardingsphere.data.pipeline.core.ingest.position.type.pk.type.IntegerPrimaryKeyIngestPosition;
 import 
org.apache.shardingsphere.data.pipeline.core.metadata.loader.PipelineTableMetaDataUtils;
 import 
org.apache.shardingsphere.data.pipeline.core.metadata.loader.StandardPipelineTableMetaDataLoader;
-import 
org.apache.shardingsphere.data.pipeline.core.preparer.inventory.InventoryTaskSplitter;
+import 
org.apache.shardingsphere.data.pipeline.core.preparer.inventory.splitter.InventoryTaskSplitter;
 import org.apache.shardingsphere.data.pipeline.core.task.InventoryTask;
 import 
org.apache.shardingsphere.data.pipeline.scenario.migration.config.MigrationJobConfiguration;
 import 
org.apache.shardingsphere.data.pipeline.scenario.migration.context.MigrationJobItemContext;
@@ -84,9 +84,9 @@ class InventoryTaskSplitterTest {
     }
     
     @Test
-    void assertSplitInventoryDataWithEmptyTable() throws SQLException {
+    void assertSplitWithEmptyTable() throws SQLException {
         initEmptyTablePrimaryEnvironment(dumperContext.getCommonContext());
-        List<InventoryTask> actual = 
inventoryTaskSplitter.splitInventoryData(jobItemContext);
+        List<InventoryTask> actual = 
inventoryTaskSplitter.split(jobItemContext);
         assertThat(actual.size(), is(1));
         InventoryTask task = actual.get(0);
         assertThat(((IntegerPrimaryKeyIngestPosition) 
task.getTaskProgress().getPosition()).getBeginValue(), is(0L));
@@ -94,9 +94,9 @@ class InventoryTaskSplitterTest {
     }
     
     @Test
-    void assertSplitInventoryDataWithIntPrimary() throws SQLException {
+    void assertSplitWithIntPrimary() throws SQLException {
         initIntPrimaryEnvironment(dumperContext.getCommonContext());
-        List<InventoryTask> actual = 
inventoryTaskSplitter.splitInventoryData(jobItemContext);
+        List<InventoryTask> actual = 
inventoryTaskSplitter.split(jobItemContext);
         assertThat(actual.size(), is(10));
         InventoryTask task = actual.get(9);
         assertThat(((IntegerPrimaryKeyIngestPosition) 
task.getTaskProgress().getPosition()).getBeginValue(), is(91L));
@@ -104,9 +104,9 @@ class InventoryTaskSplitterTest {
     }
     
     @Test
-    void assertSplitInventoryDataWithCharPrimary() throws SQLException {
+    void assertSplitWithCharPrimary() throws SQLException {
         initCharPrimaryEnvironment(dumperContext.getCommonContext());
-        List<InventoryTask> actual = 
inventoryTaskSplitter.splitInventoryData(jobItemContext);
+        List<InventoryTask> actual = 
inventoryTaskSplitter.split(jobItemContext);
         assertThat(actual.size(), is(1));
         assertThat(actual.get(0).getTaskId(), is("ds_0.t_order#0"));
         IntegerPrimaryKeyIngestPosition keyPosition = 
(IntegerPrimaryKeyIngestPosition) actual.get(0).getTaskProgress().getPosition();
@@ -115,30 +115,30 @@ class InventoryTaskSplitterTest {
     }
     
     @Test
-    void assertSplitInventoryDataWithoutPrimaryButWithUniqueIndex() throws 
SQLException {
+    void assertSplitWithoutPrimaryButWithUniqueIndex() throws SQLException {
         
initUniqueIndexOnNotNullColumnEnvironment(dumperContext.getCommonContext());
-        List<InventoryTask> actual = 
inventoryTaskSplitter.splitInventoryData(jobItemContext);
+        List<InventoryTask> actual = 
inventoryTaskSplitter.split(jobItemContext);
         assertThat(actual.size(), is(1));
     }
     
     @Test
-    void assertSplitInventoryDataWithMultipleColumnsKey() throws SQLException {
+    void assertSplitWithMultipleColumnsKey() throws SQLException {
         initUnionPrimaryEnvironment(dumperContext.getCommonContext());
         try (PipelineDataSourceWrapper dataSource = 
dataSourceManager.getDataSource(dumperContext.getCommonContext().getDataSourceConfig()))
 {
             List<PipelineColumnMetaData> uniqueKeyColumns = 
PipelineTableMetaDataUtils.getUniqueKeyColumns(null, "t_order", new 
StandardPipelineTableMetaDataLoader(dataSource));
             dumperContext.setUniqueKeyColumns(uniqueKeyColumns);
-            List<InventoryTask> actual = 
inventoryTaskSplitter.splitInventoryData(jobItemContext);
+            List<InventoryTask> actual = 
inventoryTaskSplitter.split(jobItemContext);
             assertThat(actual.size(), is(1));
         }
     }
     
     @Test
-    void assertSplitInventoryDataWithoutPrimaryAndUniqueIndex() throws 
SQLException {
+    void assertSplitWithoutPrimaryAndUniqueIndex() throws SQLException {
         initNoPrimaryEnvironment(dumperContext.getCommonContext());
         try (PipelineDataSourceWrapper dataSource = 
dataSourceManager.getDataSource(dumperContext.getCommonContext().getDataSourceConfig()))
 {
             List<PipelineColumnMetaData> uniqueKeyColumns = 
PipelineTableMetaDataUtils.getUniqueKeyColumns(null, "t_order", new 
StandardPipelineTableMetaDataLoader(dataSource));
             assertTrue(uniqueKeyColumns.isEmpty());
-            List<InventoryTask> inventoryTasks = 
inventoryTaskSplitter.splitInventoryData(jobItemContext);
+            List<InventoryTask> inventoryTasks = 
inventoryTaskSplitter.split(jobItemContext);
             assertThat(inventoryTasks.size(), is(1));
         }
     }

Reply via email to