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

totalo 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 ff873da2a5d Remove all InventoryDumper's impl (#21321)
ff873da2a5d is described below

commit ff873da2a5dd9cac9983fb9b7c805bc8e42b4ff4
Author: Liang Zhang <[email protected]>
AuthorDate: Mon Oct 3 00:25:33 2022 +0800

    Remove all InventoryDumper's impl (#21321)
    
    * Fix test case AbstractInventoryDumper
    
    * Refactor InventoryDumper
---
 .../api/executor/AbstractLifecycleExecutor.java    |  2 +-
 .../api/ingest/dumper/InventoryDumper.java         | 24 ------
 .../spi/ingest/dumper/InventoryDumperCreator.java  | 47 -----------
 .../dumper/InventoryDumperCreatorFactory.java      | 45 -----------
 .../core/ingest/dumper/DefaultInventoryDumper.java | 35 ---------
 .../dumper/DefaultInventoryDumperCreator.java      | 43 ----------
 ...ctInventoryDumper.java => InventoryDumper.java} | 16 ++--
 .../data/pipeline/core/task/InventoryTask.java     |  7 +-
 ...peline.spi.ingest.dumper.InventoryDumperCreator | 18 -----
 .../mysql/MySqlInventoryDumperCreator.java         | 42 ----------
 .../mysql/ingest/MySQLInventoryDumper.java         | 42 ----------
 ...peline.spi.ingest.dumper.InventoryDumperCreator | 18 -----
 .../mysql/ingest/MySQLInventoryDumperTest.java     | 87 ---------------------
 .../PostgreSQLInventoryDumperCreator.java          | 54 -------------
 .../ingest/PostgreSQLInventoryDumper.java          | 43 ----------
 ...peline.spi.ingest.dumper.InventoryDumperCreator | 19 -----
 .../ingest/PostgreSQLJdbcDumperTest.java           | 91 ----------------------
 .../dumper/InventoryDumperCreatorFactoryTest.java  | 88 ---------------------
 .../core/fixture/FixtureInventoryDumper.java       | 33 --------
 .../fixture/FixtureInventoryDumperCreator.java     | 54 -------------
 ...peline.spi.ingest.dumper.InventoryDumperCreator | 18 -----
 .../src/test/resources/logback-test.xml            |  2 +-
 22 files changed, 11 insertions(+), 817 deletions(-)

diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/api/executor/AbstractLifecycleExecutor.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/api/executor/AbstractLifecycleExecutor.java
index 7d66a954843..dc5e8759b43 100644
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/api/executor/AbstractLifecycleExecutor.java
+++ 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/api/executor/AbstractLifecycleExecutor.java
@@ -35,8 +35,8 @@ public abstract class AbstractLifecycleExecutor implements 
LifecycleExecutor {
     
     private static final DateTimeFormatter DATE_TIME_FORMATTER = 
DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss");
     
-    @Setter(AccessLevel.PROTECTED)
     @Getter(AccessLevel.PROTECTED)
+    @Setter(AccessLevel.PROTECTED)
     private volatile boolean running;
     
     private volatile boolean stopped;
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/api/ingest/dumper/InventoryDumper.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/api/ingest/dumper/InventoryDumper.java
deleted file mode 100644
index 9405e29878b..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/api/ingest/dumper/InventoryDumper.java
+++ /dev/null
@@ -1,24 +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.data.pipeline.api.ingest.dumper;
-
-/**
- * Inventory dumper.
- */
-public interface InventoryDumper extends Dumper {
-}
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ingest/dumper/InventoryDumperCreator.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ingest/dumper/InventoryDumperCreator.java
deleted file mode 100644
index 3cd31849370..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ingest/dumper/InventoryDumperCreator.java
+++ /dev/null
@@ -1,47 +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.data.pipeline.spi.ingest.dumper;
-
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.InventoryDumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.channel.PipelineChannel;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.dumper.InventoryDumper;
-import 
org.apache.shardingsphere.data.pipeline.api.metadata.loader.PipelineTableMetaDataLoader;
-import org.apache.shardingsphere.infra.util.spi.annotation.SingletonSPI;
-import org.apache.shardingsphere.infra.util.spi.type.required.RequiredSPI;
-import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPI;
-
-import javax.sql.DataSource;
-
-/**
- * Inventory dumper creator.
- */
-@SingletonSPI
-public interface InventoryDumperCreator extends TypedSPI, RequiredSPI {
-    
-    /**
-     * Create inventory dumper.
-     *
-     * @param inventoryDumperConfig inventory dumper configuration
-     * @param channel channel
-     * @param sourceDataSource source data source
-     * @param sourceMetaDataLoader source meta data loader
-     * @return inventory dumper
-     */
-    InventoryDumper createInventoryDumper(InventoryDumperConfiguration 
inventoryDumperConfig, PipelineChannel channel,
-                                          DataSource sourceDataSource, 
PipelineTableMetaDataLoader sourceMetaDataLoader);
-}
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ingest/dumper/InventoryDumperCreatorFactory.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ingest/dumper/InventoryDumperCreatorFactory.java
deleted file mode 100644
index 6c7b2120630..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ingest/dumper/InventoryDumperCreatorFactory.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.data.pipeline.spi.ingest.dumper;
-
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.infra.util.spi.ShardingSphereServiceLoader;
-import 
org.apache.shardingsphere.infra.util.spi.type.required.RequiredSPIRegistry;
-import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPIRegistry;
-
-/**
- * Inventory dumper creator factory.
- */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public class InventoryDumperCreatorFactory {
-    
-    static {
-        ShardingSphereServiceLoader.register(InventoryDumperCreator.class);
-    }
-    
-    /**
-     * Get inventory dumper creator instance.
-     *
-     * @param databaseType database type
-     * @return inventory dumper creator
-     */
-    public static InventoryDumperCreator getInstance(final String 
databaseType) {
-        return 
TypedSPIRegistry.findRegisteredService(InventoryDumperCreator.class, 
databaseType).orElseGet(() -> 
RequiredSPIRegistry.getRegisteredService(InventoryDumperCreator.class));
-    }
-}
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/DefaultInventoryDumper.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/DefaultInventoryDumper.java
deleted file mode 100644
index df1b8454175..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/DefaultInventoryDumper.java
+++ /dev/null
@@ -1,35 +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.data.pipeline.core.ingest.dumper;
-
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.InventoryDumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.channel.PipelineChannel;
-import 
org.apache.shardingsphere.data.pipeline.api.metadata.loader.PipelineTableMetaDataLoader;
-
-import javax.sql.DataSource;
-
-/**
- * Default inventory dumper.
- */
-public final class DefaultInventoryDumper extends AbstractInventoryDumper {
-    
-    public DefaultInventoryDumper(final InventoryDumperConfiguration 
inventoryDumperConfig, final PipelineChannel channel,
-                                  final DataSource dataSource, final 
PipelineTableMetaDataLoader metaDataLoader) {
-        super(inventoryDumperConfig, channel, dataSource, metaDataLoader);
-    }
-}
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/DefaultInventoryDumperCreator.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/DefaultInventoryDumperCreator.java
deleted file mode 100644
index 2860dcb14f0..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/DefaultInventoryDumperCreator.java
+++ /dev/null
@@ -1,43 +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.data.pipeline.core.ingest.dumper;
-
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.InventoryDumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.channel.PipelineChannel;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.dumper.InventoryDumper;
-import 
org.apache.shardingsphere.data.pipeline.api.metadata.loader.PipelineTableMetaDataLoader;
-import 
org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator;
-
-import javax.sql.DataSource;
-
-/**
- * Default inventory dumper creator.
- */
-public final class DefaultInventoryDumperCreator implements 
InventoryDumperCreator {
-    
-    @Override
-    public InventoryDumper createInventoryDumper(final 
InventoryDumperConfiguration inventoryDumperConfig, final PipelineChannel 
channel,
-                                                 final DataSource 
sourceDataSource, final PipelineTableMetaDataLoader sourceMetaDataLoader) {
-        return new DefaultInventoryDumper(inventoryDumperConfig, channel, 
sourceDataSource, sourceMetaDataLoader);
-    }
-    
-    @Override
-    public boolean isDefault() {
-        return true;
-    }
-}
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/AbstractInventoryDumper.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/InventoryDumper.java
similarity index 93%
rename from 
shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/AbstractInventoryDumper.java
rename to 
shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/InventoryDumper.java
index 2d0656e53fa..5d41113fb64 100644
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/AbstractInventoryDumper.java
+++ 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/ingest/dumper/InventoryDumper.java
@@ -27,7 +27,7 @@ import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.InventoryDumper
 import 
org.apache.shardingsphere.data.pipeline.api.datasource.config.impl.StandardPipelineDataSourceConfiguration;
 import 
org.apache.shardingsphere.data.pipeline.api.executor.AbstractLifecycleExecutor;
 import 
org.apache.shardingsphere.data.pipeline.api.ingest.channel.PipelineChannel;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.dumper.InventoryDumper;
+import org.apache.shardingsphere.data.pipeline.api.ingest.dumper.Dumper;
 import 
org.apache.shardingsphere.data.pipeline.api.ingest.position.FinishedPosition;
 import 
org.apache.shardingsphere.data.pipeline.api.ingest.position.IngestPosition;
 import 
org.apache.shardingsphere.data.pipeline.api.ingest.position.PlaceholderPosition;
@@ -59,10 +59,10 @@ import java.sql.SQLException;
 import java.util.Optional;
 
 /**
- * Abstract JDBC dumper implement.
+ * Inventory dumper.
  */
 @Slf4j
-public abstract class AbstractInventoryDumper extends 
AbstractLifecycleExecutor implements InventoryDumper {
+public final class InventoryDumper extends AbstractLifecycleExecutor 
implements Dumper {
     
     @Getter(AccessLevel.PROTECTED)
     private final InventoryDumperConfiguration dumperConfig;
@@ -77,7 +77,7 @@ public abstract class AbstractInventoryDumper extends 
AbstractLifecycleExecutor
     
     private final LazyInitializer<PipelineTableMetaData> metaDataLoader;
     
-    protected AbstractInventoryDumper(final InventoryDumperConfiguration 
dumperConfig, final PipelineChannel channel, final DataSource dataSource, final 
PipelineTableMetaDataLoader metaDataLoader) {
+    public InventoryDumper(final InventoryDumperConfiguration dumperConfig, 
final PipelineChannel channel, final DataSource dataSource, final 
PipelineTableMetaDataLoader metaDataLoader) {
         
ShardingSpherePreconditions.checkState(StandardPipelineDataSourceConfiguration.class.equals(dumperConfig.getDataSourceConfig().getClass()),
                 () -> new 
UnsupportedSQLOperationException("AbstractInventoryDumper only support 
StandardPipelineDataSourceConfiguration"));
         this.dumperConfig = dumperConfig;
@@ -134,8 +134,7 @@ public abstract class AbstractInventoryDumper extends 
AbstractLifecycleExecutor
         int batchSize = dumperConfig.getBatchSize();
         PipelineTableMetaData tableMetaData = metaDataLoader.get();
         try (PreparedStatement preparedStatement = 
connection.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, 
ResultSet.CONCUR_READ_ONLY)) {
-            setDialectParameters(preparedStatement);
-            setPreparedStatementParameters(preparedStatement, batchSize, 
beginUniqueKeyValue);
+            setParameters(preparedStatement, batchSize, beginUniqueKeyValue);
             try (ResultSet resultSet = preparedStatement.executeQuery()) {
                 ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
                 int rowCount = 0;
@@ -157,7 +156,7 @@ public abstract class AbstractInventoryDumper extends 
AbstractLifecycleExecutor
         }
     }
     
-    private void setPreparedStatementParameters(final PreparedStatement 
preparedStatement, final int batchSize, final Object beginUniqueKeyValue) 
throws SQLException {
+    private void setParameters(final PreparedStatement preparedStatement, 
final int batchSize, final Object beginUniqueKeyValue) throws SQLException {
         preparedStatement.setFetchSize(batchSize);
         if 
(PipelineJdbcUtils.isIntegerColumn(dumperConfig.getUniqueKeyDataType())) {
             preparedStatement.setObject(1, beginUniqueKeyValue);
@@ -190,9 +189,6 @@ public abstract class AbstractInventoryDumper extends 
AbstractLifecycleExecutor
                 : 
PrimaryKeyPositionFactory.newInstance(resultSet.getObject(dumperConfig.getUniqueKey()),
 ((PrimaryKeyPosition<?>) dumperConfig.getPosition()).getEndValue());
     }
     
-    protected void setDialectParameters(final PreparedStatement 
preparedStatement) throws SQLException {
-    }
-    
     @Override
     protected void doStop() {
     }
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/task/InventoryTask.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/task/InventoryTask.java
index bf07e2bdcb5..ca90aeb859e 100644
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/task/InventoryTask.java
+++ 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/task/InventoryTask.java
@@ -34,9 +34,9 @@ import 
org.apache.shardingsphere.data.pipeline.api.metadata.loader.PipelineTable
 import 
org.apache.shardingsphere.data.pipeline.api.task.progress.InventoryTaskProgress;
 import org.apache.shardingsphere.data.pipeline.core.execute.ExecuteCallback;
 import org.apache.shardingsphere.data.pipeline.core.execute.ExecuteEngine;
+import 
org.apache.shardingsphere.data.pipeline.core.ingest.dumper.InventoryDumper;
 import 
org.apache.shardingsphere.data.pipeline.spi.importer.ImporterCreatorFactory;
 import 
org.apache.shardingsphere.data.pipeline.spi.ingest.channel.PipelineChannelCreator;
-import 
org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreatorFactory;
 
 import javax.sql.DataSource;
 import java.util.List;
@@ -45,8 +45,8 @@ import java.util.concurrent.CompletableFuture;
 /**
  * Inventory task.
  */
+@ToString(exclude = {"inventoryDumperExecuteEngine", 
"inventoryImporterExecuteEngine", "channel", "dumper", "importer"})
 @Slf4j
-@ToString(exclude = {"inventoryDumperExecuteEngine", "importerExecuteEngine", 
"channel", "dumper", "importer"})
 public final class InventoryTask implements PipelineTask, AutoCloseable {
     
     @Getter
@@ -73,8 +73,7 @@ public final class InventoryTask implements PipelineTask, 
AutoCloseable {
         this.inventoryDumperExecuteEngine = inventoryDumperExecuteEngine;
         this.inventoryImporterExecuteEngine = inventoryImporterExecuteEngine;
         channel = createChannel(pipelineChannelCreator);
-        dumper = 
InventoryDumperCreatorFactory.getInstance(inventoryDumperConfig.getDataSourceConfig().getDatabaseType().getType())
-                .createInventoryDumper(inventoryDumperConfig, channel, 
sourceDataSource, sourceMetaDataLoader);
+        dumper = new InventoryDumper(inventoryDumperConfig, channel, 
sourceDataSource, sourceMetaDataLoader);
         importer = 
ImporterCreatorFactory.getInstance(importerConfig.getDataSourceConfig().getDatabaseType().getType()).createImporter(importerConfig,
 dataSourceManager, channel, jobProgressListener);
         position = inventoryDumperConfig.getPosition();
     }
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator
deleted file mode 100644
index 03a43360d78..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator
+++ /dev/null
@@ -1,18 +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.
-#
-
-org.apache.shardingsphere.data.pipeline.core.ingest.dumper.DefaultInventoryDumperCreator
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/MySqlInventoryDumperCreator.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/MySqlInventoryDumperCreator.java
deleted file mode 100644
index 225fbeb9e9e..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/MySqlInventoryDumperCreator.java
+++ /dev/null
@@ -1,42 +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.data.pipeline.mysql;
-
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.InventoryDumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.channel.PipelineChannel;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.dumper.InventoryDumper;
-import 
org.apache.shardingsphere.data.pipeline.api.metadata.loader.PipelineTableMetaDataLoader;
-import 
org.apache.shardingsphere.data.pipeline.mysql.ingest.MySQLInventoryDumper;
-import 
org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator;
-
-import javax.sql.DataSource;
-
-public class MySqlInventoryDumperCreator implements InventoryDumperCreator {
-    
-    @Override
-    public InventoryDumper createInventoryDumper(final 
InventoryDumperConfiguration inventoryDumperConfig,
-                                                 final PipelineChannel 
channel, final DataSource sourceDataSource,
-                                                 final 
PipelineTableMetaDataLoader sourceMetaDataLoader) {
-        return new MySQLInventoryDumper(inventoryDumperConfig, channel, 
sourceDataSource, sourceMetaDataLoader);
-    }
-    
-    @Override
-    public String getType() {
-        return "MySQL";
-    }
-}
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/MySQLInventoryDumper.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/MySQLInventoryDumper.java
deleted file mode 100644
index 307653b2920..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/MySQLInventoryDumper.java
+++ /dev/null
@@ -1,42 +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.data.pipeline.mysql.ingest;
-
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.InventoryDumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.channel.PipelineChannel;
-import 
org.apache.shardingsphere.data.pipeline.api.metadata.loader.PipelineTableMetaDataLoader;
-import 
org.apache.shardingsphere.data.pipeline.core.ingest.dumper.AbstractInventoryDumper;
-
-import javax.sql.DataSource;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-
-/**
- * MySQL JDBC Dumper.
- */
-public final class MySQLInventoryDumper extends AbstractInventoryDumper {
-    
-    public MySQLInventoryDumper(final InventoryDumperConfiguration 
dumperConfig, final PipelineChannel channel, final DataSource dataSource, final 
PipelineTableMetaDataLoader metaDataLoader) {
-        super(dumperConfig, channel, dataSource, metaDataLoader);
-    }
-    
-    @Override
-    protected void setDialectParameters(final PreparedStatement 
preparedStatement) throws SQLException {
-        preparedStatement.setFetchSize(Integer.MIN_VALUE);
-    }
-}
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator
deleted file mode 100644
index b7d86e9a51b..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator
+++ /dev/null
@@ -1,18 +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.
-#
-
-org.apache.shardingsphere.data.pipeline.mysql.MySqlInventoryDumperCreator
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/test/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/MySQLInventoryDumperTest.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/test/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/MySQLInventoryDumperTest.java
deleted file mode 100644
index 5f861f0f51c..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/test/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/MySQLInventoryDumperTest.java
+++ /dev/null
@@ -1,87 +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.data.pipeline.mysql.ingest;
-
-import lombok.SneakyThrows;
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.DumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.InventoryDumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.datasource.PipelineDataSourceManager;
-import 
org.apache.shardingsphere.data.pipeline.api.datasource.PipelineDataSourceWrapper;
-import 
org.apache.shardingsphere.data.pipeline.api.datasource.config.impl.StandardPipelineDataSourceConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.core.datasource.DefaultPipelineDataSourceManager;
-import 
org.apache.shardingsphere.data.pipeline.core.ingest.channel.memory.SimpleMemoryPipelineChannel;
-import 
org.apache.shardingsphere.data.pipeline.core.metadata.loader.StandardPipelineTableMetaDataLoader;
-import org.junit.Before;
-import org.junit.Test;
-
-import javax.sql.DataSource;
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-public final class MySQLInventoryDumperTest {
-    
-    private MySQLInventoryDumper mysqlJdbcDumper;
-    
-    @Before
-    public void setUp() {
-        PipelineDataSourceManager dataSourceManager = new 
DefaultPipelineDataSourceManager();
-        InventoryDumperConfiguration dumperConfig = 
mockInventoryDumperConfiguration();
-        PipelineDataSourceWrapper dataSource = 
dataSourceManager.getDataSource(dumperConfig.getDataSourceConfig());
-        mysqlJdbcDumper = new 
MySQLInventoryDumper(mockInventoryDumperConfiguration(), new 
SimpleMemoryPipelineChannel(100), dataSource, new 
StandardPipelineTableMetaDataLoader(dataSource));
-        initTableData(dataSource);
-    }
-    
-    private InventoryDumperConfiguration mockInventoryDumperConfiguration() {
-        DumperConfiguration dumperConfig = mockDumperConfiguration();
-        InventoryDumperConfiguration result = new 
InventoryDumperConfiguration(dumperConfig);
-        result.setActualTableName("t_order_0");
-        result.setLogicTableName("t_order");
-        return result;
-    }
-    
-    private DumperConfiguration mockDumperConfiguration() {
-        DumperConfiguration result = new DumperConfiguration();
-        result.setDataSourceConfig(new 
StandardPipelineDataSourceConfiguration("jdbc:h2:mem:test;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MySQL",
 "root", "root"));
-        return result;
-    }
-    
-    @SneakyThrows(SQLException.class)
-    private void initTableData(final DataSource dataSource) {
-        try (Connection connection = dataSource.getConnection(); Statement 
statement = connection.createStatement()) {
-            statement.execute("DROP TABLE IF EXISTS t_order");
-            statement.execute("CREATE TABLE t_order (order_id INT PRIMARY KEY, 
user_id VARCHAR(12))");
-            statement.execute("INSERT INTO t_order (order_id, user_id) VALUES 
(1, 'xxx'), (999, 'yyy')");
-        }
-    }
-    
-    @Test
-    public void assertCreatePreparedStatement() throws SQLException {
-        Connection connection = mock(Connection.class);
-        when(connection.prepareStatement("", ResultSet.TYPE_FORWARD_ONLY, 
ResultSet.CONCUR_READ_ONLY)).thenReturn(mock(PreparedStatement.class));
-        PreparedStatement preparedStatement = connection.prepareStatement("", 
ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
-        mysqlJdbcDumper.setDialectParameters(preparedStatement);
-        verify(preparedStatement).setFetchSize(Integer.MIN_VALUE);
-    }
-}
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/PostgreSQLInventoryDumperCreator.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/PostgreSQLInventoryDumperCreator.java
deleted file mode 100644
index 6394e2e9cb2..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/PostgreSQLInventoryDumperCreator.java
+++ /dev/null
@@ -1,54 +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.data.pipeline.postgresql;
-
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.InventoryDumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.channel.PipelineChannel;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.dumper.InventoryDumper;
-import 
org.apache.shardingsphere.data.pipeline.api.metadata.loader.PipelineTableMetaDataLoader;
-import 
org.apache.shardingsphere.data.pipeline.postgresql.ingest.PostgreSQLInventoryDumper;
-import 
org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator;
-
-import javax.sql.DataSource;
-import java.util.Collection;
-import java.util.Collections;
-
-/**
- * PostgreSQL inventory dumper creator.
- */
-public class PostgreSQLInventoryDumperCreator implements 
InventoryDumperCreator {
-    
-    private static final Collection<String> TYPE_ALIASES = 
Collections.singletonList("openGauss");
-    
-    @Override
-    public InventoryDumper createInventoryDumper(final 
InventoryDumperConfiguration inventoryDumperConfig,
-                                                 final PipelineChannel 
channel, final DataSource sourceDataSource,
-                                                 final 
PipelineTableMetaDataLoader sourceMetaDataLoader) {
-        return new PostgreSQLInventoryDumper(inventoryDumperConfig, channel, 
sourceDataSource, sourceMetaDataLoader);
-    }
-    
-    @Override
-    public String getType() {
-        return "PostgreSQL";
-    }
-    
-    @Override
-    public Collection<String> getTypeAliases() {
-        return TYPE_ALIASES;
-    }
-}
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/ingest/PostgreSQLInventoryDumper.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/ingest/PostgreSQLInventoryDumper.java
deleted file mode 100644
index 143fdc5172c..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/ingest/PostgreSQLInventoryDumper.java
+++ /dev/null
@@ -1,43 +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.data.pipeline.postgresql.ingest;
-
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.InventoryDumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.channel.PipelineChannel;
-import 
org.apache.shardingsphere.data.pipeline.api.metadata.loader.PipelineTableMetaDataLoader;
-import 
org.apache.shardingsphere.data.pipeline.core.ingest.dumper.AbstractInventoryDumper;
-
-import javax.sql.DataSource;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-
-/**
- * PostgreSQL JDBC dumper.
- */
-public final class PostgreSQLInventoryDumper extends AbstractInventoryDumper {
-    
-    public PostgreSQLInventoryDumper(final InventoryDumperConfiguration 
inventoryDumperConfig, final PipelineChannel channel,
-                                     final DataSource dataSource, final 
PipelineTableMetaDataLoader metaDataLoader) {
-        super(inventoryDumperConfig, channel, dataSource, metaDataLoader);
-    }
-    
-    @Override
-    protected void setDialectParameters(final PreparedStatement 
preparedStatement) throws SQLException {
-        preparedStatement.setFetchSize(1);
-    }
-}
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.Inventory
 [...]
deleted file mode 100644
index 4864bf53ce9..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator
+++ /dev/null
@@ -1,19 +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.
-#
-
-org.apache.shardingsphere.data.pipeline.postgresql.PostgreSQLInventoryDumperCreator
-
diff --git 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/test/java/org/apache/shardingsphere/data/pipeline/postgresql/ingest/PostgreSQLJdbcDumperTest.java
 
b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/test/java/org/apache/shardingsphere/data/pipeline/postgresql/ingest/PostgreSQLJdbcDumperTest.java
deleted file mode 100644
index c04d4915aa2..00000000000
--- 
a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/test/java/org/apache/shardingsphere/data/pipeline/postgresql/ingest/PostgreSQLJdbcDumperTest.java
+++ /dev/null
@@ -1,91 +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.data.pipeline.postgresql.ingest;
-
-import lombok.SneakyThrows;
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.DumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.InventoryDumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.datasource.PipelineDataSourceManager;
-import 
org.apache.shardingsphere.data.pipeline.api.datasource.PipelineDataSourceWrapper;
-import 
org.apache.shardingsphere.data.pipeline.api.datasource.config.impl.StandardPipelineDataSourceConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.core.datasource.DefaultPipelineDataSourceManager;
-import 
org.apache.shardingsphere.data.pipeline.core.ingest.channel.memory.SimpleMemoryPipelineChannel;
-import 
org.apache.shardingsphere.data.pipeline.core.metadata.loader.StandardPipelineTableMetaDataLoader;
-import org.junit.Before;
-import org.junit.Test;
-
-import javax.sql.DataSource;
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.hamcrest.MatcherAssert.assertThat;
-
-public final class PostgreSQLJdbcDumperTest {
-    
-    private PipelineDataSourceWrapper dataSource;
-    
-    private PostgreSQLInventoryDumper jdbcDumper;
-    
-    @Before
-    public void setUp() {
-        PipelineDataSourceManager dataSourceManager = new 
DefaultPipelineDataSourceManager();
-        InventoryDumperConfiguration dumperConfig = 
mockInventoryDumperConfiguration();
-        dataSource = 
dataSourceManager.getDataSource(dumperConfig.getDataSourceConfig());
-        jdbcDumper = new 
PostgreSQLInventoryDumper(mockInventoryDumperConfiguration(), new 
SimpleMemoryPipelineChannel(100),
-                dataSource, new 
StandardPipelineTableMetaDataLoader(dataSource));
-        initTableData(dataSource);
-    }
-    
-    private InventoryDumperConfiguration mockInventoryDumperConfiguration() {
-        DumperConfiguration dumperConfig = mockDumperConfiguration();
-        InventoryDumperConfiguration result = new 
InventoryDumperConfiguration(dumperConfig);
-        result.setActualTableName("t_order");
-        result.setLogicTableName("t_order");
-        return result;
-    }
-    
-    @SneakyThrows(SQLException.class)
-    private void initTableData(final DataSource dataSource) {
-        try (
-                Connection connection = dataSource.getConnection();
-                Statement statement = connection.createStatement()) {
-            statement.execute("DROP TABLE IF EXISTS t_order");
-            statement.execute("CREATE TABLE t_order (order_id INT PRIMARY KEY, 
user_id VARCHAR(12))");
-            statement.execute("INSERT INTO t_order (order_id, user_id) VALUES 
(1, 'xxx'), (999, 'yyy')");
-        }
-    }
-    
-    @Test
-    public void assertCreatePreparedStatement() throws SQLException {
-        try (
-                Connection connection = dataSource.getConnection();
-                PreparedStatement preparedStatement = 
connection.prepareStatement("SELECT * FROM t_order")) {
-            jdbcDumper.setDialectParameters(preparedStatement);
-            assertThat(preparedStatement.getFetchSize(), is(1));
-        }
-    }
-    
-    private DumperConfiguration mockDumperConfiguration() {
-        DumperConfiguration result = new DumperConfiguration();
-        result.setDataSourceConfig(new 
StandardPipelineDataSourceConfiguration("jdbc:h2:mem:test;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=PostgreSQL",
 "root", "root"));
-        return result;
-    }
-}
diff --git 
a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/dumper/InventoryDumperCreatorFactoryTest.java
 
b/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/dumper/InventoryDumperCreatorFactoryTest.java
deleted file mode 100644
index db2b2abbac9..00000000000
--- 
a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/dumper/InventoryDumperCreatorFactoryTest.java
+++ /dev/null
@@ -1,88 +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.data.pipeline.core.dumper;
-
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.DumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.InventoryDumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.datasource.PipelineDataSourceWrapper;
-import 
org.apache.shardingsphere.data.pipeline.api.datasource.config.impl.StandardPipelineDataSourceConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.dumper.InventoryDumper;
-import 
org.apache.shardingsphere.data.pipeline.core.fixture.FixtureInventoryDumper;
-import 
org.apache.shardingsphere.data.pipeline.core.ingest.channel.memory.SimpleMemoryPipelineChannel;
-import 
org.apache.shardingsphere.data.pipeline.core.ingest.dumper.DefaultInventoryDumper;
-import 
org.apache.shardingsphere.data.pipeline.core.metadata.loader.StandardPipelineTableMetaDataLoader;
-import 
org.apache.shardingsphere.data.pipeline.mysql.ingest.MySQLInventoryDumper;
-import 
org.apache.shardingsphere.data.pipeline.postgresql.ingest.PostgreSQLInventoryDumper;
-import 
org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreatorFactory;
-import org.junit.Test;
-
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.hamcrest.MatcherAssert.assertThat;
-
-public final class InventoryDumperCreatorFactoryTest {
-    
-    @Test
-    public void assertInventoryDumperCreatorForMySQL() {
-        InventoryDumper actual = createInventoryDumper("MySQL");
-        assertThat(actual, instanceOf(MySQLInventoryDumper.class));
-    }
-    
-    @Test
-    public void assertInventoryDumperCreatorForPostgreSQL() {
-        InventoryDumper actual = createInventoryDumper("PostgreSQL");
-        assertThat(actual, instanceOf(PostgreSQLInventoryDumper.class));
-    }
-    
-    @Test
-    public void assertInventoryDumperCreatorForOpenGauss() {
-        InventoryDumper actual = createInventoryDumper("openGauss");
-        assertThat(actual, instanceOf(PostgreSQLInventoryDumper.class));
-    }
-    
-    @Test
-    public void assertInventoryDumperCreatorForOracle() {
-        InventoryDumper actual = createInventoryDumper("Oracle");
-        assertThat(actual, instanceOf(DefaultInventoryDumper.class));
-    }
-    
-    @Test
-    public void assertInventoryDumperCreatorForFixture() {
-        InventoryDumper actual = createInventoryDumper("Fixture");
-        assertThat(actual, instanceOf(FixtureInventoryDumper.class));
-    }
-    
-    private InventoryDumper createInventoryDumper(final String databaseType) {
-        PipelineDataSourceWrapper dataSource = null;
-        return InventoryDumperCreatorFactory.getInstance(databaseType)
-                .createInventoryDumper(mockInventoryDumperConfiguration(), new 
SimpleMemoryPipelineChannel(100), dataSource, new 
StandardPipelineTableMetaDataLoader(dataSource));
-    }
-    
-    private InventoryDumperConfiguration mockInventoryDumperConfiguration() {
-        DumperConfiguration dumperConfig = mockDumperConfiguration();
-        InventoryDumperConfiguration result = new 
InventoryDumperConfiguration(dumperConfig);
-        result.setActualTableName("t_order");
-        result.setLogicTableName("t_order");
-        return result;
-    }
-    
-    private DumperConfiguration mockDumperConfiguration() {
-        DumperConfiguration result = new DumperConfiguration();
-        result.setDataSourceConfig(new 
StandardPipelineDataSourceConfiguration("jdbc:h2:mem:test;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=PostgreSQL",
 "root", "root"));
-        return result;
-    }
-}
diff --git 
a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/fixture/FixtureInventoryDumper.java
 
b/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/fixture/FixtureInventoryDumper.java
deleted file mode 100644
index e05453c882e..00000000000
--- 
a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/fixture/FixtureInventoryDumper.java
+++ /dev/null
@@ -1,33 +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.data.pipeline.core.fixture;
-
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.InventoryDumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.channel.PipelineChannel;
-import 
org.apache.shardingsphere.data.pipeline.api.metadata.loader.PipelineTableMetaDataLoader;
-import 
org.apache.shardingsphere.data.pipeline.core.ingest.dumper.AbstractInventoryDumper;
-
-import javax.sql.DataSource;
-
-public final class FixtureInventoryDumper extends AbstractInventoryDumper {
-    
-    public FixtureInventoryDumper(final InventoryDumperConfiguration 
dumperConfig, final PipelineChannel channel,
-                                  final DataSource dataSource, final 
PipelineTableMetaDataLoader metaDataLoader) {
-        super(dumperConfig, channel, dataSource, metaDataLoader);
-    }
-}
diff --git 
a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/fixture/FixtureInventoryDumperCreator.java
 
b/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/fixture/FixtureInventoryDumperCreator.java
deleted file mode 100644
index d41cde2b172..00000000000
--- 
a/shardingsphere-test/shardingsphere-pipeline-test/src/test/java/org/apache/shardingsphere/data/pipeline/core/fixture/FixtureInventoryDumperCreator.java
+++ /dev/null
@@ -1,54 +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.data.pipeline.core.fixture;
-
-import 
org.apache.shardingsphere.data.pipeline.api.config.ingest.InventoryDumperConfiguration;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.channel.PipelineChannel;
-import 
org.apache.shardingsphere.data.pipeline.api.ingest.dumper.InventoryDumper;
-import 
org.apache.shardingsphere.data.pipeline.api.metadata.loader.PipelineTableMetaDataLoader;
-import 
org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator;
-
-import javax.sql.DataSource;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-/**
- * Fixture iInventory dumper creator.
- */
-public class FixtureInventoryDumperCreator implements InventoryDumperCreator {
-    
-    private static final Collection<String> TYPE_ALIASES = 
Collections.unmodifiableList(Arrays.asList("Fixture", "H2"));
-    
-    @Override
-    public InventoryDumper createInventoryDumper(final 
InventoryDumperConfiguration inventoryDumperConfig,
-                                                 final PipelineChannel 
channel, final DataSource sourceDataSource,
-                                                 final 
PipelineTableMetaDataLoader sourceMetaDataLoader) {
-        return new FixtureInventoryDumper(inventoryDumperConfig, channel, 
sourceDataSource, sourceMetaDataLoader);
-    }
-    
-    @Override
-    public String getType() {
-        return "Fixture";
-    }
-    
-    @Override
-    public Collection<String> getTypeAliases() {
-        return TYPE_ALIASES;
-    }
-}
diff --git 
a/shardingsphere-test/shardingsphere-pipeline-test/src/test/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator
 
b/shardingsphere-test/shardingsphere-pipeline-test/src/test/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator
deleted file mode 100644
index 2989bc73c9b..00000000000
--- 
a/shardingsphere-test/shardingsphere-pipeline-test/src/test/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.InventoryDumperCreator
+++ /dev/null
@@ -1,18 +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.
-#
-
-org.apache.shardingsphere.data.pipeline.core.fixture.FixtureInventoryDumperCreator
diff --git 
a/shardingsphere-test/shardingsphere-pipeline-test/src/test/resources/logback-test.xml
 
b/shardingsphere-test/shardingsphere-pipeline-test/src/test/resources/logback-test.xml
index bffa220ab38..ad329c3f139 100644
--- 
a/shardingsphere-test/shardingsphere-pipeline-test/src/test/resources/logback-test.xml
+++ 
b/shardingsphere-test/shardingsphere-pipeline-test/src/test/resources/logback-test.xml
@@ -25,7 +25,7 @@
     <logger name="org.apache.zookeeper" level="off" />
     <logger 
name="org.apache.shardingsphere.data.pipeline.core.prepare.InventoryTaskSplitter"
 level="off" />
     <logger 
name="org.apache.shardingsphere.data.pipeline.scenario.migration.MigrationJobAPIImpl"
 level="off" />
-    <logger 
name="org.apache.shardingsphere.data.pipeline.core.ingest.dumper.AbstractInventoryDumper"
 level="off" />
+    <logger 
name="org.apache.shardingsphere.data.pipeline.core.ingest.dumper.InventoryDumper"
 level="off" />
     
     <root>
         <level value="error" />

Reply via email to