From fd2a57d82ab9be835ff589d3ab3cbca9aecaf7af Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Mon, 1 Apr 2024 10:38:43 +0800 Subject: [PATCH 1/9] [Improve][Test] Fix test unstable on `ResourceManger` and `EventReport` module (#6620) --- .github/workflows/backend.yml | 4 +- .../seatunnel/SeaTunnelContainer.java | 2 + .../ResourceManagerFunctionTest.java | 56 ------------------- .../resourcemanager/ResourceManagerTest.java | 29 +++++++++- 4 files changed, 33 insertions(+), 58 deletions(-) delete mode 100644 seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/ResourceManagerFunctionTest.java diff --git a/.github/workflows/backend.yml b/.github/workflows/backend.yml index e63a5ac51ea..519cf8533d2 100644 --- a/.github/workflows/backend.yml +++ b/.github/workflows/backend.yml @@ -556,7 +556,7 @@ jobs: - name: run seatunnel zeta integration test if: needs.changes.outputs.api == 'true' run: | - ./mvnw -T 1 -B verify -DskipUT=true -DskipIT=false -D"license.skipAddThirdParty"=true --no-snapshot-updates -pl :connector-seatunnel-e2e-base -am -Pci + ./mvnw -T 1 -B verify -DskipUT=true -DskipIT=false -D"license.skipAddThirdParty"=true --no-snapshot-updates -pl :connector-seatunnel-e2e-base,:connector-console-seatunnel-e2e -am -Pci env: MAVEN_OPTS: -Xmx4096m engine-k8s-it: @@ -578,6 +578,8 @@ jobs: env: KUBECONFIG: /etc/rancher/k3s/k3s.yaml - uses: actions/checkout@v2 + - name: free disk space + run: tools/github/free_disk_space.sh - name: Set up JDK ${{ matrix.java }} uses: actions/setup-java@v3 with: diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java index aa4d62024f5..ef83f83257f 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/seatunnel/SeaTunnelContainer.java @@ -248,6 +248,8 @@ private static boolean isSystemThread(String s) { || s.startsWith("Timer-") || s.contains("InterruptTimer") || s.contains("Java2D Disposer") + || s.contains("OkHttp ConnectionPool") + || s.startsWith("http-report-event-scheduler") || s.contains( "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner") || s.startsWith("Log4j2-TF-") diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/ResourceManagerFunctionTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/ResourceManagerFunctionTest.java deleted file mode 100644 index acb4237f070..00000000000 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/ResourceManagerFunctionTest.java +++ /dev/null @@ -1,56 +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.seatunnel.engine.server.resourcemanager; - -import org.apache.seatunnel.engine.server.AbstractSeaTunnelServerTest; -import org.apache.seatunnel.engine.server.resourcemanager.resource.ResourceProfile; -import org.apache.seatunnel.engine.server.resourcemanager.resource.SlotProfile; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -import com.hazelcast.cluster.Address; - -import java.util.ArrayList; -import java.util.List; -import java.util.Set; -import java.util.concurrent.ExecutionException; -import java.util.stream.Collectors; - -public class ResourceManagerFunctionTest - extends AbstractSeaTunnelServerTest { - - @Test - public void testApplyResourceWithRandomResult() - throws ExecutionException, InterruptedException { - FakeResourceManager resourceManager = new FakeResourceManager(nodeEngine); - - List resourceProfiles = new ArrayList<>(); - resourceProfiles.add(new ResourceProfile()); - resourceProfiles.add(new ResourceProfile()); - resourceProfiles.add(new ResourceProfile()); - resourceProfiles.add(new ResourceProfile()); - resourceProfiles.add(new ResourceProfile()); - List slotProfiles = resourceManager.applyResources(1L, resourceProfiles).get(); - Assertions.assertEquals(slotProfiles.size(), 5); - - Set
addresses = - slotProfiles.stream().map(SlotProfile::getWorker).collect(Collectors.toSet()); - Assertions.assertTrue(addresses.size() > 1); - } -} diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/ResourceManagerTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/ResourceManagerTest.java index 6dc246d8f46..92442838895 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/ResourceManagerTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/ResourceManagerTest.java @@ -27,11 +27,15 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import com.hazelcast.cluster.Address; + import java.util.ArrayList; import java.util.List; +import java.util.Set; import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; -public class ResourceManagerTest extends AbstractSeaTunnelServerTest { +public class ResourceManagerTest extends AbstractSeaTunnelServerTest { private ResourceManager resourceManager; @@ -77,4 +81,27 @@ public void testApplyRequest() throws ExecutionException, InterruptedException { new ResourceProfile(CPU.of(0), Memory.of(Long.MAX_VALUE))) .get()); } + + @Test + public void testApplyResourceWithRandomResult() + throws ExecutionException, InterruptedException { + FakeResourceManager resourceManager = new FakeResourceManager(nodeEngine); + + List resourceProfiles = new ArrayList<>(); + resourceProfiles.add(new ResourceProfile()); + resourceProfiles.add(new ResourceProfile()); + resourceProfiles.add(new ResourceProfile()); + resourceProfiles.add(new ResourceProfile()); + resourceProfiles.add(new ResourceProfile()); + List slotProfiles = resourceManager.applyResources(1L, resourceProfiles).get(); + Assertions.assertEquals(slotProfiles.size(), 5); + + boolean hasDifferentWorker = false; + for (int i = 0; i < 5; i++) { + Set
addresses = + slotProfiles.stream().map(SlotProfile::getWorker).collect(Collectors.toSet()); + hasDifferentWorker = addresses.size() > 1; + } + Assertions.assertTrue(hasDifferentWorker, "should have different worker for each slot"); + } } From cdb1856e8444e7c00002d4ae1cad90b2a38f0ddd Mon Sep 17 00:00:00 2001 From: Eric Date: Mon, 1 Apr 2024 14:52:27 +0800 Subject: [PATCH 2/9] [Hotfix] Fix DEFAULT TABLE problem (#6352) --- pom.xml | 1 + .../api/table/catalog/CatalogTableUtil.java | 4 ++-- .../jdbc/internal/dialect/mysql/MysqlDialect.java | 8 ++++++-- .../jdbc/internal/dialect/oracle/OracleDialect.java | 8 ++++++-- .../jdbc/internal/dialect/psql/PostgresDialect.java | 7 +++++-- .../internal/dialect/sqlserver/SqlServerDialect.java | 8 ++++++-- .../connector/amazondynamodb/AmazondynamodbIT.java | 12 +++++++----- 7 files changed, 33 insertions(+), 15 deletions(-) diff --git a/pom.xml b/pom.xml index 6c0bb7e719b..4d4a3731ae6 100644 --- a/pom.xml +++ b/pom.xml @@ -21,6 +21,7 @@ org.apache apache 31 + org.apache.seatunnel diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/CatalogTableUtil.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/CatalogTableUtil.java index 6f2b6adeb25..b268fe612e8 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/CatalogTableUtil.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/CatalogTableUtil.java @@ -115,14 +115,14 @@ public static List getCatalogTables( return optionalCatalog .map( c -> { - long startTime = System.currentTimeMillis(); try (Catalog catalog = c) { + long startTime = System.currentTimeMillis(); catalog.open(); List catalogTables = catalog.getTables(readonlyConfig); log.info( String.format( - "Get catalog tables, cost time: %d", + "Get catalog tables, cost time: %d ms", System.currentTimeMillis() - startTime)); if (catalogTables.isEmpty()) { throw new SeaTunnelException( diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java index a6632a58732..5527417e916 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java @@ -174,12 +174,16 @@ public Long approximateRowCntStatement(Connection connection, JdbcSourceTable ta // 2. If a query is configured but does not contain a WHERE clause and tablePath is // configured , use TABLE STATUS. // 3. If a query is configured with a WHERE clause, or a query statement is configured but - // tablePath is not, use COUNT(*). + // tablePath is TablePath.DEFAULT, use COUNT(*). boolean useTableStats = StringUtils.isBlank(table.getQuery()) || (!table.getQuery().toLowerCase().contains("where") - && table.getTablePath() != null); + && table.getTablePath() != null + && !TablePath.DEFAULT + .getFullName() + .equals(table.getTablePath().getFullName())); + if (useTableStats) { // The statement used to get approximate row count which is less // accurate than COUNT(*), but is more efficient for large table. diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java index 1cfeb8d7056..8dedc6dfc19 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleDialect.java @@ -184,12 +184,16 @@ public Long approximateRowCntStatement(Connection connection, JdbcSourceTable ta // 2. If a query is configured but does not contain a WHERE clause and tablePath is // configured, use TABLE STATUS. // 3. If a query is configured with a WHERE clause, or a query statement is configured but - // tablePath is not, use COUNT(*). + // tablePath is TablePath.DEFAULT, use COUNT(*). boolean useTableStats = StringUtils.isBlank(table.getQuery()) || (!table.getQuery().toLowerCase().contains("where") - && table.getTablePath() != null); + && table.getTablePath() != null + && !TablePath.DEFAULT + .getFullName() + .equals(table.getTablePath().getFullName())); + if (useTableStats) { TablePath tablePath = table.getTablePath(); String analyzeTable = diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psql/PostgresDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psql/PostgresDialect.java index d1bf6257ec5..51c5eb67d21 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psql/PostgresDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psql/PostgresDialect.java @@ -155,12 +155,15 @@ public Long approximateRowCntStatement(Connection connection, JdbcSourceTable ta // 2. If a query is configured but does not contain a WHERE clause and tablePath is // configured, use TABLE STATUS. // 3. If a query is configured with a WHERE clause, or a query statement is configured but - // tablePath is not, use COUNT(*). + // tablePath is TablePath.DEFAULT, use COUNT(*). boolean useTableStats = StringUtils.isBlank(table.getQuery()) || (!table.getQuery().toLowerCase().contains("where") - && table.getTablePath() != null); + && table.getTablePath() != null + && !TablePath.DEFAULT + .getFullName() + .equals(table.getTablePath().getFullName())); if (useTableStats) { String rowCountQuery = String.format( diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlServerDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlServerDialect.java index 8826e1fdc9e..87e7418966d 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlServerDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlServerDialect.java @@ -165,12 +165,16 @@ public Long approximateRowCntStatement(Connection connection, JdbcSourceTable ta // 2. If a query is configured but does not contain a WHERE clause and tablePath is // configured, use TABLE STATUS. // 3. If a query is configured with a WHERE clause, or a query statement is configured but - // tablePath is not, use COUNT(*). + // tablePath is TablePath.DEFAULT, use COUNT(*). boolean useTableStats = StringUtils.isBlank(table.getQuery()) || (!table.getQuery().toLowerCase().contains("where") - && table.getTablePath() != null); + && table.getTablePath() != null + && !TablePath.DEFAULT + .getFullName() + .equals(table.getTablePath().getFullName())); + if (useTableStats) { TablePath tablePath = table.getTablePath(); try (Statement stmt = connection.createStatement()) { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-amazondynamodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/amazondynamodb/AmazondynamodbIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-amazondynamodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/amazondynamodb/AmazondynamodbIT.java index a7a9e3f7024..5ae72f6d956 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-amazondynamodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/amazondynamodb/AmazondynamodbIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-amazondynamodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/amazondynamodb/AmazondynamodbIT.java @@ -94,9 +94,11 @@ public class AmazondynamodbIT extends TestSuiteBase implements TestResource { @TestTemplate public void testAmazondynamodb(TestContainer container) throws Exception { + assertHasData(SOURCE_TABLE); Container.ExecResult execResult = container.executeJob(AMAZONDYNAMODB_JOB_CONFIG); Assertions.assertEquals(0, execResult.getExitCode()); - assertHasData(); + assertHasData(SOURCE_TABLE); + assertHasData(SINK_TABLE); compareResult(); clearSinkTable(); } @@ -168,10 +170,10 @@ private void clearSinkTable() { createTable(dynamoDbClient, SINK_TABLE); } - private void assertHasData() { - ScanResponse scan = - dynamoDbClient.scan(ScanRequest.builder().tableName(SINK_TABLE).build()); - Assertions.assertTrue(scan.hasItems(), "sink table is empty."); + private void assertHasData(String tableName) { + ScanResponse scan = dynamoDbClient.scan(ScanRequest.builder().tableName(tableName).build()); + Assertions.assertTrue( + !scan.items().isEmpty(), String.format("table %s is empty.", tableName)); } private void compareResult() { From 2346d0ea7c5e840ddfe532b1a1cceb9093f5c4b3 Mon Sep 17 00:00:00 2001 From: Jarvis Date: Tue, 2 Apr 2024 10:11:26 +0800 Subject: [PATCH 3/9] [E2E] Enable StarRocksCDCSinkIT (#6626) --- .../starrocks/StarRocksCDCSinkIT.java | 22 +++++++++++++++++-- .../write-cdc-changelog-to-starrocks.conf | 2 +- 2 files changed, 21 insertions(+), 3 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-starrocks-e2e/src/test/java/org/apache/seatunnel/e2e/connector/starrocks/StarRocksCDCSinkIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-starrocks-e2e/src/test/java/org/apache/seatunnel/e2e/connector/starrocks/StarRocksCDCSinkIT.java index ee7e24ce1c8..1a16662f99f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-starrocks-e2e/src/test/java/org/apache/seatunnel/e2e/connector/starrocks/StarRocksCDCSinkIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-starrocks-e2e/src/test/java/org/apache/seatunnel/e2e/connector/starrocks/StarRocksCDCSinkIT.java @@ -19,12 +19,15 @@ import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.TestTemplate; import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; @@ -53,7 +56,6 @@ import static org.awaitility.Awaitility.given; @Slf4j -@Disabled("There are still errors unfixed @Hisoka-X") public class StarRocksCDCSinkIT extends TestSuiteBase implements TestResource { private static final String DOCKER_IMAGE = "d87904488/starrocks-starter:2.2.1"; private static final String DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; @@ -87,6 +89,18 @@ public class StarRocksCDCSinkIT extends TestSuiteBase implements TestResource { private Connection jdbcConnection; private GenericContainer starRocksServer; + @TestContainerExtension + private final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && curl -O " + + SR_DRIVER_JAR); + Assertions.assertEquals(0, extraCommands.getExitCode()); + }; + @BeforeAll @Override public void startUp() { @@ -119,6 +133,10 @@ public void tearDown() throws Exception { } @TestTemplate + @DisabledOnContainer( + value = {}, + type = {EngineType.SPARK}, + disabledReason = "Currently Spark engine unsupported DELETE operation") public void testStarRocksSink(TestContainer container) throws Exception { Container.ExecResult execResult = container.executeJob("/write-cdc-changelog-to-starrocks.conf"); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-starrocks-e2e/src/test/resources/write-cdc-changelog-to-starrocks.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-starrocks-e2e/src/test/resources/write-cdc-changelog-to-starrocks.conf index 83488e32556..e358e97d624 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-starrocks-e2e/src/test/resources/write-cdc-changelog-to-starrocks.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-starrocks-e2e/src/test/resources/write-cdc-changelog-to-starrocks.conf @@ -65,7 +65,7 @@ sink { password = "" database = "test" table = "e2e_table_sink" - + base-url = "jdbc:mysql://starrocks_cdc_e2e:9030/test" batch_max_rows = 100 max_retries = 3 From 6e0c81d492997cb962f7b23ef192f13071fbc835 Mon Sep 17 00:00:00 2001 From: YalikWang <34478654+YalikWang@users.noreply.github.com> Date: Tue, 2 Apr 2024 10:42:02 +0800 Subject: [PATCH 4/9] [fix][connector-rocketmq]Fix a NPE problem when checkpoint.interval is set too small(#6624) (#6625) --- .../rocketmq/source/RocketMqSourceReader.java | 26 +++++++++++-------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSourceReader.java b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSourceReader.java index fd4f9860724..2beef96f1bc 100644 --- a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSourceReader.java +++ b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSourceReader.java @@ -242,17 +242,21 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { Long offset = entry.getValue(); try { if (messageQueue != null && offset != null) { - consumerThreads - .get(messageQueue) - .getTasks() - .put( - consumer -> { - if (this.metadata.isEnabledCommitCheckpoint()) { - consumer.getOffsetStore() - .updateOffset(messageQueue, offset, false); - consumer.getOffsetStore().persist(messageQueue); - } - }); + RocketMqConsumerThread rocketMqConsumerThread = + consumerThreads.get(messageQueue); + if (rocketMqConsumerThread != null) { + rocketMqConsumerThread + .getTasks() + .put( + consumer -> { + if (this.metadata.isEnabledCommitCheckpoint()) { + consumer.getOffsetStore() + .updateOffset( + messageQueue, offset, false); + consumer.getOffsetStore().persist(messageQueue); + } + }); + } } } catch (InterruptedException e) { log.error("commit offset failed", e); From 72ebc73e87487125e1aad7a01cd622e8b8c1f4b6 Mon Sep 17 00:00:00 2001 From: xiaochen <598457447@qq.com> Date: Tue, 2 Apr 2024 10:49:45 +0800 Subject: [PATCH 5/9] [Fix][Doc] Fix some spell errors (#6628) --- docs/en/seatunnel-engine/engine-jar-storage-mode.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/seatunnel-engine/engine-jar-storage-mode.md b/docs/en/seatunnel-engine/engine-jar-storage-mode.md index cdb30e9dc18..eff40eac9b4 100644 --- a/docs/en/seatunnel-engine/engine-jar-storage-mode.md +++ b/docs/en/seatunnel-engine/engine-jar-storage-mode.md @@ -15,7 +15,7 @@ We are committed to ongoing efforts to enhance and stabilize this functionality, We can enable the optimization job submission process, which is configured in the `seatunel.yaml`. After enabling the optimization of the Seatunnel job submission process configuration item, users can use the Seatunnel Zeta engine as the execution engine without placing the connector Jar packages required for task execution or the third-party Jar packages that the connector relies on in each engine `connector` directory. Users only need to place all the Jar packages for task execution on the client that submits the job, and the client will automatically upload the Jars required for task execution to the Zeta engine. It is necessary to enable this configuration item when submitting jobs in Docker or k8s mode, -which can fundamentally solve the problem of large container images caused by the heavy weight of the Seatunnrl Zeta engine. In the image, only the core framework package of the Zeta engine needs to be provided, +which can fundamentally solve the problem of large container images caused by the heavy weight of the Seatunnel Zeta engine. In the image, only the core framework package of the Zeta engine needs to be provided, and then the jar package of the connector and the third-party jar package that the connector relies on can be separately uploaded to the pod for distribution. After enabling the optimization job submission process configuration item, you do not need to place the following two types of Jar packages in the Zeta engine: @@ -26,7 +26,7 @@ COMMON_ PLUGIN_ JARS refers to the third-party Jar package that the connector re When common jars do not exist in Zeta's `lib`, it can upload the local common jars of the client to the `lib` directory of all engine nodes. This way, even if the user does not place a jar on all nodes in Zeta's `lib`, the task can still be executed normally. However, we do not recommend relying on the configuration item of opening the optimization job submission process to upload the third-party Jar package that the connector relies on. -If you use Zeta Engine, please add the the third-party jar package files that the connector relies on to `$SEATUNNEL_HOME/lib/` directory on each node, such as jdbc drivers. +If you use Zeta Engine, please add the third-party jar package files that the connector relies on to `$SEATUNNEL_HOME/lib/` directory on each node, such as jdbc drivers. # ConnectorJar storage strategy @@ -36,7 +36,7 @@ Two different storage strategies provide a more flexible storage mode for Jar fi ## Related configuration -| paramemter | default value | describe | +| parameter | default value | describe | |-------------------------------------|---------------|----------------------------------------------------------------------------------------------------------------------------------------------------| | connector-jar-storage-enable | false | Whether to enable uploading the connector Jar package to the engine. The default enabled state is false. | | connector-jar-storage-mode | SHARED | Engine-side Jar package storage mode selection. There are two optional modes, SHARED and ISOLATED. The default Jar package storage mode is SHARED. | @@ -79,7 +79,7 @@ Example: ```yaml jar-storage: - connector-jar-storage-enable:true + connector-jar-storage-enable: true connector-jar-storage-mode: SHARED connector-jar-storage-path: "" connector-jar-cleanup-task-interval: 3600 From 2a0a0b9d19cca478f01fde32958b6dda33a45737 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Tue, 2 Apr 2024 10:50:20 +0800 Subject: [PATCH 6/9] [Fix][Connector-V2] Fix add hive partition error when partition already existed (#6577) --- .../seatunnel/hive/commit/HiveSinkAggregatedCommitter.java | 3 --- .../seatunnel/hive/utils/HiveMetaStoreProxy.java | 7 ++++++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/commit/HiveSinkAggregatedCommitter.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/commit/HiveSinkAggregatedCommitter.java index 0e423f3e875..0f57f864580 100644 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/commit/HiveSinkAggregatedCommitter.java +++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/commit/HiveSinkAggregatedCommitter.java @@ -24,7 +24,6 @@ import org.apache.seatunnel.connectors.seatunnel.file.sink.commit.FileSinkAggregatedCommitter; import org.apache.seatunnel.connectors.seatunnel.hive.utils.HiveMetaStoreProxy; -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; import org.apache.thrift.TException; import lombok.extern.slf4j.Slf4j; @@ -71,8 +70,6 @@ public List commit( try { hiveMetaStore.addPartitions(dbName, tableName, partitions); log.info("Add these partitions {}", partitions); - } catch (AlreadyExistsException e) { - log.warn("These partitions {} are already exists", partitions); } catch (TException e) { log.error("Failed to add these partitions {}", partitions, e); errorCommitInfos.add(aggregatedCommitInfo); diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java index e2941340f88..6a1288b661e 100644 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java +++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java @@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.thrift.TException; @@ -131,7 +132,11 @@ public void addPartitions( @NonNull String dbName, @NonNull String tableName, List partitions) throws TException { for (String partition : partitions) { - hiveMetaStoreClient.appendPartition(dbName, tableName, partition); + try { + hiveMetaStoreClient.appendPartition(dbName, tableName, partition); + } catch (AlreadyExistsException e) { + log.warn("The partition {} are already exists", partition); + } } } From e60beb28ecdea41cc3730630f487cd6fa398568c Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Tue, 2 Apr 2024 13:35:58 +0800 Subject: [PATCH 7/9] [Improve][CDC] Improve read performance when record not contains schema field (#6571) --- .../connectors/cdc/base/utils/SourceRecordUtils.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/SourceRecordUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/SourceRecordUtils.java index 872669eacd3..abbdb5b76a8 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/SourceRecordUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/SourceRecordUtils.java @@ -202,10 +202,8 @@ public static TablePath getTablePath(SourceRecord record) { String databaseName = sourceStruct.getString(AbstractSourceInfo.DATABASE_NAME_KEY); String tableName = sourceStruct.getString(AbstractSourceInfo.TABLE_NAME_KEY); String schemaName = null; - try { + if (sourceStruct.schema().field(AbstractSourceInfo.SCHEMA_NAME_KEY) != null) { schemaName = sourceStruct.getString(AbstractSourceInfo.SCHEMA_NAME_KEY); - } catch (Throwable e) { - // ignore } return TablePath.of(databaseName, schemaName, tableName); } From fe33422161b4ad7f210b1f3d9eb3de49c4657cc3 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Tue, 2 Apr 2024 16:09:18 +0800 Subject: [PATCH 8/9] [Improve][CDC] Optimize split state memory allocation in increment phase (#6554) --- pom.xml | 14 + .../relational/JdbcSourceEventDispatcher.java | 18 + .../enumerator/HybridSplitAssigner.java | 22 ++ .../IncrementalSourceEnumerator.java | 12 + .../enumerator/IncrementalSplitAssigner.java | 23 ++ .../enumerator/SnapshotSplitAssigner.java | 28 ++ .../event/CompletedSnapshotPhaseEvent.java | 34 ++ .../reader/IncrementalSourceReader.java | 15 +- .../IncrementalSourceRecordEmitter.java | 28 ++ .../IncrementalSourceStreamFetcher.java | 15 +- .../split/state/IncrementalSplitState.java | 41 ++ .../cdc/base/utils/SourceRecordUtils.java | 7 + .../enumerator/HybridSplitAssignerTest.java | 132 +++++++ .../IncrementalSourceStreamFetcherTest.java | 367 ++++++++++++++++++ .../state/IncrementalSplitStateTest.java | 169 ++++++++ .../LogMinerStreamingChangeEventSource.java | 1 + .../SqlServerStreamingChangeEventSource.java | 1 + .../source/reader/SourceReaderBase.java | 7 +- 18 files changed, 921 insertions(+), 13 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/event/CompletedSnapshotPhaseEvent.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/HybridSplitAssignerTest.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcherTest.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/split/state/IncrementalSplitStateTest.java diff --git a/pom.xml b/pom.xml index 4d4a3731ae6..0f59747f72f 100644 --- a/pom.xml +++ b/pom.xml @@ -117,6 +117,7 @@ 1.81 4.13.2 5.9.0 + 4.11.0 1.3.3 3.3.0 3.2.0 @@ -357,6 +358,13 @@ ${junit4.version} + + org.mockito + mockito-junit-jupiter + ${mockito.version} + test + + com.fasterxml.jackson.core jackson-annotations @@ -521,6 +529,12 @@ test + + org.mockito + mockito-junit-jupiter + test + + diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/relational/JdbcSourceEventDispatcher.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/relational/JdbcSourceEventDispatcher.java index 23dfabd9fa4..90cc8126f45 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/relational/JdbcSourceEventDispatcher.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/relational/JdbcSourceEventDispatcher.java @@ -25,7 +25,9 @@ import org.apache.kafka.connect.source.SourceRecord; import io.debezium.config.CommonConnectorConfig; +import io.debezium.config.Configuration; import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.heartbeat.Heartbeat; import io.debezium.pipeline.DataChangeEvent; import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.source.spi.EventMetadataProvider; @@ -37,6 +39,8 @@ import io.debezium.schema.TopicSelector; import io.debezium.util.SchemaNameAdjuster; +import java.time.Duration; +import java.time.temporal.ChronoUnit; import java.util.Map; /** @@ -71,6 +75,10 @@ public JdbcSourceEventDispatcher( filter, changeEventCreator, metadataProvider, + Heartbeat.create( + getHeartbeatInterval(connectorConfig), + topicSelector.getHeartbeatTopic(), + connectorConfig.getLogicalName()), schemaNameAdjuster); this.queue = queue; this.topic = topicSelector.getPrimaryTopic(); @@ -92,4 +100,14 @@ public void dispatchWatermarkEvent( sourcePartition, topic, sourceSplit.splitId(), watermarkKind, watermark); queue.enqueue(new DataChangeEvent(sourceRecord)); } + + private static Duration getHeartbeatInterval(CommonConnectorConfig connectorConfig) { + Configuration configuration = connectorConfig.getConfig(); + Duration heartbeatInterval = + configuration.getDuration(Heartbeat.HEARTBEAT_INTERVAL, ChronoUnit.MILLIS); + if (heartbeatInterval.isZero()) { + return Duration.ofMillis(5000); + } + return heartbeatInterval; + } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/HybridSplitAssigner.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/HybridSplitAssigner.java index 9070e2fb88f..d6b0bdb96cb 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/HybridSplitAssigner.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/HybridSplitAssigner.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.connectors.cdc.base.source.enumerator; +import org.apache.seatunnel.shade.com.google.common.annotations.VisibleForTesting; + import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; import org.apache.seatunnel.connectors.cdc.base.dialect.DataSourceDialect; import org.apache.seatunnel.connectors.cdc.base.source.enumerator.state.HybridPendingSplitsState; @@ -31,9 +33,11 @@ import io.debezium.relational.TableId; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Optional; +import java.util.function.Predicate; /** Assigner for Hybrid split which contains snapshot splits and incremental splits. */ public class HybridSplitAssigner implements SplitAssigner { @@ -146,4 +150,22 @@ public void notifyCheckpointComplete(long checkpointId) { snapshotSplitAssigner.notifyCheckpointComplete(checkpointId); incrementalSplitAssigner.notifyCheckpointComplete(checkpointId); } + + @VisibleForTesting + IncrementalSplitAssigner getIncrementalSplitAssigner() { + return incrementalSplitAssigner; + } + + @VisibleForTesting + SnapshotSplitAssigner getSnapshotSplitAssigner() { + return snapshotSplitAssigner; + } + + public boolean completedSnapshotPhase(List tableIds) { + return Arrays.asList( + snapshotSplitAssigner.completedSnapshotPhase(tableIds), + incrementalSplitAssigner.completedSnapshotPhase(tableIds)) + .stream() + .allMatch(Predicate.isEqual(true)); + } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/IncrementalSourceEnumerator.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/IncrementalSourceEnumerator.java index 86f7ac42def..b17b910e5d8 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/IncrementalSourceEnumerator.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/IncrementalSourceEnumerator.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.source.SourceEvent; import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.connectors.cdc.base.source.enumerator.state.PendingSplitsState; +import org.apache.seatunnel.connectors.cdc.base.source.event.CompletedSnapshotPhaseEvent; import org.apache.seatunnel.connectors.cdc.base.source.event.CompletedSnapshotSplitsAckEvent; import org.apache.seatunnel.connectors.cdc.base.source.event.CompletedSnapshotSplitsReportEvent; import org.apache.seatunnel.connectors.cdc.base.source.event.SnapshotSplitWatermark; @@ -120,6 +121,17 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { .map(SnapshotSplitWatermark::getSplitId) .collect(Collectors.toList())); context.sendEventToSourceReader(subtaskId, ackEvent); + } else if (sourceEvent instanceof CompletedSnapshotPhaseEvent) { + LOG.debug( + "The enumerator receives completed snapshot phase event {} from subtask {}.", + sourceEvent, + subtaskId); + CompletedSnapshotPhaseEvent event = (CompletedSnapshotPhaseEvent) sourceEvent; + if (splitAssigner instanceof HybridSplitAssigner) { + ((HybridSplitAssigner) splitAssigner).completedSnapshotPhase(event.getTableIds()); + LOG.info( + "Clean the SnapshotSplitAssigner#assignedSplits/splitCompletedOffsets to empty."); + } } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/IncrementalSplitAssigner.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/IncrementalSplitAssigner.java index fe8204f6cd2..7b45ee1ef61 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/IncrementalSplitAssigner.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/IncrementalSplitAssigner.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.connectors.cdc.base.source.enumerator; +import org.apache.seatunnel.shade.com.google.common.annotations.VisibleForTesting; + import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; import org.apache.seatunnel.connectors.cdc.base.source.enumerator.state.IncrementalPhaseState; @@ -45,6 +47,8 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkArgument; + /** Assigner for incremental split. */ public class IncrementalSplitAssigner implements SplitAssigner { @@ -255,4 +259,23 @@ private IncrementalSplit createIncrementalSplit( completedSnapshotSplitInfos, checkpointDataType); } + + @VisibleForTesting + void setSplitAssigned(boolean assigned) { + this.splitAssigned = assigned; + } + + public boolean completedSnapshotPhase(List tableIds) { + checkArgument(splitAssigned && noMoreSplits()); + + for (String splitKey : new ArrayList<>(context.getAssignedSnapshotSplit().keySet())) { + SnapshotSplit assignedSplit = context.getAssignedSnapshotSplit().get(splitKey); + if (tableIds.contains(assignedSplit.getTableId())) { + context.getAssignedSnapshotSplit().remove(splitKey); + context.getSplitCompletedOffsets().remove(assignedSplit.splitId()); + } + } + return context.getAssignedSnapshotSplit().isEmpty() + && context.getSplitCompletedOffsets().isEmpty(); + } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/SnapshotSplitAssigner.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/SnapshotSplitAssigner.java index 443343947cd..c16dd811028 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/SnapshotSplitAssigner.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/SnapshotSplitAssigner.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.connectors.cdc.base.source.enumerator; +import org.apache.seatunnel.shade.com.google.common.annotations.VisibleForTesting; + import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; import org.apache.seatunnel.connectors.cdc.base.dialect.DataSourceDialect; import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkSplitter; @@ -45,6 +47,8 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.stream.Collectors; +import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkArgument; + /** Assigner for snapshot split. */ public class SnapshotSplitAssigner implements SplitAssigner { private static final Logger LOG = LoggerFactory.getLogger(SnapshotSplitAssigner.class); @@ -278,4 +282,28 @@ public boolean isCompleted() { private boolean allSplitsCompleted() { return noMoreSplits() && assignedSplits.size() == splitCompletedOffsets.size(); } + + @VisibleForTesting + Map getAssignedSplits() { + return assignedSplits; + } + + @VisibleForTesting + Map getSplitCompletedOffsets() { + return splitCompletedOffsets; + } + + public boolean completedSnapshotPhase(List tableIds) { + checkArgument(isCompleted() && allSplitsCompleted()); + + for (String splitKey : new ArrayList<>(assignedSplits.keySet())) { + SnapshotSplit assignedSplit = assignedSplits.get(splitKey); + if (tableIds.contains(assignedSplit.getTableId())) { + assignedSplits.remove(splitKey); + splitCompletedOffsets.remove(assignedSplit.splitId()); + } + } + + return assignedSplits.isEmpty() && splitCompletedOffsets.isEmpty(); + } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/event/CompletedSnapshotPhaseEvent.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/event/CompletedSnapshotPhaseEvent.java new file mode 100644 index 00000000000..623bf1c1a98 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/event/CompletedSnapshotPhaseEvent.java @@ -0,0 +1,34 @@ +/* + * 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.seatunnel.connectors.cdc.base.source.event; + +import org.apache.seatunnel.api.source.SourceEvent; + +import io.debezium.relational.TableId; +import lombok.AllArgsConstructor; +import lombok.Data; + +import java.util.List; + +@Data +@AllArgsConstructor +public class CompletedSnapshotPhaseEvent implements SourceEvent { + private static final long serialVersionUID = 1L; + + private List tableIds; +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/IncrementalSourceReader.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/IncrementalSourceReader.java index 7f9d890197d..829f68763da 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/IncrementalSourceReader.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/IncrementalSourceReader.java @@ -22,6 +22,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; import org.apache.seatunnel.connectors.cdc.base.dialect.DataSourceDialect; +import org.apache.seatunnel.connectors.cdc.base.source.event.CompletedSnapshotPhaseEvent; import org.apache.seatunnel.connectors.cdc.base.source.event.CompletedSnapshotSplitsReportEvent; import org.apache.seatunnel.connectors.cdc.base.source.event.SnapshotSplitWatermark; import org.apache.seatunnel.connectors.cdc.base.source.split.IncrementalSplit; @@ -207,7 +208,19 @@ protected SourceSplitStateBase initializedState(SourceSplitBase split) { debeziumDeserializationSchema.restoreCheckpointProducedType( incrementalSplit.getCheckpointDataType()); } - return new IncrementalSplitState(split.asIncrementalSplit()); + IncrementalSplitState splitState = new IncrementalSplitState(incrementalSplit); + if (splitState.autoEnterPureIncrementPhaseIfAllowed()) { + log.info( + "The incremental split[{}] startup position {} is equal the maxSnapshotSplitsHighWatermark {}, auto enter pure increment phase.", + incrementalSplit.splitId(), + splitState.getStartupOffset(), + splitState.getMaxSnapshotSplitsHighWatermark()); + log.info("Clean the IncrementalSplit#completedSnapshotSplitInfos to empty."); + CompletedSnapshotPhaseEvent event = + new CompletedSnapshotPhaseEvent(splitState.getTableIds()); + context.sendSourceEventToEnumerator(event); + } + return splitState; } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/IncrementalSourceRecordEmitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/IncrementalSourceRecordEmitter.java index 65474a0d9fd..a98a9d09591 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/IncrementalSourceRecordEmitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/IncrementalSourceRecordEmitter.java @@ -22,9 +22,11 @@ import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.table.event.SchemaChangeEvent; +import org.apache.seatunnel.connectors.cdc.base.source.event.CompletedSnapshotPhaseEvent; import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; import org.apache.seatunnel.connectors.cdc.base.source.offset.OffsetFactory; import org.apache.seatunnel.connectors.cdc.base.source.split.SourceRecords; +import org.apache.seatunnel.connectors.cdc.base.source.split.state.IncrementalSplitState; import org.apache.seatunnel.connectors.cdc.base.source.split.state.SourceSplitStateBase; import org.apache.seatunnel.connectors.cdc.debezium.DebeziumDeserializationSchema; import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordEmitter; @@ -65,6 +67,7 @@ public class IncrementalSourceRecordEmitter protected final OffsetFactory offsetFactory; + protected final SourceReader.Context context; protected final Counter recordFetchDelay; protected final Counter recordEmitDelay; protected final EventListener eventListener; @@ -76,6 +79,7 @@ public IncrementalSourceRecordEmitter( this.debeziumDeserializationSchema = debeziumDeserializationSchema; this.outputCollector = new OutputCollector<>(); this.offsetFactory = offsetFactory; + this.context = context; this.recordFetchDelay = context.getMetricsContext().counter(CDC_RECORD_FETCH_DELAY); this.recordEmitDelay = context.getMetricsContext().counter(CDC_RECORD_EMIT_DELAY); this.eventListener = context.getEventListener(); @@ -90,6 +94,7 @@ public void emitRecord( SourceRecord next = elementIterator.next(); reportMetrics(next); processElement(next, collector, splitState); + markEnterPureIncrementPhase(next, splitState); } } @@ -138,6 +143,29 @@ protected void processElement( } } + private void markEnterPureIncrementPhase( + SourceRecord element, SourceSplitStateBase splitState) { + if (splitState.isIncrementalSplitState()) { + IncrementalSplitState incrementalSplitState = splitState.asIncrementalSplitState(); + if (incrementalSplitState.isEnterPureIncrementPhase()) { + return; + } + Offset position = getOffsetPosition(element); + if (incrementalSplitState.markEnterPureIncrementPhaseIfNeed(position)) { + log.info( + "The current record position {} is after the maxSnapshotSplitsHighWatermark {}, " + + "mark enter pure increment phase.", + position, + incrementalSplitState.getMaxSnapshotSplitsHighWatermark()); + log.info("Clean the IncrementalSplit#completedSnapshotSplitInfos to empty."); + + CompletedSnapshotPhaseEvent completedSnapshotPhaseEvent = + new CompletedSnapshotPhaseEvent(incrementalSplitState.getTableIds()); + context.sendSourceEventToEnumerator(completedSnapshotPhaseEvent); + } + } + } + private Offset getWatermark(SourceRecord watermarkEvent) { return getOffsetPosition(watermarkEvent.sourceOffset()); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java index e34970054b1..4cad739ac6a 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java @@ -170,7 +170,7 @@ private Iterator splitNormalStream(List batchEve *

After event batch: [checkpoint-before] [SchemaChangeEvent-1, SchemaChangeEvent-2, * checkpoint-after] [a, b, c, d, e] */ - private Iterator splitSchemaChangeStream(List batchEvents) { + Iterator splitSchemaChangeStream(List batchEvents) { List sourceRecordsSet = new ArrayList<>(); List sourceRecordList = new ArrayList<>(); @@ -181,11 +181,6 @@ private Iterator splitSchemaChangeStream(List ba if (!shouldEmit(currentRecord)) { continue; } - if (!SourceRecordUtils.isDataChangeRecord(currentRecord) - && !SourceRecordUtils.isSchemaChangeEvent(currentRecord)) { - sourceRecordList.add(currentRecord); - continue; - } if (SourceRecordUtils.isSchemaChangeEvent(currentRecord)) { if (!schemaChangeResolver.support(currentRecord)) { @@ -208,9 +203,11 @@ private Iterator splitSchemaChangeStream(List ba sourceRecordList = new ArrayList<>(); sourceRecordList.add(currentRecord); } - } else if (SourceRecordUtils.isDataChangeRecord(currentRecord)) { + } else if (SourceRecordUtils.isDataChangeRecord(currentRecord) + || SourceRecordUtils.isHeartbeatRecord(currentRecord)) { if (previousRecord == null - || SourceRecordUtils.isDataChangeRecord(previousRecord)) { + || SourceRecordUtils.isDataChangeRecord(previousRecord) + || SourceRecordUtils.isHeartbeatRecord(previousRecord)) { sourceRecordList.add(currentRecord); } else { sourceRecordList.add( @@ -274,7 +271,7 @@ public void close() { } /** Returns the record should emit or not. */ - private boolean shouldEmit(SourceRecord sourceRecord) { + boolean shouldEmit(SourceRecord sourceRecord) { if (taskContext.isDataChangeRecord(sourceRecord)) { Offset position = taskContext.getStreamOffset(sourceRecord); TableId tableId = getTableId(sourceRecord); diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/split/state/IncrementalSplitState.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/split/state/IncrementalSplitState.java index 4157569766f..c04026bf1ea 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/split/state/IncrementalSplitState.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/split/state/IncrementalSplitState.java @@ -24,6 +24,7 @@ import lombok.Getter; import lombok.Setter; +import java.util.Comparator; import java.util.List; /** The state of split to describe the change log of table(s). */ @@ -39,11 +40,27 @@ public class IncrementalSplitState extends SourceSplitStateBase { /** Obtained by configuration, may not end */ private Offset stopOffset; + private Offset maxSnapshotSplitsHighWatermark; + private volatile boolean enterPureIncrementPhase; + public IncrementalSplitState(IncrementalSplit split) { super(split); this.tableIds = split.getTableIds(); this.startupOffset = split.getStartupOffset(); this.stopOffset = split.getStopOffset(); + + if (split.getCompletedSnapshotSplitInfos().isEmpty()) { + this.maxSnapshotSplitsHighWatermark = null; + this.enterPureIncrementPhase = true; + } else { + this.maxSnapshotSplitsHighWatermark = + split.getCompletedSnapshotSplitInfos().stream() + .filter(e -> e.getWatermark() != null) + .max(Comparator.comparing(o -> o.getWatermark().getHighWatermark())) + .map(e -> e.getWatermark().getHighWatermark()) + .get(); + this.enterPureIncrementPhase = false; + } } @Override @@ -56,4 +73,28 @@ public IncrementalSplit toSourceSplit() { getStopOffset(), incrementalSplit.getCompletedSnapshotSplitInfos()); } + + public synchronized boolean markEnterPureIncrementPhaseIfNeed(Offset currentRecordPosition) { + if (enterPureIncrementPhase) { + return false; + } + + if (currentRecordPosition.isAtOrAfter(maxSnapshotSplitsHighWatermark)) { + split.asIncrementalSplit().getCompletedSnapshotSplitInfos().clear(); + this.enterPureIncrementPhase = true; + return true; + } + + return false; + } + + public synchronized boolean autoEnterPureIncrementPhaseIfAllowed() { + if (!enterPureIncrementPhase + && maxSnapshotSplitsHighWatermark.compareTo(startupOffset) == 0) { + split.asIncrementalSplit().getCompletedSnapshotSplitInfos().clear(); + enterPureIncrementPhase = true; + return true; + } + return false; + } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/SourceRecordUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/SourceRecordUtils.java index abbdb5b76a8..e06213b06d5 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/SourceRecordUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/SourceRecordUtils.java @@ -47,6 +47,8 @@ private SourceRecordUtils() {} public static final String SCHEMA_CHANGE_EVENT_KEY_NAME = "io.debezium.connector.mysql.SchemaChangeKey"; + public static final String HEARTBEAT_VALUE_SCHEMA_KEY_NAME = + "io.debezium.connector.common.Heartbeat"; private static final DocumentReader DOCUMENT_READER = DocumentReader.defaultReader(); /** Converts a {@link ResultSet} row to an array of Objects. */ @@ -105,6 +107,11 @@ public static boolean isDataChangeRecord(SourceRecord record) { && value.getString(Envelope.FieldName.OPERATION) != null; } + public static boolean isHeartbeatRecord(SourceRecord record) { + Schema valueSchema = record.valueSchema(); + return valueSchema != null && valueSchema.name().equals(HEARTBEAT_VALUE_SCHEMA_KEY_NAME); + } + public static TableId getTableId(SourceRecord dataRecord) { Struct value = (Struct) dataRecord.value(); Struct source = value.getStruct(Envelope.FieldName.SOURCE); diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/HybridSplitAssignerTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/HybridSplitAssignerTest.java new file mode 100644 index 00000000000..2c931eb9e43 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/HybridSplitAssignerTest.java @@ -0,0 +1,132 @@ +/* + * 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.seatunnel.connectors.cdc.base.source.enumerator; + +import org.apache.seatunnel.connectors.cdc.base.source.enumerator.state.HybridPendingSplitsState; +import org.apache.seatunnel.connectors.cdc.base.source.enumerator.state.SnapshotPhaseState; +import org.apache.seatunnel.connectors.cdc.base.source.event.SnapshotSplitWatermark; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import io.debezium.relational.TableId; + +import java.util.AbstractMap; +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class HybridSplitAssignerTest { + @Test + public void testCompletedSnapshotPhase() { + Map assignedSplits = createAssignedSplits(); + Map splitCompletedOffsets = createSplitCompletedOffsets(); + SnapshotPhaseState snapshotPhaseState = + new SnapshotPhaseState( + Collections.emptyList(), + Collections.emptyList(), + assignedSplits, + splitCompletedOffsets, + true, + Collections.emptyList(), + false, + false); + HybridPendingSplitsState checkpointState = + new HybridPendingSplitsState(snapshotPhaseState, null); + SplitAssigner.Context context = + new SplitAssigner.Context<>( + null, + Collections.emptySet(), + checkpointState.getSnapshotPhaseState().getAssignedSplits(), + checkpointState.getSnapshotPhaseState().getSplitCompletedOffsets()); + HybridSplitAssigner splitAssigner = + new HybridSplitAssigner<>(context, 1, 1, checkpointState, null, null); + splitAssigner.getIncrementalSplitAssigner().setSplitAssigned(true); + + Assertions.assertFalse( + splitAssigner.completedSnapshotPhase(Arrays.asList(TableId.parse("db1.table1")))); + Assertions.assertFalse( + splitAssigner.getSnapshotSplitAssigner().getAssignedSplits().isEmpty()); + Assertions.assertFalse( + splitAssigner.getSnapshotSplitAssigner().getSplitCompletedOffsets().isEmpty()); + Assertions.assertFalse(context.getAssignedSnapshotSplit().isEmpty()); + Assertions.assertFalse(context.getSplitCompletedOffsets().isEmpty()); + + Assertions.assertTrue( + splitAssigner.completedSnapshotPhase(Arrays.asList(TableId.parse("db1.table2")))); + Assertions.assertTrue( + splitAssigner.getSnapshotSplitAssigner().getAssignedSplits().isEmpty()); + Assertions.assertTrue( + splitAssigner.getSnapshotSplitAssigner().getSplitCompletedOffsets().isEmpty()); + Assertions.assertTrue(context.getAssignedSnapshotSplit().isEmpty()); + Assertions.assertTrue(context.getSplitCompletedOffsets().isEmpty()); + } + + private static Map createAssignedSplits() { + return Stream.of( + new AbstractMap.SimpleEntry<>( + "db1.table1.1", + new SnapshotSplit( + "db1.table1.1", + TableId.parse("db1.table1"), + null, + null, + null)), + new AbstractMap.SimpleEntry<>( + "db1.table1.2", + new SnapshotSplit( + "db1.table1.2", + TableId.parse("db1.table1"), + null, + null, + null)), + new AbstractMap.SimpleEntry<>( + "db1.table2.1", + new SnapshotSplit( + "db1.table2.1", + TableId.parse("db1.table2"), + null, + null, + null)), + new AbstractMap.SimpleEntry<>( + "db1.table2.2", + new SnapshotSplit( + "db1.table2.2", + TableId.parse("db1.table2"), + null, + null, + null))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + private static Map createSplitCompletedOffsets() { + return Stream.of( + new AbstractMap.SimpleEntry<>( + "db1.table1.1", new SnapshotSplitWatermark(null, null, null)), + new AbstractMap.SimpleEntry<>( + "db1.table1.2", new SnapshotSplitWatermark(null, null, null)), + new AbstractMap.SimpleEntry<>( + "db1.table2.1", new SnapshotSplitWatermark(null, null, null)), + new AbstractMap.SimpleEntry<>( + "db1.table2.2", new SnapshotSplitWatermark(null, null, null))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcherTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcherTest.java new file mode 100644 index 00000000000..64ac4f4a0c1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcherTest.java @@ -0,0 +1,367 @@ +/* + * 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.seatunnel.connectors.cdc.base.source.reader.external; + +import org.apache.seatunnel.connectors.cdc.base.schema.SchemaChangeResolver; +import org.apache.seatunnel.connectors.cdc.base.source.split.SourceRecords; +import org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkEvent; +import org.apache.seatunnel.connectors.cdc.base.utils.SourceRecordUtils; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import io.debezium.data.Envelope; +import io.debezium.heartbeat.Heartbeat; +import io.debezium.pipeline.DataChangeEvent; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +public class IncrementalSourceStreamFetcherTest { + + @Test + public void testSplitSchemaChangeStream() throws Exception { + IncrementalSourceStreamFetcher fetcher = createFetcher(); + + List inputEvents = new ArrayList<>(); + List records = new ArrayList<>(); + inputEvents.add(new DataChangeEvent(createDataEvent())); + inputEvents.add(new DataChangeEvent(createDataEvent())); + Iterator outputEvents = fetcher.splitSchemaChangeStream(inputEvents); + outputEvents.forEachRemaining(records::add); + + Assertions.assertEquals(1, records.size()); + Assertions.assertEquals(2, records.get(0).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(0).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(0).getSourceRecordList().get(1))); + + inputEvents = new ArrayList<>(); + records = new ArrayList<>(); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + outputEvents = fetcher.splitSchemaChangeStream(inputEvents); + outputEvents.forEachRemaining(records::add); + + Assertions.assertEquals(2, records.size()); + Assertions.assertEquals(1, records.get(0).getSourceRecordList().size()); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeBeforeWatermarkEvent( + records.get(0).getSourceRecordList().get(0))); + Assertions.assertEquals(3, records.get(1).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(1).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(1).getSourceRecordList().get(1))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeAfterWatermarkEvent( + records.get(1).getSourceRecordList().get(2))); + + inputEvents = new ArrayList<>(); + records = new ArrayList<>(); + inputEvents.add(new DataChangeEvent(createDataEvent())); + inputEvents.add(new DataChangeEvent(createDataEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + outputEvents = fetcher.splitSchemaChangeStream(inputEvents); + outputEvents.forEachRemaining(records::add); + + Assertions.assertEquals(2, records.size()); + Assertions.assertEquals(3, records.get(0).getSourceRecordList().size()); + Assertions.assertEquals(3, records.get(1).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(0).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(0).getSourceRecordList().get(1))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeBeforeWatermarkEvent( + records.get(0).getSourceRecordList().get(2))); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(1).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(1).getSourceRecordList().get(1))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeAfterWatermarkEvent( + records.get(1).getSourceRecordList().get(2))); + + inputEvents = new ArrayList<>(); + records = new ArrayList<>(); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + inputEvents.add(new DataChangeEvent(createDataEvent())); + inputEvents.add(new DataChangeEvent(createDataEvent())); + outputEvents = fetcher.splitSchemaChangeStream(inputEvents); + outputEvents.forEachRemaining(records::add); + + Assertions.assertEquals(3, records.size()); + Assertions.assertEquals(1, records.get(0).getSourceRecordList().size()); + Assertions.assertEquals(3, records.get(1).getSourceRecordList().size()); + Assertions.assertEquals(2, records.get(2).getSourceRecordList().size()); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeBeforeWatermarkEvent( + records.get(0).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(1).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(1).getSourceRecordList().get(1))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeAfterWatermarkEvent( + records.get(1).getSourceRecordList().get(2))); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(2).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(2).getSourceRecordList().get(1))); + + inputEvents = new ArrayList<>(); + records = new ArrayList<>(); + inputEvents.add(new DataChangeEvent(createDataEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + inputEvents.add(new DataChangeEvent(createDataEvent())); + outputEvents = fetcher.splitSchemaChangeStream(inputEvents); + outputEvents.forEachRemaining(records::add); + + Assertions.assertEquals(3, records.size()); + Assertions.assertEquals(2, records.get(0).getSourceRecordList().size()); + Assertions.assertEquals(3, records.get(1).getSourceRecordList().size()); + Assertions.assertEquals(1, records.get(2).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(0).getSourceRecordList().get(0))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeBeforeWatermarkEvent( + records.get(0).getSourceRecordList().get(1))); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(1).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(1).getSourceRecordList().get(1))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeAfterWatermarkEvent( + records.get(1).getSourceRecordList().get(2))); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(2).getSourceRecordList().get(0))); + + inputEvents = new ArrayList<>(); + records = new ArrayList<>(); + inputEvents.add(new DataChangeEvent(createDataEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + inputEvents.add(new DataChangeEvent(createDataEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + outputEvents = fetcher.splitSchemaChangeStream(inputEvents); + outputEvents.forEachRemaining(records::add); + + Assertions.assertEquals(4, records.size()); + Assertions.assertEquals(2, records.get(0).getSourceRecordList().size()); + Assertions.assertEquals(2, records.get(1).getSourceRecordList().size()); + Assertions.assertEquals(2, records.get(2).getSourceRecordList().size()); + Assertions.assertEquals(2, records.get(3).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(0).getSourceRecordList().get(0))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeBeforeWatermarkEvent( + records.get(0).getSourceRecordList().get(1))); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(1).getSourceRecordList().get(0))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeAfterWatermarkEvent( + records.get(1).getSourceRecordList().get(1))); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(2).getSourceRecordList().get(0))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeBeforeWatermarkEvent( + records.get(2).getSourceRecordList().get(1))); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(3).getSourceRecordList().get(0))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeAfterWatermarkEvent( + records.get(3).getSourceRecordList().get(1))); + + inputEvents = new ArrayList<>(); + records = new ArrayList<>(); + inputEvents.add(new DataChangeEvent(createHeartbeatEvent())); + inputEvents.add(new DataChangeEvent(createDataEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + inputEvents.add(new DataChangeEvent(createHeartbeatEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + inputEvents.add(new DataChangeEvent(createDataEvent())); + inputEvents.add(new DataChangeEvent(createDataEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + inputEvents.add(new DataChangeEvent(createHeartbeatEvent())); + inputEvents.add(new DataChangeEvent(createDataEvent())); + inputEvents.add(new DataChangeEvent(createHeartbeatEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + inputEvents.add(new DataChangeEvent(createHeartbeatEvent())); + inputEvents.add(new DataChangeEvent(createDataEvent())); + inputEvents.add(new DataChangeEvent(createSchemaChangeEvent())); + inputEvents.add(new DataChangeEvent(createDataEvent())); + inputEvents.add(new DataChangeEvent(createHeartbeatEvent())); + outputEvents = fetcher.splitSchemaChangeStream(inputEvents); + outputEvents.forEachRemaining(records::add); + + Assertions.assertEquals(11, records.size()); + Assertions.assertEquals(3, records.get(0).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isHeartbeatRecord(records.get(0).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(0).getSourceRecordList().get(1))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeBeforeWatermarkEvent( + records.get(0).getSourceRecordList().get(2))); + Assertions.assertEquals(2, records.get(1).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(1).getSourceRecordList().get(0))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeAfterWatermarkEvent( + records.get(1).getSourceRecordList().get(1))); + Assertions.assertEquals(2, records.get(2).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isHeartbeatRecord(records.get(2).getSourceRecordList().get(0))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeBeforeWatermarkEvent( + records.get(2).getSourceRecordList().get(1))); + Assertions.assertEquals(2, records.get(3).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(3).getSourceRecordList().get(0))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeAfterWatermarkEvent( + records.get(3).getSourceRecordList().get(1))); + Assertions.assertEquals(3, records.get(4).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(4).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(4).getSourceRecordList().get(1))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeBeforeWatermarkEvent( + records.get(4).getSourceRecordList().get(2))); + Assertions.assertEquals(2, records.get(5).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(5).getSourceRecordList().get(0))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeAfterWatermarkEvent( + records.get(5).getSourceRecordList().get(1))); + Assertions.assertEquals(4, records.get(6).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isHeartbeatRecord(records.get(6).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(6).getSourceRecordList().get(1))); + Assertions.assertTrue( + SourceRecordUtils.isHeartbeatRecord(records.get(6).getSourceRecordList().get(2))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeBeforeWatermarkEvent( + records.get(6).getSourceRecordList().get(3))); + Assertions.assertEquals(3, records.get(7).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(7).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(7).getSourceRecordList().get(1))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeAfterWatermarkEvent( + records.get(7).getSourceRecordList().get(2))); + Assertions.assertEquals(3, records.get(8).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isHeartbeatRecord(records.get(8).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(8).getSourceRecordList().get(1))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeBeforeWatermarkEvent( + records.get(8).getSourceRecordList().get(2))); + Assertions.assertEquals(2, records.get(9).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isSchemaChangeEvent(records.get(9).getSourceRecordList().get(0))); + Assertions.assertTrue( + WatermarkEvent.isSchemaChangeAfterWatermarkEvent( + records.get(9).getSourceRecordList().get(1))); + Assertions.assertEquals(2, records.get(10).getSourceRecordList().size()); + Assertions.assertTrue( + SourceRecordUtils.isDataChangeRecord(records.get(10).getSourceRecordList().get(0))); + Assertions.assertTrue( + SourceRecordUtils.isHeartbeatRecord(records.get(10).getSourceRecordList().get(1))); + } + + static SourceRecord createSchemaChangeEvent() { + Schema keySchema = + SchemaBuilder.struct().name(SourceRecordUtils.SCHEMA_CHANGE_EVENT_KEY_NAME).build(); + SourceRecord record = + new SourceRecord( + Collections.emptyMap(), + Collections.emptyMap(), + null, + keySchema, + null, + null, + null); + Assertions.assertTrue(SourceRecordUtils.isSchemaChangeEvent(record)); + return record; + } + + static SourceRecord createDataEvent() { + Schema valueSchema = + SchemaBuilder.struct() + .field(Envelope.FieldName.OPERATION, Schema.STRING_SCHEMA) + .build(); + Struct value = new Struct(valueSchema); + value.put(valueSchema.field(Envelope.FieldName.OPERATION), "c"); + SourceRecord record = + new SourceRecord( + Collections.emptyMap(), + Collections.emptyMap(), + null, + null, + null, + valueSchema, + value); + Assertions.assertTrue(SourceRecordUtils.isDataChangeRecord(record)); + return record; + } + + static SourceRecord createHeartbeatEvent() throws InterruptedException { + Heartbeat heartbeat = Heartbeat.create(Duration.ofNanos(1), "test", "test"); + AtomicReference eventRef = new AtomicReference<>(); + heartbeat.forcedBeat( + Collections.singletonMap("heartbeat", "heartbeat"), + Collections.singletonMap("heartbeat", "heartbeat"), + sourceRecord -> eventRef.set(sourceRecord)); + return eventRef.get(); + } + + static IncrementalSourceStreamFetcher createFetcher() { + SchemaChangeResolver schemaChangeResolver = mock(SchemaChangeResolver.class); + when(schemaChangeResolver.support(any())).thenReturn(true); + IncrementalSourceStreamFetcher fetcher = + new IncrementalSourceStreamFetcher(null, 0, schemaChangeResolver); + IncrementalSourceStreamFetcher spy = spy(fetcher); + doReturn(true).when(spy).shouldEmit(any()); + return spy; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/split/state/IncrementalSplitStateTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/split/state/IncrementalSplitStateTest.java new file mode 100644 index 00000000000..4a0b40852a2 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/split/state/IncrementalSplitStateTest.java @@ -0,0 +1,169 @@ +/* + * 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.seatunnel.connectors.cdc.base.source.split.state; + +import org.apache.seatunnel.connectors.cdc.base.source.event.SnapshotSplitWatermark; +import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; +import org.apache.seatunnel.connectors.cdc.base.source.split.CompletedSnapshotSplitInfo; +import org.apache.seatunnel.connectors.cdc.base.source.split.IncrementalSplit; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import io.debezium.relational.TableId; +import lombok.AllArgsConstructor; +import lombok.ToString; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class IncrementalSplitStateTest { + + @Test + public void testMarkEnterPureIncrementPhaseIfNeed() { + Offset startupOffset = new TestOffset(100); + List snapshotSplits = Collections.emptyList(); + IncrementalSplit split = createIncrementalSplit(startupOffset, snapshotSplits); + IncrementalSplitState splitState = new IncrementalSplitState(split); + Assertions.assertNull(splitState.getMaxSnapshotSplitsHighWatermark()); + Assertions.assertTrue(splitState.isEnterPureIncrementPhase()); + Assertions.assertFalse(splitState.markEnterPureIncrementPhaseIfNeed(null)); + + startupOffset = new TestOffset(100); + snapshotSplits = + Stream.of( + createCompletedSnapshotSplitInfo( + "test1", new TestOffset(100), new TestOffset(100)), + createCompletedSnapshotSplitInfo( + "test2", new TestOffset(100), new TestOffset(100))) + .collect(Collectors.toList()); + split = createIncrementalSplit(startupOffset, snapshotSplits); + splitState = new IncrementalSplitState(split); + Assertions.assertEquals(startupOffset, splitState.getMaxSnapshotSplitsHighWatermark()); + Assertions.assertFalse(splitState.isEnterPureIncrementPhase()); + Assertions.assertFalse(splitState.markEnterPureIncrementPhaseIfNeed(new TestOffset(99))); + Assertions.assertFalse(splitState.isEnterPureIncrementPhase()); + Assertions.assertFalse(snapshotSplits.isEmpty()); + Assertions.assertTrue(splitState.markEnterPureIncrementPhaseIfNeed(new TestOffset(100))); + Assertions.assertTrue(snapshotSplits.isEmpty()); + Assertions.assertFalse(splitState.markEnterPureIncrementPhaseIfNeed(new TestOffset(100))); + Assertions.assertFalse(splitState.markEnterPureIncrementPhaseIfNeed(new TestOffset(101))); + + startupOffset = new TestOffset(100); + snapshotSplits = + Stream.of( + createCompletedSnapshotSplitInfo( + "test1", new TestOffset(1), new TestOffset(50)), + createCompletedSnapshotSplitInfo( + "test2", new TestOffset(50), new TestOffset(200))) + .collect(Collectors.toList()); + split = createIncrementalSplit(startupOffset, snapshotSplits); + splitState = new IncrementalSplitState(split); + Assertions.assertEquals( + new TestOffset(200), splitState.getMaxSnapshotSplitsHighWatermark()); + Assertions.assertFalse(splitState.isEnterPureIncrementPhase()); + Assertions.assertTrue(splitState.markEnterPureIncrementPhaseIfNeed(new TestOffset(201))); + Assertions.assertTrue(splitState.isEnterPureIncrementPhase()); + Assertions.assertTrue(snapshotSplits.isEmpty()); + Assertions.assertFalse(splitState.markEnterPureIncrementPhaseIfNeed(new TestOffset(200))); + Assertions.assertTrue(splitState.isEnterPureIncrementPhase()); + Assertions.assertFalse(splitState.markEnterPureIncrementPhaseIfNeed(new TestOffset(201))); + Assertions.assertFalse(splitState.markEnterPureIncrementPhaseIfNeed(new TestOffset(202))); + } + + @Test + public void testAutoEnterPureIncrementPhaseIfAllowed() { + Offset startupOffset = new TestOffset(100); + List snapshotSplits = Collections.emptyList(); + IncrementalSplit split = createIncrementalSplit(startupOffset, snapshotSplits); + IncrementalSplitState splitState = new IncrementalSplitState(split); + Assertions.assertTrue(splitState.isEnterPureIncrementPhase()); + Assertions.assertFalse(splitState.autoEnterPureIncrementPhaseIfAllowed()); + + startupOffset = new TestOffset(100); + snapshotSplits = + Stream.of( + createCompletedSnapshotSplitInfo( + "test1", new TestOffset(100), new TestOffset(100)), + createCompletedSnapshotSplitInfo( + "test2", new TestOffset(100), new TestOffset(100))) + .collect(Collectors.toList()); + split = createIncrementalSplit(startupOffset, snapshotSplits); + splitState = new IncrementalSplitState(split); + + Assertions.assertFalse(splitState.isEnterPureIncrementPhase()); + Assertions.assertTrue(splitState.autoEnterPureIncrementPhaseIfAllowed()); + Assertions.assertTrue(splitState.isEnterPureIncrementPhase()); + Assertions.assertFalse(splitState.autoEnterPureIncrementPhaseIfAllowed()); + Assertions.assertTrue(splitState.isEnterPureIncrementPhase()); + + startupOffset = new TestOffset(100); + snapshotSplits = + Stream.of( + createCompletedSnapshotSplitInfo( + "test1", new TestOffset(100), new TestOffset(100)), + createCompletedSnapshotSplitInfo( + "test2", new TestOffset(100), new TestOffset(101))) + .collect(Collectors.toList()); + split = createIncrementalSplit(startupOffset, snapshotSplits); + splitState = new IncrementalSplitState(split); + Assertions.assertFalse(splitState.isEnterPureIncrementPhase()); + Assertions.assertFalse(splitState.autoEnterPureIncrementPhaseIfAllowed()); + } + + private static IncrementalSplit createIncrementalSplit( + Offset startupOffset, List snapshotSplits) { + return new IncrementalSplit( + "test", + Arrays.asList(new TableId("db", "schema", "table")), + startupOffset, + null, + snapshotSplits, + null); + } + + private static CompletedSnapshotSplitInfo createCompletedSnapshotSplitInfo( + String splitId, Offset lowWatermark, Offset highWatermark) { + return new CompletedSnapshotSplitInfo( + splitId, + new TableId("db", "schema", "table"), + null, + null, + null, + new SnapshotSplitWatermark(null, lowWatermark, highWatermark)); + } + + @ToString + @AllArgsConstructor + static class TestOffset extends Offset { + private int offset; + + @Override + public int compareTo(Offset o) { + return Integer.compare(offset, ((TestOffset) o).offset); + } + + @Override + public boolean equals(Object o) { + return o instanceof TestOffset && offset == ((TestOffset) o).offset; + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/io/debezium/connector/oracle/logminer/LogMinerStreamingChangeEventSource.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/io/debezium/connector/oracle/logminer/LogMinerStreamingChangeEventSource.java index 4ce4d17a028..0e9e9368657 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/io/debezium/connector/oracle/logminer/LogMinerStreamingChangeEventSource.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/io/debezium/connector/oracle/logminer/LogMinerStreamingChangeEventSource.java @@ -241,6 +241,7 @@ public void execute(ChangeEventSourceContext context, OracleOffsetContext offset // log before proceeding. if (archiveLogOnlyMode && startScn.equals(endScn)) { pauseBetweenMiningSessions(); + dispatcher.dispatchHeartbeatEvent(offsetContext); continue; } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java index 3053598c664..05eba0d96f9 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java @@ -181,6 +181,7 @@ public void execute( && shouldIncreaseFromLsn) { LOGGER.debug("No change in the database"); metronome.pause(); + dispatcher.dispatchHeartbeatEvent(offsetContext); continue; } diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/source/reader/SourceReaderBase.java b/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/source/reader/SourceReaderBase.java index 668ed78dcac..29dd2ff6f5e 100644 --- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/source/reader/SourceReaderBase.java +++ b/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/source/reader/SourceReaderBase.java @@ -26,6 +26,7 @@ import org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.SplitFetcherManager; import org.apache.seatunnel.connectors.seatunnel.common.source.reader.splitreader.SplitReader; +import lombok.Getter; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; @@ -62,7 +63,7 @@ public abstract class SourceReaderBase currentFetch; - private SplitContext currentSplitContext; + protected SplitContext currentSplitContext; private Collector currentSplitOutput; private boolean noMoreSplitsAssignment; @@ -234,9 +235,9 @@ public int getNumberOfCurrentlyAssignedSplits() { protected abstract SplitT toSplitType(String splitId, SplitStateT splitState); @RequiredArgsConstructor - private static final class SplitContext { + protected static final class SplitContext { final String splitId; - final SplitStateT state; + @Getter final SplitStateT state; Collector splitOutput; Collector getOrCreateSplitOutput(Collector output) { From 80f392afbb654145ea3188ba51b1f4bafcd2ef5b Mon Sep 17 00:00:00 2001 From: L-Gryps <42635285+L-Gryps@users.noreply.github.com> Date: Tue, 2 Apr 2024 16:42:52 +0800 Subject: [PATCH 9/9] [feature][connector-v2] add xugudb connector (#6561) --- .github/workflows/backend.yml | 2 +- docs/en/connector-v2/sink/Jdbc.md | 1 + docs/en/connector-v2/source/Jdbc.md | 1 + .../connector-jdbc/pom.xml | 7 + .../jdbc/catalog/xugu/XuguCatalog.java | 266 +++++++ .../jdbc/catalog/xugu/XuguCatalogFactory.java | 63 ++ .../xugu/XuguCreateTableSqlBuilder.java | 141 ++++ .../internal/dialect/DatabaseIdentifier.java | 1 + .../internal/dialect/xugu/XuguDialect.java | 231 ++++++ .../dialect/xugu/XuguDialectFactory.java | 45 ++ .../dialect/xugu/XuguJdbcRowConverter.java | 29 + .../dialect/xugu/XuguTypeConverter.java | 385 ++++++++++ .../internal/dialect/xugu/XuguTypeMapper.java | 63 ++ .../dialect/xugu/XuguTypeConverterTest.java | 660 ++++++++++++++++++ .../src/main/assembly/assembly-bin-ci.xml | 1 + .../connector-jdbc-e2e-part-7/pom.xml | 5 + .../connectors/seatunnel/jdbc/JdbcXuguIT.java | 246 +++++++ .../resources/jdbc_xugu_source_and_sink.conf | 47 ++ .../jdbc_xugu_source_and_upsert_sink.conf | 48 ++ 19 files changed, 2241 insertions(+), 1 deletion(-) create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalogFactory.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCreateTableSqlBuilder.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguDialect.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguDialectFactory.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguJdbcRowConverter.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverter.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeMapper.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverterTest.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcXuguIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_xugu_source_and_sink.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_xugu_source_and_upsert_sink.conf diff --git a/.github/workflows/backend.yml b/.github/workflows/backend.yml index 519cf8533d2..9975d477dae 100644 --- a/.github/workflows/backend.yml +++ b/.github/workflows/backend.yml @@ -997,7 +997,7 @@ jobs: java-version: ${{ matrix.java }} distribution: 'temurin' cache: 'maven' - - name: run jdbc connectors integration test (part-6) + - name: run jdbc connectors integration test (part-7) if: needs.changes.outputs.api == 'true' run: | ./mvnw -B -T 1 verify -DskipUT=true -DskipIT=false -D"license.skipAddThirdParty"=true --no-snapshot-updates -pl :connector-jdbc-e2e-part-7 -am -Pci diff --git a/docs/en/connector-v2/sink/Jdbc.md b/docs/en/connector-v2/sink/Jdbc.md index f0b74414a4e..c2591761ec5 100644 --- a/docs/en/connector-v2/sink/Jdbc.md +++ b/docs/en/connector-v2/sink/Jdbc.md @@ -235,6 +235,7 @@ there are some reference value for params above. | Vertica | com.vertica.jdbc.Driver | jdbc:vertica://localhost:5433 | / | https://repo1.maven.org/maven2/com/vertica/jdbc/vertica-jdbc/12.0.3-0/vertica-jdbc-12.0.3-0.jar | | Kingbase | com.kingbase8.Driver | jdbc:kingbase8://localhost:54321/db_test | / | https://repo1.maven.org/maven2/cn/com/kingbase/kingbase8/8.6.0/kingbase8-8.6.0.jar | | OceanBase | com.oceanbase.jdbc.Driver | jdbc:oceanbase://localhost:2881 | / | https://repo1.maven.org/maven2/com/oceanbase/oceanbase-client/2.4.3/oceanbase-client-2.4.3.jar | +| xugu | com.xugu.cloudjdbc.Driver | jdbc:xugu://localhost:5138 | / | https://repo1.maven.org/maven2/com/xugudb/xugu-jdbc/12.2.0/xugu-jdbc-12.2.0.jar | ## Example diff --git a/docs/en/connector-v2/source/Jdbc.md b/docs/en/connector-v2/source/Jdbc.md index 09c3ab636d7..225576001d7 100644 --- a/docs/en/connector-v2/source/Jdbc.md +++ b/docs/en/connector-v2/source/Jdbc.md @@ -227,6 +227,7 @@ there are some reference value for params above. | Kingbase | com.kingbase8.Driver | jdbc:kingbase8://localhost:54321/db_test | https://repo1.maven.org/maven2/cn/com/kingbase/kingbase8/8.6.0/kingbase8-8.6.0.jar | | OceanBase | com.oceanbase.jdbc.Driver | jdbc:oceanbase://localhost:2881 | https://repo1.maven.org/maven2/com/oceanbase/oceanbase-client/2.4.3/oceanbase-client-2.4.3.jar | | Hive | org.apache.hive.jdbc.HiveDriver | jdbc:hive2://localhost:10000 | https://repo1.maven.org/maven2/org/apache/hive/hive-jdbc/3.1.3/hive-jdbc-3.1.3-standalone.jar | +| xugu | com.xugu.cloudjdbc.Driver | jdbc:xugu://localhost:5138 | https://repo1.maven.org/maven2/com/xugudb/xugu-jdbc/12.2.0/xugu-jdbc-12.2.0.jar | ## Example diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 5880036c90f..db8c95dd0fd 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -50,6 +50,7 @@ 8.6.0 3.1.3 2.4.3 + 12.2.0 @@ -188,6 +189,12 @@ ${oceanbase.jdbc.version} provided + + com.xugudb + xugu-jdbc + ${xugu.jdbc.version} + provided + diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java new file mode 100644 index 00000000000..462e109c76a --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalog.java @@ -0,0 +1,266 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc.catalog.xugu; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.exception.CatalogException; +import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.utils.CatalogUtils; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu.XuguTypeConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu.XuguTypeMapper; + +import org.apache.commons.lang3.StringUtils; + +import lombok.extern.slf4j.Slf4j; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +@Slf4j +public class XuguCatalog extends AbstractJdbcCatalog { + + protected static List EXCLUDED_SCHEMAS = + Collections.unmodifiableList(Arrays.asList("GUEST", "SYSAUDITOR", "SYSSSO")); + + private static final String SELECT_COLUMNS_SQL_TEMPLATE = + "SELECT\n" + + " dc.COLUMN_NAME,\n" + + " CASE\n" + + " WHEN dc.TYPE_NAME LIKE 'INTERVAL%%' THEN 'INTERVAL' ELSE REGEXP_SUBSTR(dc.TYPE_NAME, '^[^(]+')\n" + + " END AS TYPE_NAME,\n" + + " dc.TYPE_NAME ||\n" + + " CASE\n" + + " WHEN dc.TYPE_NAME IN ('VARCHAR', 'CHAR') THEN '(' || dc.COLUMN_LENGTH || ')'\n" + + " WHEN dc.TYPE_NAME IN ('NUMERIC') AND dc.COLUMN_PRECISION IS NOT NULL AND dc.COLUMN_SCALE IS NOT NULL THEN '(' || dc.COLUMN_PRECISION || ', ' || dc.COLUMN_SCALE || ')'\n" + + " WHEN dc.TYPE_NAME IN ('NUMERIC') AND dc.COLUMN_PRECISION IS NOT NULL AND dc.COLUMN_SCALE IS NULL THEN '(' || dc.COLUMN_PRECISION || ')'\n" + + " WHEN dc.TYPE_NAME IN ('TIMESTAMP') THEN '(' || dc.COLUMN_SCALE || ')'\n" + + " END AS FULL_TYPE_NAME,\n" + + " dc.COLUMN_LENGTH,\n" + + " dc.COLUMN_PRECISION,\n" + + " dc.COLUMN_SCALE,\n" + + " dc.COLUMN_COMMENT,\n" + + " dc.DEFAULT_VALUE,\n" + + " CASE\n" + + " dc.IS_NULLABLE WHEN TRUE THEN 'NO' ELSE 'YES'\n" + + " END AS IS_NULLABLE\n" + + "FROM\n" + + " (\n" + + " SELECT\n" + + " c.col_name AS COLUMN_NAME,\n" + + " CASE\n" + + " WHEN c.type_name = 'CHAR' AND c.\"VARYING\" = TRUE THEN 'VARCHAR'\n" + + " WHEN c.type_name = 'DATETIME' AND c.TIMESTAMP_T = 'i' THEN 'TIMESTAMP' ELSE c.type_name\n" + + " END AS TYPE_NAME,\n" + + " DECODE(c.type_name,\n" + + " 'TINYINT', 1, 'SMALLINT', 2,\n" + + " 'INTEGER', 4, 'BIGINT', 8,\n" + + " 'FLOAT', 4, 'DOUBLE', 8,\n" + + " 'NUMERIC', 17,\n" + + " 'CHAR', DECODE(c.scale, -1, 60000, c.scale),\n" + + " 'DATE', 4, 'DATETIME', 8,\n" + + " 'TIMESTAMP', 8, 'DATETIME WITH TIME ZONE', 8,\n" + + " 'TIME', 4, 'TIME WITH TIME ZONE', 4,\n" + + " 'INTERVAL YEAR', 4, 'INTERVAL MONTH', 4,\n" + + " 'INTERVAL DAY', 4, 'INTERVAL HOUR', 4,\n" + + " 'INTERVAL MINUTE', 4, 'INTERVAL SECOND', 8,\n" + + " 'INTERVAL YEAR TO MONTH', 4,\n" + + " 'INTERVAL DAY TO HOUR', 4,\n" + + " 'INTERVAL DAY TO MINUTE', 4,\n" + + " 'INTERVAL DAY TO SECOND', 8,\n" + + " 'INTERVAL HOUR TO MINUTE', 4,\n" + + " 'INTERVAL HOUR TO SECOND', 8,\n" + + " 'INTERVAL MINUTE TO SECOND', 8,\n" + + " 'CLOB', 2147483648,\n" + + " 'BLOB', 2147483648, 'BINARY', 2147483648,\n" + + " 'GUID', 2, 'BOOLEAN', 1,\n" + + " 'ROWVERSION', 8, 'ROWID', 10, NULL) AS COLUMN_LENGTH,\n" + + " DECODE(TRUNC(c.scale / 65536), 0, NULL, TRUNC(c.scale / 65536)::INTEGER) AS COLUMN_PRECISION,\n" + + " DECODE(DECODE(c.type_name, 'CHAR',-1, c.scale),-1, NULL, MOD(c.scale, 65536)) AS COLUMN_SCALE,\n" + + " c.comments AS COLUMN_COMMENT,\n" + + " c.DEF_VAL AS DEFAULT_VALUE,\n" + + " c.NOT_NULl AS IS_NULLABLE\n" + + " FROM\n" + + " dba_columns c\n" + + " LEFT JOIN dba_tables tab ON\n" + + " c.db_id = tab.db_id\n" + + " AND c.table_id = tab.table_id\n" + + " LEFT JOIN dba_schemas sc ON\n" + + " tab.schema_id = sc.schema_id\n" + + " AND tab.db_id = sc.db_id\n" + + " WHERE\n" + + " sc.schema_name = '%s'\n" + + " AND tab.table_name = '%s'\n" + + ") AS dc \n"; + + public XuguCatalog( + String catalogName, + String username, + String pwd, + JdbcUrlUtil.UrlInfo urlInfo, + String defaultSchema) { + super(catalogName, username, pwd, urlInfo, defaultSchema); + } + + @Override + protected String getListDatabaseSql() { + return "SELECT DB_NAME FROM dba_databases"; + } + + @Override + protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { + return new XuguCreateTableSqlBuilder(table).build(tablePath); + } + + @Override + protected String getDropTableSql(TablePath tablePath) { + return String.format("DROP TABLE %s", tablePath.getSchemaAndTableName("\"")); + } + + @Override + protected String getCreateDatabaseSql(String databaseName) { + return String.format("CREATE DATABASE \"%s\"", databaseName); + } + + @Override + protected String getDropDatabaseSql(String databaseName) { + return String.format("DROP DATABASE \"%s\"", databaseName); + } + + @Override + protected String getListTableSql(String databaseName) { + return "SELECT user_name ,table_name FROM all_users au \n" + + "INNER JOIN all_tables at ON au.user_id=at.user_id AND au.db_id=at.db_id"; + } + + @Override + protected String getTableName(ResultSet rs) throws SQLException { + if (EXCLUDED_SCHEMAS.contains(rs.getString(1))) { + return null; + } + return rs.getString(1) + "." + rs.getString(2); + } + + @Override + protected String getSelectColumnsSql(TablePath tablePath) { + return String.format( + SELECT_COLUMNS_SQL_TEMPLATE, tablePath.getSchemaName(), tablePath.getTableName()); + } + + @Override + protected Column buildColumn(ResultSet resultSet) throws SQLException { + String columnName = resultSet.getString("COLUMN_NAME"); + String typeName = resultSet.getString("TYPE_NAME"); + String fullTypeName = resultSet.getString("FULL_TYPE_NAME"); + long columnLength = resultSet.getLong("COLUMN_LENGTH"); + Long columnPrecision = resultSet.getObject("COLUMN_PRECISION", Long.class); + Integer columnScale = resultSet.getObject("COLUMN_SCALE", Integer.class); + String columnComment = resultSet.getString("COLUMN_COMMENT"); + Object defaultValue = resultSet.getObject("DEFAULT_VALUE"); + boolean isNullable = resultSet.getString("IS_NULLABLE").equals("YES"); + + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name(columnName) + .columnType(fullTypeName) + .dataType(typeName) + .length(columnLength) + .precision(columnPrecision) + .scale(columnScale) + .nullable(isNullable) + .defaultValue(defaultValue) + .comment(columnComment) + .build(); + return XuguTypeConverter.INSTANCE.convert(typeDefine); + } + + @Override + protected String getUrlFromDatabaseName(String databaseName) { + return defaultUrl; + } + + @Override + protected String getOptionTableName(TablePath tablePath) { + return tablePath.getSchemaAndTableName(); + } + + @Override + public boolean tableExists(TablePath tablePath) throws CatalogException { + try { + if (StringUtils.isNotBlank(tablePath.getDatabaseName())) { + return databaseExists(tablePath.getDatabaseName()) + && listTables(tablePath.getDatabaseName()) + .contains(tablePath.getSchemaAndTableName()); + } + return listTables().contains(tablePath.getSchemaAndTableName()); + } catch (DatabaseNotExistException e) { + return false; + } + } + + private List listTables() { + List databases = listDatabases(); + return listTables(databases.get(0)); + } + + @Override + public CatalogTable getTable(String sqlQuery) throws SQLException { + Connection defaultConnection = getConnection(defaultUrl); + return CatalogUtils.getCatalogTable(defaultConnection, sqlQuery, new XuguTypeMapper()); + } + + @Override + protected String getTruncateTableSql(TablePath tablePath) { + return String.format( + "TRUNCATE TABLE \"%s\".\"%s\"", + tablePath.getSchemaName(), tablePath.getTableName()); + } + + @Override + protected String getExistDataSql(TablePath tablePath) { + return String.format( + "SELECT * FROM \"%s\".\"%s\" WHERE ROWNUM = 1", + tablePath.getSchemaName(), tablePath.getTableName()); + } + + @Override + protected List getConstraintKeys(DatabaseMetaData metaData, TablePath tablePath) + throws SQLException { + try { + return getConstraintKeys( + metaData, + tablePath.getDatabaseName(), + tablePath.getSchemaName(), + tablePath.getTableName()); + } catch (SQLException e) { + log.info("Obtain constraint failure", e); + return new ArrayList<>(); + } + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalogFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalogFactory.java new file mode 100644 index 00000000000..ac0f3e24ae9 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCatalogFactory.java @@ -0,0 +1,63 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc.catalog.xugu; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.configuration.util.OptionValidationException; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.factory.CatalogFactory; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleURLParser; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; + +import com.google.auto.service.AutoService; + +import java.util.Optional; + +@AutoService(Factory.class) +public class XuguCatalogFactory implements CatalogFactory { + + @Override + public String factoryIdentifier() { + return DatabaseIdentifier.XUGU; + } + + @Override + public Catalog createCatalog(String catalogName, ReadonlyConfig options) { + String urlWithDatabase = options.get(JdbcCatalogOptions.BASE_URL); + JdbcUrlUtil.UrlInfo urlInfo = OracleURLParser.parse(urlWithDatabase); + Optional defaultDatabase = urlInfo.getDefaultDatabase(); + if (!defaultDatabase.isPresent()) { + throw new OptionValidationException(JdbcCatalogOptions.BASE_URL); + } + return new XuguCatalog( + catalogName, + options.get(JdbcCatalogOptions.USERNAME), + options.get(JdbcCatalogOptions.PASSWORD), + urlInfo, + options.get(JdbcCatalogOptions.SCHEMA)); + } + + @Override + public OptionRule optionRule() { + return JdbcCatalogOptions.BASE_RULE.build(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCreateTableSqlBuilder.java new file mode 100644 index 00000000000..19bce1a8ca0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/xugu/XuguCreateTableSqlBuilder.java @@ -0,0 +1,141 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc.catalog.xugu; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.utils.CatalogUtils; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu.XuguTypeConverter; + +import org.apache.commons.lang3.StringUtils; + +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; + +public class XuguCreateTableSqlBuilder { + + private List columns; + private PrimaryKey primaryKey; + private String sourceCatalogName; + private String fieldIde; + + public XuguCreateTableSqlBuilder(CatalogTable catalogTable) { + this.columns = catalogTable.getTableSchema().getColumns(); + this.primaryKey = catalogTable.getTableSchema().getPrimaryKey(); + this.sourceCatalogName = catalogTable.getCatalogName(); + this.fieldIde = catalogTable.getOptions().get("fieldIde"); + } + + public String build(TablePath tablePath) { + StringBuilder createTableSql = new StringBuilder(); + createTableSql + .append("CREATE TABLE ") + .append(tablePath.getSchemaAndTableName("\"")) + .append(" (\n"); + + List columnSqls = + columns.stream() + .map(column -> CatalogUtils.getFieldIde(buildColumnSql(column), fieldIde)) + .collect(Collectors.toList()); + + // Add primary key directly in the create table statement + if (primaryKey != null + && primaryKey.getColumnNames() != null + && primaryKey.getColumnNames().size() > 0) { + columnSqls.add(buildPrimaryKeySql(primaryKey)); + } + + createTableSql.append(String.join(",\n", columnSqls)); + createTableSql.append("\n)"); + + List commentSqls = + columns.stream() + .filter(column -> StringUtils.isNotBlank(column.getComment())) + .map( + column -> + buildColumnCommentSql( + column, tablePath.getSchemaAndTableName("\""))) + .collect(Collectors.toList()); + + if (!commentSqls.isEmpty()) { + createTableSql.append(";\n"); + createTableSql.append(String.join(";\n", commentSqls)); + } + + return createTableSql.toString(); + } + + private String buildColumnSql(Column column) { + StringBuilder columnSql = new StringBuilder(); + columnSql.append("\"").append(column.getName()).append("\" "); + + String columnType = + StringUtils.equalsIgnoreCase(DatabaseIdentifier.XUGU, sourceCatalogName) + ? column.getSourceType() + : XuguTypeConverter.INSTANCE.reconvert(column).getColumnType(); + columnSql.append(columnType); + + if (!column.isNullable()) { + columnSql.append(" NOT NULL"); + } + + return columnSql.toString(); + } + + private String buildPrimaryKeySql(PrimaryKey primaryKey) { + String randomSuffix = UUID.randomUUID().toString().replace("-", "").substring(0, 4); + String columnNamesString = + primaryKey.getColumnNames().stream() + .map(columnName -> "\"" + columnName + "\"") + .collect(Collectors.joining(", ")); + + // In xugu database, the maximum length for an identifier is 30 characters. + String primaryKeyStr = primaryKey.getPrimaryKey(); + if (primaryKeyStr.length() > 25) { + primaryKeyStr = primaryKeyStr.substring(0, 25); + } + + return CatalogUtils.getFieldIde( + "CONSTRAINT " + + primaryKeyStr + + "_" + + randomSuffix + + " PRIMARY KEY (" + + columnNamesString + + ")", + fieldIde); + } + + private String buildColumnCommentSql(Column column, String tableName) { + StringBuilder columnCommentSql = new StringBuilder(); + columnCommentSql + .append(CatalogUtils.quoteIdentifier("COMMENT ON COLUMN ", fieldIde)) + .append(tableName) + .append("."); + columnCommentSql + .append(CatalogUtils.quoteIdentifier(column.getName(), fieldIde, "\"")) + .append(CatalogUtils.quoteIdentifier(" IS '", fieldIde)) + .append(column.getComment()) + .append("'"); + return columnCommentSql.toString(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java index 3b1738afb27..2f6aabc502c 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/DatabaseIdentifier.java @@ -38,4 +38,5 @@ public class DatabaseIdentifier { public static final String VERTICA = "Vertica"; public static final String OCENABASE = "OceanBase"; public static final String TIDB = "TiDB"; + public static final String XUGU = "XUGU"; } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguDialect.java new file mode 100644 index 00000000000..1ef617b3936 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguDialect.java @@ -0,0 +1,231 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu; + +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.dialectenum.FieldIdeEnum; +import org.apache.seatunnel.connectors.seatunnel.jdbc.source.JdbcSourceTable; + +import org.apache.commons.lang3.StringUtils; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +public class XuguDialect implements JdbcDialect { + + private static final int DEFAULT_XUGU_FETCH_SIZE = 500; + public String fieldIde = FieldIdeEnum.ORIGINAL.getValue(); + + public XuguDialect(String fieldIde) { + this.fieldIde = fieldIde; + } + + public XuguDialect() {} + + @Override + public String dialectName() { + return DatabaseIdentifier.XUGU; + } + + @Override + public JdbcRowConverter getRowConverter() { + return new XuguJdbcRowConverter(); + } + + @Override + public JdbcDialectTypeMapper getJdbcDialectTypeMapper() { + return new XuguTypeMapper(); + } + + @Override + public String quoteIdentifier(String identifier) { + if (identifier.contains(".")) { + String[] parts = identifier.split("\\."); + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < parts.length - 1; i++) { + sb.append("\"").append(parts[i]).append("\"").append("."); + } + return sb.append("\"") + .append(getFieldIde(parts[parts.length - 1], fieldIde)) + .append("\"") + .toString(); + } + + return "\"" + getFieldIde(identifier, fieldIde) + "\""; + } + + @Override + public String tableIdentifier(String database, String tableName) { + return quoteIdentifier(tableName); + } + + @Override + public String extractTableName(TablePath tablePath) { + return tablePath.getSchemaAndTableName(); + } + + @Override + public TablePath parse(String tablePath) { + return TablePath.of(tablePath, true); + } + + @Override + public String tableIdentifier(TablePath tablePath) { + return tablePath.getSchemaAndTableName(); + } + + @Override + public Optional getUpsertStatement( + String database, String tableName, String[] fieldNames, String[] uniqueKeyFields) { + List nonUniqueKeyFields = + Arrays.stream(fieldNames) + .filter(fieldName -> !Arrays.asList(uniqueKeyFields).contains(fieldName)) + .collect(Collectors.toList()); + String valuesBinding = + Arrays.stream(fieldNames) + .map(fieldName -> ":" + fieldName + " " + quoteIdentifier(fieldName)) + .collect(Collectors.joining(", ")); + + String usingClause = String.format("SELECT %s FROM DUAL", valuesBinding); + String onConditions = + Arrays.stream(uniqueKeyFields) + .map( + fieldName -> + String.format( + "TARGET.%s=SOURCE.%s", + quoteIdentifier(fieldName), + quoteIdentifier(fieldName))) + .collect(Collectors.joining(" AND ")); + String updateSetClause = + nonUniqueKeyFields.stream() + .map( + fieldName -> + String.format( + "TARGET.%s=SOURCE.%s", + quoteIdentifier(fieldName), + quoteIdentifier(fieldName))) + .collect(Collectors.joining(", ")); + String insertFields = + Arrays.stream(fieldNames) + .map(this::quoteIdentifier) + .collect(Collectors.joining(", ")); + String insertValues = + Arrays.stream(fieldNames) + .map(fieldName -> "SOURCE." + quoteIdentifier(fieldName)) + .collect(Collectors.joining(", ")); + + String upsertSQL = + String.format( + " MERGE INTO %s TARGET" + + " USING (%s) SOURCE" + + " ON (%s) " + + " WHEN MATCHED THEN" + + " UPDATE SET %s" + + " WHEN NOT MATCHED THEN" + + " INSERT (%s) VALUES (%s)", + tableIdentifier(database, tableName), + usingClause, + onConditions, + updateSetClause, + insertFields, + insertValues); + + return Optional.of(upsertSQL); + } + + @Override + public PreparedStatement creatPreparedStatement( + Connection connection, String queryTemplate, int fetchSize) throws SQLException { + PreparedStatement statement = + connection.prepareStatement( + queryTemplate, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY); + if (fetchSize > 0) { + statement.setFetchSize(fetchSize); + } else { + statement.setFetchSize(DEFAULT_XUGU_FETCH_SIZE); + } + return statement; + } + + @Override + public Object queryNextChunkMax( + Connection connection, + JdbcSourceTable table, + String columnName, + int chunkSize, + Object includedLowerBound) + throws SQLException { + String quotedColumn = quoteIdentifier(columnName); + String sqlQuery; + if (StringUtils.isNotBlank(table.getQuery())) { + sqlQuery = + String.format( + "SELECT MAX(%s) FROM (" + + "SELECT %s FROM (%s) WHERE %s >= ? ORDER BY %s ASC " + + ") WHERE ROWNUM <= %s", + quotedColumn, + quotedColumn, + table.getQuery(), + quotedColumn, + quotedColumn, + chunkSize); + } else { + sqlQuery = + String.format( + "SELECT MAX(%s) FROM (" + + "SELECT %s FROM %s WHERE %s >= ? ORDER BY %s ASC " + + ") WHERE ROWNUM <= %s", + quotedColumn, + quotedColumn, + table.getTablePath().getSchemaAndTableName(), + quotedColumn, + quotedColumn, + chunkSize); + } + + try (PreparedStatement ps = connection.prepareStatement(sqlQuery)) { + ps.setObject(1, includedLowerBound); + try (ResultSet rs = ps.executeQuery()) { + if (!rs.next()) { + // this should never happen + throw new SQLException( + String.format("No result returned after running query [%s]", sqlQuery)); + } + return rs.getObject(1); + } + } + } + + @Override + public ResultSetMetaData getResultSetMetaData(Connection conn, String query) + throws SQLException { + PreparedStatement ps = conn.prepareStatement(query); + return ps.executeQuery().getMetaData(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguDialectFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguDialectFactory.java new file mode 100644 index 00000000000..0e489b728b5 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguDialectFactory.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectFactory; + +import com.google.auto.service.AutoService; + +import javax.annotation.Nonnull; + +/** Factory for {@link XuguDialect}. */ +@AutoService(JdbcDialectFactory.class) +public class XuguDialectFactory implements JdbcDialectFactory { + + @Override + public boolean acceptsURL(String url) { + return url.startsWith("jdbc:xugu:"); + } + + @Override + public JdbcDialect create() { + return new XuguDialect(); + } + + @Override + public JdbcDialect create(@Nonnull String compatibleMode, String fieldIde) { + return new XuguDialect(fieldIde); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguJdbcRowConverter.java new file mode 100644 index 00000000000..4590761965c --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguJdbcRowConverter.java @@ -0,0 +1,29 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.AbstractJdbcRowConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; + +public class XuguJdbcRowConverter extends AbstractJdbcRowConverter { + + @Override + public String converterName() { + return DatabaseIdentifier.XUGU; + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverter.java new file mode 100644 index 00000000000..54a8805f3bc --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverter.java @@ -0,0 +1,385 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu; + +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.api.table.converter.TypeConverter; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; +import org.apache.seatunnel.common.exception.CommonError; +import org.apache.seatunnel.connectors.seatunnel.common.source.TypeDefineUtils; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; + +import com.google.auto.service.AutoService; +import lombok.extern.slf4j.Slf4j; + +// reference +// https://docs.xugudb.com/%E8%99%9A%E8%B0%B7%E6%95%B0%E6%8D%AE%E5%BA%93%E5%AF%B9%E5%A4%96%E5%8F%91%E5%B8%83/06%E5%8F%82%E8%80%83%E6%8C%87%E5%8D%97/SQL%E8%AF%AD%E6%B3%95%E5%8F%82%E8%80%83/%E6%95%B0%E6%8D%AE%E7%B1%BB%E5%9E%8B/%E6%A6%82%E8%BF%B0/ +@Slf4j +@AutoService(TypeConverter.class) +public class XuguTypeConverter implements TypeConverter { + // ============================data types===================== + // -------------------------number---------------------------- + public static final String XUGU_NUMERIC = "NUMERIC"; + public static final String XUGU_NUMBER = "NUMBER"; + public static final String XUGU_DECIMAL = "DECIMAL"; + public static final String XUGU_INTEGER = "INTEGER"; + public static final String XUGU_INT = "INT"; + public static final String XUGU_BIGINT = "BIGINT"; + public static final String XUGU_TINYINT = "TINYINT"; + public static final String XUGU_SMALLINT = "SMALLINT"; + public static final String XUGU_FLOAT = "FLOAT"; + public static final String XUGU_DOUBLE = "DOUBLE"; + + // ----------------------------string------------------------- + public static final String XUGU_CHAR = "CHAR"; + public static final String XUGU_NCHAR = "NCHAR"; + public static final String XUGU_VARCHAR = "VARCHAR"; + public static final String XUGU_VARCHAR2 = "VARCHAR2"; + public static final String XUGU_CLOB = "CLOB"; + + // ------------------------------time------------------------- + public static final String XUGU_DATE = "DATE"; + public static final String XUGU_TIME = "TIME"; + public static final String XUGU_TIMESTAMP = "TIMESTAMP"; + public static final String XUGU_DATETIME = "DATETIME"; + public static final String XUGU_DATETIME_WITH_TIME_ZONE = "DATETIME WITH TIME ZONE"; + public static final String XUGU_TIME_WITH_TIME_ZONE = "TIME WITH TIME ZONE"; + public static final String XUGU_TIMESTAMP_WITH_TIME_ZONE = "TIMESTAMP WITH TIME ZONE"; + + // ---------------------------binary--------------------------- + public static final String XUGU_BINARY = "BINARY"; + public static final String XUGU_BLOB = "BLOB"; + + // ---------------------------other--------------------------- + public static final String XUGU_GUID = "GUID"; + public static final String XUGU_BOOLEAN = "BOOLEAN"; + public static final String XUGU_BOOL = "BOOL"; + public static final String XUGU_JSON = "JSON"; + + public static final int MAX_PRECISION = 38; + public static final int DEFAULT_PRECISION = MAX_PRECISION; + public static final int MAX_SCALE = 38; + public static final int DEFAULT_SCALE = 18; + public static final int TIMESTAMP_DEFAULT_SCALE = 3; + public static final int MAX_TIMESTAMP_SCALE = 6; + public static final int MAX_TIME_SCALE = 3; + public static final long MAX_VARCHAR_LENGTH = 60000; + public static final long POWER_2_16 = (long) Math.pow(2, 16); + public static final long BYTES_2GB = (long) Math.pow(2, 31); + public static final long MAX_BINARY_LENGTH = POWER_2_16 - 4; + public static final XuguTypeConverter INSTANCE = new XuguTypeConverter(); + + @Override + public String identifier() { + return DatabaseIdentifier.XUGU; + } + + @Override + public Column convert(BasicTypeDefine typeDefine) { + PhysicalColumn.PhysicalColumnBuilder builder = + PhysicalColumn.builder() + .name(typeDefine.getName()) + .sourceType(typeDefine.getColumnType()) + .nullable(typeDefine.isNullable()) + .defaultValue(typeDefine.getDefaultValue()) + .comment(typeDefine.getComment()); + + String xuguDataType = typeDefine.getDataType().toUpperCase(); + switch (xuguDataType) { + case XUGU_BOOLEAN: + case XUGU_BOOL: + builder.dataType(BasicType.BOOLEAN_TYPE); + break; + case XUGU_TINYINT: + builder.dataType(BasicType.BYTE_TYPE); + break; + case XUGU_SMALLINT: + builder.dataType(BasicType.SHORT_TYPE); + break; + case XUGU_INT: + case XUGU_INTEGER: + builder.dataType(BasicType.INT_TYPE); + break; + case XUGU_BIGINT: + builder.dataType(BasicType.LONG_TYPE); + break; + case XUGU_FLOAT: + builder.dataType(BasicType.FLOAT_TYPE); + break; + case XUGU_DOUBLE: + builder.dataType(BasicType.DOUBLE_TYPE); + break; + case XUGU_NUMBER: + case XUGU_DECIMAL: + case XUGU_NUMERIC: + DecimalType decimalType; + if (typeDefine.getPrecision() != null && typeDefine.getPrecision() > 0) { + decimalType = + new DecimalType( + typeDefine.getPrecision().intValue(), typeDefine.getScale()); + } else { + decimalType = new DecimalType(DEFAULT_PRECISION, DEFAULT_SCALE); + } + builder.dataType(decimalType); + builder.columnLength(Long.valueOf(decimalType.getPrecision())); + builder.scale(decimalType.getScale()); + break; + + case XUGU_CHAR: + case XUGU_NCHAR: + builder.dataType(BasicType.STRING_TYPE); + if (typeDefine.getLength() == null || typeDefine.getLength() <= 0) { + builder.columnLength(TypeDefineUtils.charTo4ByteLength(1L)); + } else { + builder.columnLength(typeDefine.getLength()); + } + break; + case XUGU_VARCHAR: + case XUGU_VARCHAR2: + builder.dataType(BasicType.STRING_TYPE); + if (typeDefine.getLength() == null || typeDefine.getLength() <= 0) { + builder.columnLength(TypeDefineUtils.charTo4ByteLength(MAX_VARCHAR_LENGTH)); + } else { + builder.columnLength(typeDefine.getLength()); + } + break; + case XUGU_CLOB: + builder.dataType(BasicType.STRING_TYPE); + builder.columnLength(BYTES_2GB - 1); + break; + case XUGU_JSON: + case XUGU_GUID: + builder.dataType(BasicType.STRING_TYPE); + break; + case XUGU_BINARY: + builder.dataType(PrimitiveByteArrayType.INSTANCE); + builder.columnLength(MAX_BINARY_LENGTH); + break; + case XUGU_BLOB: + builder.dataType(PrimitiveByteArrayType.INSTANCE); + builder.columnLength(BYTES_2GB - 1); + break; + case XUGU_DATE: + builder.dataType(LocalTimeType.LOCAL_DATE_TYPE); + break; + case XUGU_TIME: + case XUGU_TIME_WITH_TIME_ZONE: + builder.dataType(LocalTimeType.LOCAL_TIME_TYPE); + break; + case XUGU_DATETIME: + case XUGU_DATETIME_WITH_TIME_ZONE: + builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); + break; + case XUGU_TIMESTAMP: + case XUGU_TIMESTAMP_WITH_TIME_ZONE: + builder.dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE); + if (typeDefine.getScale() == null) { + builder.scale(TIMESTAMP_DEFAULT_SCALE); + } else { + builder.scale(typeDefine.getScale()); + } + break; + default: + throw CommonError.convertToSeaTunnelTypeError( + DatabaseIdentifier.XUGU, xuguDataType, typeDefine.getName()); + } + return builder.build(); + } + + @Override + public BasicTypeDefine reconvert(Column column) { + BasicTypeDefine.BasicTypeDefineBuilder builder = + BasicTypeDefine.builder() + .name(column.getName()) + .nullable(column.isNullable()) + .comment(column.getComment()) + .defaultValue(column.getDefaultValue()); + switch (column.getDataType().getSqlType()) { + case BOOLEAN: + builder.columnType(XUGU_BOOLEAN); + builder.dataType(XUGU_BOOLEAN); + break; + case TINYINT: + builder.columnType(XUGU_TINYINT); + builder.dataType(XUGU_TINYINT); + break; + case SMALLINT: + builder.columnType(XUGU_SMALLINT); + builder.dataType(XUGU_SMALLINT); + break; + case INT: + builder.columnType(XUGU_INTEGER); + builder.dataType(XUGU_INTEGER); + break; + case BIGINT: + builder.columnType(XUGU_BIGINT); + builder.dataType(XUGU_BIGINT); + break; + case FLOAT: + builder.columnType(XUGU_FLOAT); + builder.dataType(XUGU_FLOAT); + break; + case DOUBLE: + builder.columnType(XUGU_DOUBLE); + builder.dataType(XUGU_DOUBLE); + break; + case DECIMAL: + DecimalType decimalType = (DecimalType) column.getDataType(); + long precision = decimalType.getPrecision(); + int scale = decimalType.getScale(); + if (precision <= 0) { + precision = DEFAULT_PRECISION; + scale = DEFAULT_SCALE; + log.warn( + "The decimal column {} type decimal({},{}) is out of range, " + + "which is precision less than 0, " + + "it will be converted to decimal({},{})", + column.getName(), + decimalType.getPrecision(), + decimalType.getScale(), + precision, + scale); + } else if (precision > MAX_PRECISION) { + scale = (int) Math.max(0, scale - (precision - MAX_PRECISION)); + precision = MAX_PRECISION; + log.warn( + "The decimal column {} type decimal({},{}) is out of range, " + + "which exceeds the maximum precision of {}, " + + "it will be converted to decimal({},{})", + column.getName(), + decimalType.getPrecision(), + decimalType.getScale(), + MAX_PRECISION, + precision, + scale); + } + if (scale < 0) { + scale = 0; + log.warn( + "The decimal column {} type decimal({},{}) is out of range, " + + "which is scale less than 0, " + + "it will be converted to decimal({},{})", + column.getName(), + decimalType.getPrecision(), + decimalType.getScale(), + precision, + scale); + } else if (scale > MAX_SCALE) { + scale = MAX_SCALE; + log.warn( + "The decimal column {} type decimal({},{}) is out of range, " + + "which exceeds the maximum scale of {}, " + + "it will be converted to decimal({},{})", + column.getName(), + decimalType.getPrecision(), + decimalType.getScale(), + MAX_SCALE, + precision, + scale); + } + builder.columnType(String.format("%s(%s,%s)", XUGU_NUMERIC, precision, scale)); + builder.dataType(XUGU_NUMERIC); + builder.precision(precision); + builder.scale(scale); + break; + case BYTES: + if (column.getColumnLength() == null || column.getColumnLength() <= 0) { + builder.columnType(XUGU_BLOB); + builder.dataType(XUGU_BLOB); + } else if (column.getColumnLength() <= MAX_BINARY_LENGTH) { + builder.columnType(XUGU_BINARY); + builder.dataType(XUGU_BINARY); + } else { + builder.columnType(XUGU_BLOB); + builder.dataType(XUGU_BLOB); + } + break; + case STRING: + if (column.getColumnLength() == null || column.getColumnLength() <= 0) { + builder.columnType(String.format("%s(%s)", XUGU_VARCHAR, MAX_VARCHAR_LENGTH)); + builder.dataType(XUGU_VARCHAR); + } else if (column.getColumnLength() <= MAX_VARCHAR_LENGTH) { + builder.columnType( + String.format("%s(%s)", XUGU_VARCHAR, column.getColumnLength())); + builder.dataType(XUGU_VARCHAR); + } else { + builder.columnType(XUGU_CLOB); + builder.dataType(XUGU_CLOB); + } + break; + case DATE: + builder.columnType(XUGU_DATE); + builder.dataType(XUGU_DATE); + break; + case TIME: + builder.dataType(XUGU_TIME); + if (column.getScale() != null && column.getScale() > 0) { + Integer timeScale = column.getScale(); + if (timeScale > MAX_TIME_SCALE) { + timeScale = MAX_TIME_SCALE; + log.warn( + "The time column {} type time({}) is out of range, " + + "which exceeds the maximum scale of {}, " + + "it will be converted to time({})", + column.getName(), + column.getScale(), + MAX_SCALE, + timeScale); + } + builder.columnType(String.format("%s(%s)", XUGU_TIME, timeScale)); + builder.scale(timeScale); + } else { + builder.columnType(XUGU_TIME); + } + break; + case TIMESTAMP: + if (column.getScale() == null || column.getScale() <= 0) { + builder.columnType(XUGU_TIMESTAMP); + } else { + int timestampScale = column.getScale(); + if (column.getScale() > MAX_TIMESTAMP_SCALE) { + timestampScale = MAX_TIMESTAMP_SCALE; + log.warn( + "The timestamp column {} type timestamp({}) is out of range, " + + "which exceeds the maximum scale of {}, " + + "it will be converted to timestamp({})", + column.getName(), + column.getScale(), + MAX_TIMESTAMP_SCALE, + timestampScale); + } + builder.columnType(String.format("TIMESTAMP(%s)", timestampScale)); + builder.scale(timestampScale); + } + builder.dataType(XUGU_TIMESTAMP); + break; + default: + throw CommonError.convertToConnectorTypeError( + DatabaseIdentifier.XUGU, + column.getDataType().getSqlType().name(), + column.getName()); + } + + return builder.build(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeMapper.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeMapper.java new file mode 100644 index 00000000000..e517f56af0c --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeMapper.java @@ -0,0 +1,63 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu; + +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.connectors.seatunnel.common.source.TypeDefineUtils; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; + +import lombok.extern.slf4j.Slf4j; + +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.Arrays; + +@Slf4j +public class XuguTypeMapper implements JdbcDialectTypeMapper { + + @Override + public Column mappingColumn(BasicTypeDefine typeDefine) { + return XuguTypeConverter.INSTANCE.convert(typeDefine); + } + + @Override + public Column mappingColumn(ResultSetMetaData metadata, int colIndex) throws SQLException { + String columnName = metadata.getColumnLabel(colIndex); + String nativeType = metadata.getColumnTypeName(colIndex); + int isNullable = metadata.isNullable(colIndex); + long precision = metadata.getPrecision(colIndex); + int scale = metadata.getScale(colIndex); + if (Arrays.asList("CHAR", "NCHAR", "VARCHAR", "VARCHAR2").contains(nativeType)) { + long octetByteLength = TypeDefineUtils.charTo4ByteLength(precision); + precision = octetByteLength; + } + + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name(columnName) + .columnType(nativeType) + .dataType(nativeType) + .nullable(isNullable == ResultSetMetaData.columnNullable) + .length(precision) + .precision(precision) + .scale(scale) + .build(); + return mappingColumn(typeDefine); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverterTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverterTest.java new file mode 100644 index 00000000000..9dfd7079dfd --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/xugu/XuguTypeConverterTest.java @@ -0,0 +1,660 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu; + +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import static org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu.XuguTypeConverter.BYTES_2GB; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.xugu.XuguTypeConverter.MAX_BINARY_LENGTH; + +public class XuguTypeConverterTest { + @Test + public void testConvertUnsupported() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder().name("test").columnType("aaa").dataType("aaa").build(); + try { + XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.fail(); + } catch (SeaTunnelRuntimeException e) { + // ignore + } catch (Throwable e) { + Assertions.fail(); + } + } + + @Test + public void testReconvertUnsupported() { + Column column = + PhysicalColumn.of( + "test", + new MapType<>(BasicType.STRING_TYPE, BasicType.STRING_TYPE), + (Long) null, + true, + null, + null); + try { + XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.fail(); + } catch (SeaTunnelRuntimeException e) { + // ignore + } catch (Throwable e) { + Assertions.fail(); + } + } + + @Test + public void testConvertBoolean() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("bool") + .dataType("boolean") + .nullable(true) + .defaultValue("1") + .comment("test") + .build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.BOOLEAN_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + Assertions.assertEquals(typeDefine.isNullable(), column.isNullable()); + Assertions.assertEquals(typeDefine.getDefaultValue(), column.getDefaultValue()); + Assertions.assertEquals(typeDefine.getComment(), column.getComment()); + } + + @Test + public void testConvertTinyint() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("tinyint") + .dataType("tinyint") + .build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.BYTE_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testConvertSmallint() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("smallint") + .dataType("smallint") + .build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.SHORT_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testConvertInt() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder().name("test").columnType("int").dataType("int").build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.INT_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testConvertBigint() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("bigint") + .dataType("bigint") + .build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.LONG_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testConvertFloat() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("float") + .dataType("float") + .build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.FLOAT_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testConvertDouble() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("double") + .dataType("double") + .build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.DOUBLE_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testConvertDecimal() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("numeric(38,2)") + .dataType("numeric") + .precision(38L) + .scale(2) + .build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(new DecimalType(38, 2), column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("numeric") + .dataType("numeric") + .build(); + column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(new DecimalType(38, 18), column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testConvertChar() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder().name("test").columnType("char").dataType("char").build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); + Assertions.assertEquals(4, column.getColumnLength()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("char(10)") + .dataType("char") + .length(10L) + .build(); + column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); + Assertions.assertEquals(10, column.getColumnLength()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testConvertVarchar() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("varchar") + .dataType("varchar") + .build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); + Assertions.assertEquals(240000, column.getColumnLength()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("varchar(10)") + .dataType("varchar") + .length(10L) + .build(); + column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); + Assertions.assertEquals(10, column.getColumnLength()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("varchar2(20)") + .dataType("varchar2") + .length(20L) + .build(); + column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); + Assertions.assertEquals(20, column.getColumnLength()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testConvertOtherString() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder().name("test").columnType("clob").dataType("clob").build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); + Assertions.assertEquals(BYTES_2GB - 1, column.getColumnLength()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder().name("test").columnType("json").dataType("json").build(); + column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); + Assertions.assertEquals(null, column.getColumnLength()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testConvertBinary() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder().name("test").columnType("blob").dataType("blob").build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(PrimitiveByteArrayType.INSTANCE, column.getDataType()); + Assertions.assertEquals(BYTES_2GB - 1, column.getColumnLength()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testConvertDate() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder().name("test").columnType("date").dataType("date").build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testConvertTime() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder().name("test").columnType("time").dataType("time").build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(LocalTimeType.LOCAL_TIME_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("time with time zone") + .dataType("time with time zone") + .build(); + column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(LocalTimeType.LOCAL_TIME_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testConvertTimestamp() { + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("datetime") + .dataType("datetime") + .build(); + Column column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("datetime with time zone") + .dataType("datetime with time zone") + .build(); + column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("timestamp") + .dataType("timestamp") + .build(); + column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + Assertions.assertEquals(3, column.getScale()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("timestamp(6)") + .dataType("timestamp") + .scale(6) + .build(); + column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getScale(), column.getScale()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("timestamp with time zone") + .dataType("timestamp with time zone") + .build(); + column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + Assertions.assertEquals(3, column.getScale()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("timestamp(3) with time zone") + .dataType("timestamp with time zone") + .scale(3) + .build(); + column = XuguTypeConverter.INSTANCE.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); + Assertions.assertEquals(typeDefine.getScale(), column.getScale()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + + @Test + public void testReconvertBoolean() { + Column column = + PhysicalColumn.builder() + .name("test") + .dataType(BasicType.BOOLEAN_TYPE) + .nullable(true) + .defaultValue(true) + .comment("test") + .build(); + + BasicTypeDefine typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(XuguTypeConverter.XUGU_BOOLEAN, typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_BOOLEAN, typeDefine.getDataType()); + Assertions.assertEquals(column.isNullable(), typeDefine.isNullable()); + Assertions.assertEquals(column.getDefaultValue(), typeDefine.getDefaultValue()); + Assertions.assertEquals(column.getComment(), typeDefine.getComment()); + } + + @Test + public void testReconvertByte() { + Column column = PhysicalColumn.builder().name("test").dataType(BasicType.BYTE_TYPE).build(); + + BasicTypeDefine typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(XuguTypeConverter.XUGU_TINYINT, typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_TINYINT, typeDefine.getDataType()); + } + + @Test + public void testReconvertShort() { + Column column = + PhysicalColumn.builder().name("test").dataType(BasicType.SHORT_TYPE).build(); + + BasicTypeDefine typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(XuguTypeConverter.XUGU_SMALLINT, typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_SMALLINT, typeDefine.getDataType()); + } + + @Test + public void testReconvertInt() { + Column column = PhysicalColumn.builder().name("test").dataType(BasicType.INT_TYPE).build(); + + BasicTypeDefine typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(XuguTypeConverter.XUGU_INTEGER, typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_INTEGER, typeDefine.getDataType()); + } + + @Test + public void testReconvertLong() { + Column column = PhysicalColumn.builder().name("test").dataType(BasicType.LONG_TYPE).build(); + + BasicTypeDefine typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(XuguTypeConverter.XUGU_BIGINT, typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_BIGINT, typeDefine.getDataType()); + } + + @Test + public void testReconvertFloat() { + Column column = + PhysicalColumn.builder().name("test").dataType(BasicType.FLOAT_TYPE).build(); + + BasicTypeDefine typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(XuguTypeConverter.XUGU_FLOAT, typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_FLOAT, typeDefine.getDataType()); + } + + @Test + public void testReconvertDouble() { + Column column = + PhysicalColumn.builder().name("test").dataType(BasicType.DOUBLE_TYPE).build(); + + BasicTypeDefine typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(XuguTypeConverter.XUGU_DOUBLE, typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_DOUBLE, typeDefine.getDataType()); + } + + @Test + public void testReconvertDecimal() { + Column column = + PhysicalColumn.builder().name("test").dataType(new DecimalType(0, 0)).build(); + + BasicTypeDefine typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals( + String.format( + "%s(%s,%s)", + XuguTypeConverter.XUGU_NUMERIC, + XuguTypeConverter.DEFAULT_PRECISION, + XuguTypeConverter.DEFAULT_SCALE), + typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_NUMERIC, typeDefine.getDataType()); + + column = PhysicalColumn.builder().name("test").dataType(new DecimalType(10, 2)).build(); + + typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals( + String.format("%s(%s,%s)", XuguTypeConverter.XUGU_NUMERIC, 10, 2), + typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_NUMERIC, typeDefine.getDataType()); + } + + @Test + public void testReconvertBytes() { + Column column = + PhysicalColumn.builder() + .name("test") + .dataType(PrimitiveByteArrayType.INSTANCE) + .columnLength(null) + .build(); + + BasicTypeDefine typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(XuguTypeConverter.XUGU_BLOB, typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_BLOB, typeDefine.getDataType()); + + column = + PhysicalColumn.builder() + .name("test") + .dataType(PrimitiveByteArrayType.INSTANCE) + .columnLength(MAX_BINARY_LENGTH) + .build(); + + typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(XuguTypeConverter.XUGU_BINARY, typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_BINARY, typeDefine.getDataType()); + } + + @Test + public void testReconvertString() { + Column column = + PhysicalColumn.builder() + .name("test") + .dataType(BasicType.STRING_TYPE) + .columnLength(null) + .build(); + + BasicTypeDefine typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals("VARCHAR(60000)", typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_VARCHAR, typeDefine.getDataType()); + + column = + PhysicalColumn.builder() + .name("test") + .dataType(BasicType.STRING_TYPE) + .columnLength(1L) + .build(); + + typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals( + String.format("%s(%s)", XuguTypeConverter.XUGU_VARCHAR, column.getColumnLength()), + typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_VARCHAR, typeDefine.getDataType()); + + column = + PhysicalColumn.builder() + .name("test") + .dataType(BasicType.STRING_TYPE) + .columnLength(60000L) + .build(); + + typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals( + String.format("%s(%s)", XuguTypeConverter.XUGU_VARCHAR, column.getColumnLength()), + typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_VARCHAR, typeDefine.getDataType()); + + column = + PhysicalColumn.builder() + .name("test") + .dataType(BasicType.STRING_TYPE) + .columnLength(60001L) + .build(); + + typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(XuguTypeConverter.XUGU_CLOB, typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_CLOB, typeDefine.getDataType()); + } + + @Test + public void testReconvertDate() { + Column column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.LOCAL_DATE_TYPE) + .build(); + + BasicTypeDefine typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(XuguTypeConverter.XUGU_DATE, typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_DATE, typeDefine.getDataType()); + } + + @Test + public void testReconvertTime() { + Column column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.LOCAL_TIME_TYPE) + .build(); + + BasicTypeDefine typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(XuguTypeConverter.XUGU_TIME, typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_TIME, typeDefine.getDataType()); + } + + @Test + public void testReconvertDatetime() { + Column column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE) + .build(); + + BasicTypeDefine typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals(XuguTypeConverter.XUGU_TIMESTAMP, typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_TIMESTAMP, typeDefine.getDataType()); + + column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE) + .scale(3) + .build(); + + typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals( + String.format("%s(%s)", XuguTypeConverter.XUGU_TIMESTAMP, column.getScale()), + typeDefine.getColumnType()); + Assertions.assertEquals(XuguTypeConverter.XUGU_TIMESTAMP, typeDefine.getDataType()); + Assertions.assertEquals(column.getScale(), typeDefine.getScale()); + + column = + PhysicalColumn.builder() + .name("test") + .dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE) + .scale(6) + .build(); + + typeDefine = XuguTypeConverter.INSTANCE.reconvert(column); + Assertions.assertEquals(column.getName(), typeDefine.getName()); + Assertions.assertEquals( + String.format("%s(%s)", XuguTypeConverter.XUGU_TIMESTAMP, 6), + typeDefine.getColumnType()); + } +} diff --git a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml index 5c1171a82da..1da0d3014b6 100644 --- a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml +++ b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml @@ -203,6 +203,7 @@ com.teradata.jdbc:terajdbc4:jar com.amazon.redshift:redshift-jdbc42:jar net.snowflake.snowflake-jdbc:jar + com.xugudb:xugu-jdbc:jar ${artifact.file.name} /lib diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/pom.xml index 09d511594f4..af4c61d5b65 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/pom.xml @@ -91,6 +91,11 @@ vertica-jdbc test + + com.xugudb + xugu-jdbc + test + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcXuguIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcXuguIT.java new file mode 100644 index 00000000000..5fdae0ad939 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcXuguIT.java @@ -0,0 +1,246 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.xugu.XuguCatalog; + +import org.apache.commons.lang3.tuple.Pair; + +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.utility.DockerLoggerFactory; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +@Slf4j +public class JdbcXuguIT extends AbstractJdbcIT { + + private static final String XUGU_IMAGE = "xugudb/xugudb:v12"; + private static final String XUGU_CONTAINER_HOST = "e2e_xugudb"; + private static final String XUGU_SCHEMA = "SYSDBA"; + private static final String XUGU_DATABASE = "SYSTEM"; + private static final String XUGU_SOURCE = "e2e_table_source"; + private static final String XUGU_SINK = "e2e_table_sink"; + private static final String CATALOG_DATABASE = "catalog_database"; + private static final String CATALOG_TABLE = "e2e_table_catalog"; + private static final String XUGU_USERNAME = "SYSDBA"; + private static final String XUGU_PASSWORD = "SYSDBA"; + private static final int XUGU_PORT = 5138; + private static final String XUGU_URL = "jdbc:xugu://" + HOST + ":%s/%s"; + + private static final String DRIVER_CLASS = "com.xugu.cloudjdbc.Driver"; + + private static final List CONFIG_FILE = + Lists.newArrayList( + "/jdbc_xugu_source_and_upsert_sink.conf", "/jdbc_xugu_source_and_sink.conf"); + private static final String CREATE_SQL = + "create table if not exists %s" + + "(\n" + + " XUGU_NUMERIC NUMERIC(10,2),\n" + + " XUGU_NUMBER NUMBER(10,2),\n" + + " XUGU_INTEGER INTEGER,\n" + + " XUGU_INT INT,\n" + + " XUGU_BIGINT BIGINT,\n" + + " XUGU_TINYINT TINYINT,\n" + + " XUGU_SMALLINT SMALLINT,\n" + + " XUGU_FLOAT FLOAT,\n" + + " XUGU_DOUBLE DOUBLE,\n" + + " XUGU_CHAR CHAR,\n" + + " XUGU_NCHAR NCHAR,\n" + + " XUGU_VARCHAR VARCHAR,\n" + + " XUGU_VARCHAR2 VARCHAR2,\n" + + " XUGU_CLOB CLOB,\n" + + " XUGU_DATE DATE,\n" + + " XUGU_TIME TIME,\n" + + " XUGU_TIMESTAMP TIMESTAMP,\n" + + " XUGU_DATETIME DATETIME,\n" + + " XUGU_TIME_WITH_TIME_ZONE TIME WITH TIME ZONE,\n" + + " XUGU_TIMESTAMP_WITH_TIME_ZONE TIMESTAMP WITH TIME ZONE,\n" + + " XUGU_BINARY BINARY,\n" + + " XUGU_BLOB BLOB,\n" + + " XUGU_GUID GUID,\n" + + " XUGU_BOOLEAN BOOLEAN,\n" + + " CONSTRAINT \"XUGU_PK\" PRIMARY KEY(XUGU_INT)" + + ")"; + private static final String[] fieldNames = + new String[] { + "XUGU_NUMERIC", + "XUGU_NUMBER", + "XUGU_INTEGER", + "XUGU_INT", + "XUGU_BIGINT", + "XUGU_TINYINT", + "XUGU_SMALLINT", + "XUGU_FLOAT", + "XUGU_DOUBLE", + "XUGU_CHAR", + "XUGU_NCHAR", + "XUGU_VARCHAR", + "XUGU_VARCHAR2", + "XUGU_CLOB", + "XUGU_DATE", + "XUGU_TIME", + "XUGU_TIMESTAMP", + "XUGU_DATETIME", + "XUGU_TIME_WITH_TIME_ZONE", + "XUGU_TIMESTAMP_WITH_TIME_ZONE", + "XUGU_BINARY", + "XUGU_BLOB", + "XUGU_GUID", + "XUGU_BOOLEAN" + }; + + @Override + JdbcCase getJdbcCase() { + Map containerEnv = new HashMap<>(); + String jdbcUrl = String.format(XUGU_URL, XUGU_PORT, XUGU_DATABASE); + Pair> testDataSet = initTestData(); + String[] fieldNames = testDataSet.getKey(); + + String insertSql = insertTable(XUGU_SCHEMA, XUGU_SOURCE, fieldNames); + + return JdbcCase.builder() + .dockerImage(XUGU_IMAGE) + .networkAliases(XUGU_CONTAINER_HOST) + .containerEnv(containerEnv) + .driverClass(DRIVER_CLASS) + .host(HOST) + .port(XUGU_PORT) + .localPort(XUGU_PORT) + .jdbcTemplate(XUGU_URL) + .jdbcUrl(jdbcUrl) + .userName(XUGU_USERNAME) + .password(XUGU_PASSWORD) + .schema(XUGU_SCHEMA) + .database(XUGU_DATABASE) + .sourceTable(XUGU_SOURCE) + .sinkTable(XUGU_SINK) + .catalogDatabase(CATALOG_DATABASE) + .catalogSchema(XUGU_SCHEMA) + .catalogTable(CATALOG_TABLE) + .createSql(CREATE_SQL) + .configFile(CONFIG_FILE) + .insertSql(insertSql) + .testData(testDataSet) + .build(); + } + + @Override + void compareResult(String executeKey) { + defaultCompare(executeKey, fieldNames, "XUGU_INT"); + } + + @Override + String driverUrl() { + return "https://repo1.maven.org/maven2/com/xugudb/xugu-jdbc/12.2.0/xugu-jdbc-12.2.0.jar"; + } + + @Override + Pair> initTestData() { + List rows = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + SeaTunnelRow row = + new SeaTunnelRow( + new Object[] { + BigDecimal.valueOf(1.12), + BigDecimal.valueOf(i, 2), + i, + i, + Long.parseLong("1"), + i, + i, + Float.parseFloat("1.1"), + Double.parseDouble("1.1"), + String.format("f1_%s", i), + String.format("f1_%s", i), + String.format("f1_%s", i), + String.format("f1_%s", i), + String.format("f1_%s", i), + Date.valueOf(LocalDate.now()), + Time.valueOf(LocalTime.now()), + new Timestamp(System.currentTimeMillis()), + Timestamp.valueOf(LocalDateTime.now()), + Time.valueOf(LocalTime.now()), + new Timestamp(System.currentTimeMillis()), + null, + null, + null, + false + }); + rows.add(row); + } + + return Pair.of(fieldNames, rows); + } + + @Override + protected GenericContainer initContainer() { + GenericContainer container = + new GenericContainer<>(XUGU_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(XUGU_CONTAINER_HOST) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(XUGU_IMAGE))); + container.setPortBindings(Lists.newArrayList(String.format("%s:%s", XUGU_PORT, XUGU_PORT))); + + return container; + } + + @Override + public String quoteIdentifier(String field) { + return "\"" + field + "\""; + } + + @Override + protected void clearTable(String database, String schema, String table) { + clearTable(schema, table); + } + + @Override + protected String buildTableInfoWithSchema(String database, String schema, String table) { + return buildTableInfoWithSchema(schema, table); + } + + @Override + protected void initCatalog() { + String jdbcUrl = jdbcCase.getJdbcUrl().replace(HOST, dbServer.getHost()); + catalog = + new XuguCatalog( + "xugu", + jdbcCase.getUserName(), + jdbcCase.getPassword(), + JdbcUrlUtil.getUrlInfo(jdbcUrl), + XUGU_SCHEMA); + catalog.open(); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_xugu_source_and_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_xugu_source_and_sink.conf new file mode 100644 index 00000000000..09154809f36 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_xugu_source_and_sink.conf @@ -0,0 +1,47 @@ +# +# 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. +# + +env { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + jdbc { + url = "jdbc:xugu://e2e_xugudb:5138/SYSTEM?batch_mode=false" + driver = "com.xugu.cloudjdbc.Driver" + connection_check_timeout_sec = 100 + user = "SYSDBA" + password = "SYSDBA" + query = "select * from e2e_table_source;" + } +} + +transform { +} + +sink { + jdbc { + url = "jdbc:xugu://e2e_xugudb:5138/SYSTEM?batch_mode=false" + driver = "com.xugu.cloudjdbc.Driver" + user = "SYSDBA" + password = "SYSDBA" + query = """INSERT INTO SYSDBA.e2e_table_sink + (XUGU_NUMERIC, XUGU_NUMBER, XUGU_INTEGER, XUGU_INT, XUGU_BIGINT, XUGU_TINYINT, XUGU_SMALLINT, XUGU_FLOAT, XUGU_DOUBLE, XUGU_CHAR, XUGU_NCHAR, XUGU_VARCHAR, XUGU_VARCHAR2, XUGU_CLOB, XUGU_DATE, XUGU_TIME, XUGU_TIMESTAMP, XUGU_DATETIME, XUGU_TIME_WITH_TIME_ZONE, XUGU_TIMESTAMP_WITH_TIME_ZONE, XUGU_BINARY, XUGU_BLOB, XUGU_GUID, XUGU_BOOLEAN) + VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?);""" + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_xugu_source_and_upsert_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_xugu_source_and_upsert_sink.conf new file mode 100644 index 00000000000..669118f1660 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/resources/jdbc_xugu_source_and_upsert_sink.conf @@ -0,0 +1,48 @@ +# +# 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. +# + +env { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + jdbc { + url = "jdbc:xugu://e2e_xugudb:5138/SYSTEM" + driver = "com.xugu.cloudjdbc.Driver" + connection_check_timeout_sec = 100 + user = "SYSDBA" + password = "SYSDBA" + query = "select * from e2e_table_source;" + } +} + +transform { +} + +sink { + jdbc { + url = "jdbc:xugu://e2e_xugudb:5138/SYSTEM?batch_mode=false" + driver = "com.xugu.cloudjdbc.Driver" + user = "SYSDBA" + password = "SYSDBA" + generate_sink_sql = true + primary_keys = ["XUGU_INT"] + table = "SYSDBA.e2e_table_sink" + database = "SYSTEM" + } +}