From e572a9baa78e05d44551e757e12237dc9bbdc29f Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Sun, 12 May 2024 18:13:14 +0800 Subject: [PATCH 01/14] [Improve][Doc] Add sql-config doc into sidebars.js (#6836) --- docs/sidebars.js | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/sidebars.js b/docs/sidebars.js index 10f04e0b503..ad1093689f9 100644 --- a/docs/sidebars.js +++ b/docs/sidebars.js @@ -90,6 +90,7 @@ const sidebars = { "concept/connector-v2-features", 'concept/schema-feature', 'concept/JobEnvConfig', + 'concept/sql-config', 'concept/speed-limit', 'concept/event-listener' ] From b6d075194bfe79154afb19128964201addd6e2a0 Mon Sep 17 00:00:00 2001 From: Carl-Zhou-CN <1058249259@qq.com> Date: Mon, 13 May 2024 21:06:10 +0800 Subject: [PATCH 02/14] [bugFix][Connector-V2][Doris] The multi-FE configuration is supported (#6341) --- .../connectors/doris/rest/RestService.java | 3 +- .../doris/sink/writer/DorisSinkWriter.java | 3 +- .../doris/rest/models/RestServiceTest.java | 56 +++++++++++++++++++ 3 files changed, 60 insertions(+), 2 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-doris/src/test/java/org/apache/seatunnel/connectors/doris/rest/models/RestServiceTest.java diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/rest/RestService.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/rest/RestService.java index 315f36cfa2d..b516157443a 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/rest/RestService.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/rest/RestService.java @@ -237,7 +237,8 @@ static String[] parseIdentifier(String tableIdentifier, Logger logger) } @VisibleForTesting - static String randomEndpoint(String feNodes, Logger logger) throws DorisConnectorException { + public static String randomEndpoint(String feNodes, Logger logger) + throws DorisConnectorException { logger.trace("Parse fenodes '{}'.", feNodes); if (StringUtils.isEmpty(feNodes)) { String errMsg = diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisSinkWriter.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisSinkWriter.java index 496b91b25be..b5aa5274216 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisSinkWriter.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/writer/DorisSinkWriter.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.connectors.doris.config.DorisConfig; import org.apache.seatunnel.connectors.doris.exception.DorisConnectorErrorCode; import org.apache.seatunnel.connectors.doris.exception.DorisConnectorException; +import org.apache.seatunnel.connectors.doris.rest.RestService; import org.apache.seatunnel.connectors.doris.rest.models.RespContent; import org.apache.seatunnel.connectors.doris.serialize.DorisSerializer; import org.apache.seatunnel.connectors.doris.serialize.SeaTunnelRowSerializer; @@ -96,7 +97,7 @@ public DorisSinkWriter( } private void initializeLoad() { - String backend = dorisConfig.getFrontends(); + String backend = RestService.randomEndpoint(dorisConfig.getFrontends(), log); try { this.dorisStreamLoad = new DorisStreamLoad( diff --git a/seatunnel-connectors-v2/connector-doris/src/test/java/org/apache/seatunnel/connectors/doris/rest/models/RestServiceTest.java b/seatunnel-connectors-v2/connector-doris/src/test/java/org/apache/seatunnel/connectors/doris/rest/models/RestServiceTest.java new file mode 100644 index 00000000000..aa917d57662 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/test/java/org/apache/seatunnel/connectors/doris/rest/models/RestServiceTest.java @@ -0,0 +1,56 @@ +/* + * 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.doris.rest.models; + +import org.apache.seatunnel.connectors.doris.rest.RestService; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import lombok.extern.slf4j.Slf4j; + +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +@Slf4j +public class RestServiceTest { + + @Test + void testRandomEndpoint() { + + List list = + Arrays.asList( + "fe_host1:fe_http_port1", + "fe_host2:fe_http_port2", + "fe_host3:fe_http_port3", + "fe_host4:fe_http_port4", + "fe_host5:fe_http_port5"); + + boolean hasDifferentAddress = false; + for (int i = 0; i < 5; i++) { + Set addresses = + list.stream() + .map(address -> RestService.randomEndpoint(String.join(",", list), log)) + .collect(Collectors.toSet()); + hasDifferentAddress = addresses.size() > 1; + } + Assertions.assertTrue(hasDifferentAddress); + } +} From ee3b7c37239f6f696fe6bb8f43bd7078465ff837 Mon Sep 17 00:00:00 2001 From: Eric Date: Tue, 14 May 2024 10:19:11 +0800 Subject: [PATCH 03/14] [Improve][JDBC Source] Fix Split can not be cancel (#6825) --- .../splitter/AbstractJdbcSourceChunkSplitter.java | 2 +- .../enumerator/splitter/JdbcSourceChunkSplitter.java | 4 ++-- .../java/jdbc/source/JdbcSourceChunkSplitterTest.java | 2 +- .../splitter/AbstractJdbcSourceChunkSplitterTest.java | 2 +- .../cdc/mysql/source/eumerator/MySqlChunkSplitter.java | 2 +- .../connectors/seatunnel/cdc/mysql/utils/MySqlUtils.java | 5 ++++- .../cdc/oracle/source/eumerator/OracleChunkSplitter.java | 2 +- .../seatunnel/cdc/oracle/utils/OracleUtils.java | 5 ++++- .../postgres/source/enumerator/PostgresChunkSplitter.java | 4 ++-- .../seatunnel/cdc/postgres/utils/PostgresUtils.java | 5 ++++- .../source/source/eumerator/SqlServerChunkSplitter.java | 2 +- .../cdc/sqlserver/source/utils/SqlServerUtils.java | 5 ++++- .../seatunnel/jdbc/internal/dialect/JdbcDialect.java | 5 ++++- .../jdbc/internal/dialect/mysql/MysqlDialect.java | 5 ++++- .../connectors/seatunnel/jdbc/source/ChunkSplitter.java | 4 ++-- .../seatunnel/jdbc/source/DynamicChunkSplitter.java | 8 ++++---- .../seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java | 3 +-- .../seatunnel/connectors/seatunnel/jdbc/JdbcIrisIT.java | 2 +- .../connectors/seatunnel/jdbc/JdbcMysqlSplitIT.java | 2 +- 19 files changed, 43 insertions(+), 26 deletions(-) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/AbstractJdbcSourceChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/AbstractJdbcSourceChunkSplitter.java index e10c70795b1..60a208de866 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/AbstractJdbcSourceChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/AbstractJdbcSourceChunkSplitter.java @@ -114,7 +114,7 @@ public Collection generateSplits(TableId tableId) { } private List splitTableIntoChunks( - JdbcConnection jdbc, TableId tableId, Column splitColumn) throws SQLException { + JdbcConnection jdbc, TableId tableId, Column splitColumn) throws Exception { final String splitColumnName = splitColumn.name(); final Object[] minMax = queryMinMax(jdbc, tableId, splitColumn); final Object min = minMax[0]; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/JdbcSourceChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/JdbcSourceChunkSplitter.java index 3981ddfa7c5..d64469d3b3e 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/JdbcSourceChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/JdbcSourceChunkSplitter.java @@ -81,7 +81,7 @@ default Object queryMin( @Deprecated Object[] sampleDataFromColumn( JdbcConnection jdbc, TableId tableId, String columnName, int samplingRate) - throws SQLException; + throws Exception; /** * Performs a sampling operation on the specified column of a table in a JDBC-connected @@ -97,7 +97,7 @@ Object[] sampleDataFromColumn( */ default Object[] sampleDataFromColumn( JdbcConnection jdbc, TableId tableId, Column column, int samplingRate) - throws SQLException { + throws Exception { return sampleDataFromColumn(jdbc, tableId, column.name(), samplingRate); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/jdbc/source/JdbcSourceChunkSplitterTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/jdbc/source/JdbcSourceChunkSplitterTest.java index 32617fe18c0..d7bf573cc14 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/jdbc/source/JdbcSourceChunkSplitterTest.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/jdbc/source/JdbcSourceChunkSplitterTest.java @@ -83,7 +83,7 @@ public Object queryMin( @Override public Object[] sampleDataFromColumn( JdbcConnection jdbc, TableId tableId, String columnName, int samplingRate) - throws SQLException { + throws Exception { return new Object[0]; } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/AbstractJdbcSourceChunkSplitterTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/AbstractJdbcSourceChunkSplitterTest.java index 6f646eb6bed..f89e720c438 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/AbstractJdbcSourceChunkSplitterTest.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/test/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/AbstractJdbcSourceChunkSplitterTest.java @@ -195,7 +195,7 @@ public Object queryMin( @Override public Object[] sampleDataFromColumn( JdbcConnection jdbc, TableId tableId, String columnName, int samplingRate) - throws SQLException { + throws Exception { return new Object[0]; } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/eumerator/MySqlChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/eumerator/MySqlChunkSplitter.java index b4982f2cbeb..732b21e395d 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/eumerator/MySqlChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/eumerator/MySqlChunkSplitter.java @@ -57,7 +57,7 @@ public Object queryMin( @Override public Object[] sampleDataFromColumn( JdbcConnection jdbc, TableId tableId, String columnName, int inverseSamplingRate) - throws SQLException { + throws Exception { return MySqlUtils.skipReadAndSortSampleData(jdbc, tableId, columnName, inverseSamplingRate); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlUtils.java index 9b06ddda967..777be9d1d65 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlUtils.java @@ -146,7 +146,7 @@ public static Object[] sampleDataFromColumn( public static Object[] skipReadAndSortSampleData( JdbcConnection jdbc, TableId tableId, String columnName, int inverseSamplingRate) - throws SQLException { + throws Exception { final String sampleQuery = String.format("SELECT %s FROM %s", quote(columnName), quote(tableId)); @@ -172,6 +172,9 @@ public static Object[] skipReadAndSortSampleData( if (count % inverseSamplingRate == 0) { results.add(rs.getObject(1)); } + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Thread interrupted"); + } } } finally { if (rs != null) { diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/eumerator/OracleChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/eumerator/OracleChunkSplitter.java index 52df70cbc89..6525c3a2dbe 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/eumerator/OracleChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/eumerator/OracleChunkSplitter.java @@ -61,7 +61,7 @@ public Object queryMin( @Override public Object[] sampleDataFromColumn( JdbcConnection jdbc, TableId tableId, String columnName, int inverseSamplingRate) - throws SQLException { + throws Exception { return OracleUtils.skipReadAndSortSampleData( jdbc, tableId, columnName, inverseSamplingRate); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleUtils.java index 8d67c0f1412..cad2a3c836d 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/utils/OracleUtils.java @@ -150,7 +150,7 @@ public static Object[] sampleDataFromColumn( public static Object[] skipReadAndSortSampleData( JdbcConnection jdbc, TableId tableId, String columnName, int inverseSamplingRate) - throws SQLException { + throws Exception { final String sampleQuery = String.format("SELECT %s FROM %s", quote(columnName), quoteSchemaAndTable(tableId)); @@ -176,6 +176,9 @@ public static Object[] skipReadAndSortSampleData( if (count % inverseSamplingRate == 0) { results.add(rs.getObject(1)); } + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Thread interrupted"); + } } } finally { if (rs != null) { diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/enumerator/PostgresChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/enumerator/PostgresChunkSplitter.java index 2aab573d2e8..fb1aec572da 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/enumerator/PostgresChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/enumerator/PostgresChunkSplitter.java @@ -70,7 +70,7 @@ public Object queryMin( @Override public Object[] sampleDataFromColumn( JdbcConnection jdbc, TableId tableId, String columnName, int inverseSamplingRate) - throws SQLException { + throws Exception { return PostgresUtils.skipReadAndSortSampleData( jdbc, tableId, columnName, null, inverseSamplingRate); } @@ -78,7 +78,7 @@ public Object[] sampleDataFromColumn( @Override public Object[] sampleDataFromColumn( JdbcConnection jdbc, TableId tableId, Column column, int inverseSamplingRate) - throws SQLException { + throws Exception { return PostgresUtils.skipReadAndSortSampleData( jdbc, tableId, column.name(), column, inverseSamplingRate); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresUtils.java index b5cd0904532..09ea768aa27 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresUtils.java @@ -161,7 +161,7 @@ public static Object[] skipReadAndSortSampleData( String columnName, Column column, int inverseSamplingRate) - throws SQLException { + throws Exception { columnName = quote(columnName); if (column != null) { columnName = JDBC_DIALECT.convertType(columnName, column.typeName()); @@ -187,6 +187,9 @@ public static Object[] skipReadAndSortSampleData( if (count % 100000 == 0) { log.info("Processing row index: {}", count); } + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Thread interrupted"); + } if (count % inverseSamplingRate == 0) { results.add(rs.getObject(1)); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/eumerator/SqlServerChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/eumerator/SqlServerChunkSplitter.java index b6698f53190..b59bb7789dd 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/eumerator/SqlServerChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/eumerator/SqlServerChunkSplitter.java @@ -57,7 +57,7 @@ public Object queryMin( @Override public Object[] sampleDataFromColumn( JdbcConnection jdbc, TableId tableId, String columnName, int inverseSamplingRate) - throws SQLException { + throws Exception { return SqlServerUtils.skipReadAndSortSampleData( jdbc, tableId, columnName, inverseSamplingRate); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/utils/SqlServerUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/utils/SqlServerUtils.java index db1872fa648..11c2822d154 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/utils/SqlServerUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/utils/SqlServerUtils.java @@ -151,7 +151,7 @@ public static Object[] sampleDataFromColumn( public static Object[] skipReadAndSortSampleData( JdbcConnection jdbc, TableId tableId, String columnName, int inverseSamplingRate) - throws SQLException { + throws Exception { final String sampleQuery = String.format("SELECT %s FROM %s", quote(columnName), quote(tableId)); @@ -177,6 +177,9 @@ public static Object[] skipReadAndSortSampleData( if (count % inverseSamplingRate == 0) { results.add(rs.getObject(1)); } + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Thread interrupted"); + } } } finally { if (rs != null) { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java index db9b90daded..da92f821092 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java @@ -312,7 +312,7 @@ default Object[] sampleDataFromColumn( String columnName, int samplingRate, int fetchSize) - throws SQLException { + throws Exception { String sampleQuery; if (StringUtils.isNotBlank(table.getQuery())) { sampleQuery = @@ -337,6 +337,9 @@ default Object[] sampleDataFromColumn( if (count % samplingRate == 0) { results.add(rs.getObject(1)); } + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Thread interrupted"); + } } Object[] resultsArray = results.toArray(); Arrays.sort(resultsArray); 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 5527417e916..03067f6d5e3 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 @@ -131,7 +131,7 @@ public Object[] sampleDataFromColumn( String columnName, int samplingRate, int fetchSize) - throws SQLException { + throws Exception { String sampleQuery; if (StringUtils.isNotBlank(table.getQuery())) { sampleQuery = @@ -158,6 +158,9 @@ public Object[] sampleDataFromColumn( if (count % samplingRate == 0) { results.add(rs.getObject(1)); } + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Thread interrupted"); + } } Object[] resultsArray = results.toArray(); Arrays.sort(resultsArray); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/ChunkSplitter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/ChunkSplitter.java index 198dfe47cbc..f4da0a8d946 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/ChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/ChunkSplitter.java @@ -85,7 +85,7 @@ public synchronized void close() { } } - public Collection generateSplits(JdbcSourceTable table) throws SQLException { + public Collection generateSplits(JdbcSourceTable table) throws Exception { log.info("Start splitting table {} into chunks...", table.getTablePath()); long start = System.currentTimeMillis(); @@ -111,7 +111,7 @@ public Collection generateSplits(JdbcSourceTable table) throws } protected abstract Collection createSplits( - JdbcSourceTable table, SeaTunnelRowType splitKeyType) throws SQLException; + JdbcSourceTable table, SeaTunnelRowType splitKeyType) throws SQLException, Exception; public PreparedStatement generateSplitStatement(JdbcSourceSplit split, TableSchema schema) throws SQLException { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/DynamicChunkSplitter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/DynamicChunkSplitter.java index 9dc26d1ef22..d958c405dfb 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/DynamicChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/DynamicChunkSplitter.java @@ -62,7 +62,7 @@ public DynamicChunkSplitter(JdbcSourceConfig config) { @Override protected Collection createSplits( - JdbcSourceTable table, SeaTunnelRowType splitKey) throws SQLException { + JdbcSourceTable table, SeaTunnelRowType splitKey) throws Exception { return createDynamicSplits(table, splitKey); } @@ -73,7 +73,7 @@ protected PreparedStatement createSplitStatement(JdbcSourceSplit split, TableSch } private Collection createDynamicSplits( - JdbcSourceTable table, SeaTunnelRowType splitKey) throws SQLException { + JdbcSourceTable table, SeaTunnelRowType splitKey) throws Exception { String splitKeyName = splitKey.getFieldNames()[0]; SeaTunnelDataType splitKeyType = splitKey.getFieldType(0); List chunks = splitTableIntoChunks(table, splitKeyName, splitKeyType); @@ -105,7 +105,7 @@ private PreparedStatement createDynamicSplitStatement(JdbcSourceSplit split, Tab private List splitTableIntoChunks( JdbcSourceTable table, String splitColumnName, SeaTunnelDataType splitColumnType) - throws SQLException { + throws Exception { Pair minMax = queryMinMax(table, splitColumnName); Object min = minMax.getLeft(); Object max = minMax.getRight(); @@ -136,7 +136,7 @@ private List splitTableIntoChunks( private List evenlyColumnSplitChunks( JdbcSourceTable table, String splitColumnName, Object min, Object max, int chunkSize) - throws SQLException { + throws Exception { TablePath tablePath = table.getTablePath(); double distributionFactorUpper = config.getSplitEvenDistributionFactorUpperBound(); double distributionFactorLower = config.getSplitEvenDistributionFactorLowerBound(); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java index fa588cbd52d..70c9d39cf45 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleIT.java @@ -40,7 +40,6 @@ import java.math.BigDecimal; import java.sql.Date; -import java.sql.SQLException; import java.sql.Timestamp; import java.time.LocalDate; import java.time.LocalDateTime; @@ -104,7 +103,7 @@ public class JdbcOracleIT extends AbstractJdbcIT { }; @Test - public void testSampleDataFromColumnSuccess() throws SQLException { + public void testSampleDataFromColumnSuccess() throws Exception { JdbcDialect dialect = new OracleDialect(); JdbcSourceTable table = JdbcSourceTable.builder() 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/JdbcIrisIT.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/JdbcIrisIT.java index 4efa5bf651a..8fff364c3f8 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcIrisIT.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/JdbcIrisIT.java @@ -288,7 +288,7 @@ public class JdbcIrisIT extends AbstractJdbcIT { }; @Test - public void testSampleDataFromColumnSuccess() throws SQLException { + public void testSampleDataFromColumnSuccess() throws Exception { JdbcDialect dialect = new IrisDialect(); JdbcSourceTable table = JdbcSourceTable.builder() 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/JdbcMysqlSplitIT.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/JdbcMysqlSplitIT.java index cf28dd3a78e..d7df3e87c6d 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-7/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMysqlSplitIT.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/JdbcMysqlSplitIT.java @@ -468,7 +468,7 @@ public void testSplit() throws Exception { private JdbcSourceSplit[] getCheckedSplitArray( Map configMap, CatalogTable table, String splitKey, int splitNum) - throws SQLException { + throws Exception { configMap.put("partition_column", splitKey); DynamicChunkSplitter splitter = getDynamicChunkSplitter(configMap); From 4b6c13e8fcdb47fb3de816b879ca55eeb514bf76 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Tue, 14 May 2024 10:19:56 +0800 Subject: [PATCH 04/14] [Hotfix][Jdbc] Fix oracle savemode create table (#6651) --- .../jdbc/catalog/oracle/OracleCatalog.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java index e27d51d8475..1a3ac8178ce 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java @@ -120,6 +120,21 @@ protected String getCreateTableSql(TablePath tablePath, CatalogTable table) { return new OracleCreateTableSqlBuilder(table).build(tablePath); } + @Override + protected void createTableInternal(TablePath tablePath, CatalogTable table) + throws CatalogException { + String dbUrl = getUrlFromDatabaseName(tablePath.getDatabaseName()); + try { + String createTableSQL = getCreateTableSql(tablePath, table); + for (String sql : createTableSQL.split(";")) { + executeInternal(dbUrl, sql); + } + } catch (Exception e) { + // fallback to super + super.createTableInternal(tablePath, table); + } + } + @Override protected String getDropTableSql(TablePath tablePath) { return String.format("DROP TABLE %s", tablePath.getSchemaAndTableName("\"")); From 874f904ba786ec7041125d77d5a4b9b4f4464cf1 Mon Sep 17 00:00:00 2001 From: fcb-xiaobo <60566194+fcb-xiaobo@users.noreply.github.com> Date: Wed, 15 May 2024 09:55:21 +0800 Subject: [PATCH 05/14] [Doc][Improve] connector-v2 clickhouse/hbase/pulsar sink support chinese (#6811) * [Doc][Improve] connector-v2 clickhouse/hbase/pulsar sink support chinese * update doc style --------- Co-authored-by: fanchengbo --- docs/zh/connector-v2/sink/Clickhouse.md | 179 ++++++++++++++++++++++++ docs/zh/connector-v2/sink/Hbase.md | 122 ++++++++++++++++ docs/zh/connector-v2/sink/Pulsar.md | 168 ++++++++++++++++++++++ 3 files changed, 469 insertions(+) create mode 100644 docs/zh/connector-v2/sink/Clickhouse.md create mode 100644 docs/zh/connector-v2/sink/Hbase.md create mode 100644 docs/zh/connector-v2/sink/Pulsar.md diff --git a/docs/zh/connector-v2/sink/Clickhouse.md b/docs/zh/connector-v2/sink/Clickhouse.md new file mode 100644 index 00000000000..2b5e23d5568 --- /dev/null +++ b/docs/zh/connector-v2/sink/Clickhouse.md @@ -0,0 +1,179 @@ +# Clickhouse + +> Clickhouse 数据连接器 + +## 支持引擎 + +> Spark
+> Flink
+> SeaTunnel Zeta
+ +## 核心特性 + +- [ ] [精准一次](../../concept/connector-v2-features.md) +- [x] [cdc](../../concept/connector-v2-features.md) + +> Clickhouse sink 插件通过实现幂等写入可以达到精准一次,需要配合 aggregating merge tree 支持重复数据删除的引擎。 + +## 描述 + +用于将数据写入 Clickhouse。 + +## 支持的数据源信息 + +为了使用 Clickhouse 连接器,需要以下依赖项。它们可以通过 install-plugin.sh 或从 Maven 中央存储库下载。 + +| 数据源 | 支持的版本 | 依赖 | +|------------|-----------|------------------------------------------------------------------------------------------------------------| +| Clickhouse | universal | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-clickhouse) | + +## 数据类型映射 + +| SeaTunnel 数据类型 | Clickhouse 数据类型 | +|----------------|-----------------------------------------------------------------------------------------------------------------------------------------------| +| STRING | String / Int128 / UInt128 / Int256 / UInt256 / Point / Ring / Polygon MultiPolygon | +| INT | Int8 / UInt8 / Int16 / UInt16 / Int32 | +| BIGINT | UInt64 / Int64 / IntervalYear / IntervalQuarter / IntervalMonth / IntervalWeek / IntervalDay / IntervalHour / IntervalMinute / IntervalSecond | +| DOUBLE | Float64 | +| DECIMAL | Decimal | +| FLOAT | Float32 | +| DATE | Date | +| TIME | DateTime | +| ARRAY | Array | +| MAP | Map | + +## 输出选项 + +| 名称 | 类型 | 是否必须 | 默认值 | 描述 | +|---------------------------------------|---------|------|-------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| host | String | Yes | - | `ClickHouse` 集群地址, 格式是`host:port` , 允许多个`hosts`配置. 例如 `"host1:8123,host2:8123"`. | +| database | String | Yes | - | `ClickHouse` 数据库名称. | +| table | String | Yes | - | 表名称. | +| username | String | Yes | - | `ClickHouse` 用户账号. | +| password | String | Yes | - | `ClickHouse` 用户密码. | +| clickhouse.config | Map | No | | 除了上述必须由 `clickhouse-jdbc` 指定的必填参数外,用户还可以指定多个可选参数,这些参数涵盖了 `clickhouse-jdbc` 提供的所有[参数](https://github.com/ClickHouse/clickhouse-jdbc/tree/master/clickhouse-client#configuration). | +| bulk_size | String | No | 20000 | 每次通过[Clickhouse-jdbc](https://github.com/ClickHouse/clickhouse-jdbc) 写入的行数,即默认是20000. | +| split_mode | String | No | false | 此模式仅支持引擎为`Distributed`的 `clickhouse` 表。选项 `internal_replication` 应该是 `true` 。他们将在 seatunnel 中拆分分布式表数据,并直接对每个分片进行写入。分片权重定义为 `clickhouse` 将计算在内。 | +| sharding_key | String | No | - | 使用 `split_mode` 时,将数据发送到哪个节点是个问题,默认为随机选择,但可以使用`sharding_key`参数来指定分片算法的字段。此选项仅在`split_mode`为 `true` 时有效. | +| primary_key | String | No | - | 标记`clickhouse`表中的主键列,并根据主键执行INSERT/UPDATE/DELETE到`clickhouse`表. | +| support_upsert | Boolean | No | false | 支持按查询主键更新插入行. | +| allow_experimental_lightweight_delete | Boolean | No | false | 允许基于`MergeTree`表引擎实验性轻量级删除. | +| common-options | | No | - | Sink插件查用参数,详见[Sink常用选项](common-options.md). | + +## 如何创建一个clickhouse 同步任务 + +以下示例演示如何创建将随机生成的数据写入Clickhouse数据库的数据同步作业。 + +```bash +# Set the basic configuration of the task to be performed +env { + parallelism = 1 + job.mode = "BATCH" + checkpoint.interval = 1000 +} + +source { + FakeSource { + row.num = 2 + bigint.min = 0 + bigint.max = 10000000 + split.num = 1 + split.read-interval = 300 + schema { + fields { + c_bigint = bigint + } + } + } +} + +sink { + Clickhouse { + host = "127.0.0.1:9092" + database = "default" + table = "test" + username = "xxxxx" + password = "xxxxx" + } +} +``` + +### 小提示 + +> 1.[SeaTunnel 部署文档](../../start-v2/locally/deployment.md).
+> 2.需要在同步前提前创建要写入的表.
+> 3.当写入 ClickHouse 表,无需设置其结构,因为连接器会在写入前向 ClickHouse 查询当前表的结构信息.
+ +## Clickhouse 接收器配置 + +```hocon +sink { + Clickhouse { + host = "localhost:8123" + database = "default" + table = "fake_all" + username = "xxxxx" + password = "xxxxx" + clickhouse.config = { + max_rows_to_read = "100" + read_overflow_mode = "throw" + } + } +} +``` + +## 切分模式 + +```hocon +sink { + Clickhouse { + host = "localhost:8123" + database = "default" + table = "fake_all" + username = "xxxxx" + password = "xxxxx" + + # split mode options + split_mode = true + sharding_key = "age" + } +} +``` + +## CDC(Change data capture) Sink + +```hocon +sink { + Clickhouse { + host = "localhost:8123" + database = "default" + table = "fake_all" + username = "xxxxx" + password = "xxxxx" + + # cdc options + primary_key = "id" + support_upsert = true + } +} +``` + +## CDC(Change data capture) for *MergeTree engine + +```hocon +sink { + Clickhouse { + host = "localhost:8123" + database = "default" + table = "fake_all" + username = "xxxxx" + password = "xxxxx" + + # cdc options + primary_key = "id" + support_upsert = true + allow_experimental_lightweight_delete = true + } +} +``` + diff --git a/docs/zh/connector-v2/sink/Hbase.md b/docs/zh/connector-v2/sink/Hbase.md new file mode 100644 index 00000000000..9e79ed97995 --- /dev/null +++ b/docs/zh/connector-v2/sink/Hbase.md @@ -0,0 +1,122 @@ +# Hbase + +> Hbase 数据连接器 + +## 描述 + +将数据输出到hbase + +## 主要特性 + +- [ ] [精准一次](../../concept/connector-v2-features.md) + +## 选项 + +| 名称 | 类型 | 是否必须 | 默认值 | +|--------------------|---------|------|-----------------| +| zookeeper_quorum | string | yes | - | +| table | string | yes | - | +| rowkey_column | list | yes | - | +| family_name | config | yes | - | +| rowkey_delimiter | string | no | "" | +| version_column | string | no | - | +| null_mode | string | no | skip | +| wal_write | boolean | yes | false | +| write_buffer_size | string | no | 8 * 1024 * 1024 | +| encoding | string | no | utf8 | +| hbase_extra_config | string | no | - | +| common-options | | no | - | + +### zookeeper_quorum [string] + +hbase的zookeeper集群主机, 示例: "hadoop001:2181,hadoop002:2181,hadoop003:2181" + +### table [string] + +要写入的表名, 例如: "seatunnel" + +### rowkey_column [list] + +行键的列名列表, 例如: ["id", "uuid"] + +### family_name [config] + +字段的列簇名称映射。例如,上游的行如下所示: + +| id | name | age | +|----|---------------|-----| +| 1 | tyrantlucifer | 27 | + +id作为行键和其他写入不同列簇的字段,可以分配 + +family_name { +name = "info1" +age = "info2" +} + +这主要是name写入列簇info1,age写入将写给列簇 info2 + +如果要将其他字段写入同一列簇,可以分配 + +family_name { +all_columns = "info" +} + +这意味着所有字段都将写入该列簇 info + +### rowkey_delimiter [string] + +连接多行键的分隔符,默认 "" + +### version_column [string] + +版本列名称,您可以使用它来分配 hbase 记录的时间戳 + +### null_mode [double] + +写入 null 值的模式,支持 [ skip , empty], 默认 skip + +- skip: 当字段为 null ,连接器不会将此字段写入 hbase +- empty: 当字段为null时,连接器将写入并为此字段生成空值 + +### wal_write [boolean] + +wal log 写入标志,默认值 false + +### write_buffer_size [int] + +hbase 客户端的写入缓冲区大小,默认 8 * 1024 * 1024 + +### encoding [string] + +字符串字段的编码,支持[ utf8 , gbk],默认 utf8 + +### hbase_extra_config [config] + +hbase扩展配置 + +### 常见选项 + +Sink 插件常用参数,详见 Sink 常用选项 [Sink Common Options](common-options.md) + +## 案例 + +```hocon + +Hbase { + zookeeper_quorum = "hadoop001:2181,hadoop002:2181,hadoop003:2181" + table = "seatunnel_test" + rowkey_column = ["name"] + family_name { + all_columns = seatunnel + } +} + +``` + +## 更改日志 + +### 下一个版本 + +- 添加 hbase 输出连接器 ([4049](https://github.com/apache/seatunnel/pull/4049)) + diff --git a/docs/zh/connector-v2/sink/Pulsar.md b/docs/zh/connector-v2/sink/Pulsar.md new file mode 100644 index 00000000000..b85a41ae9df --- /dev/null +++ b/docs/zh/connector-v2/sink/Pulsar.md @@ -0,0 +1,168 @@ +# Pulsar + +> Pulsar 数据连接器 + +## 引擎支持 + +> Spark
+> Flink
+> Seatunnel Zeta
+ +## 核心特性 + +- [x] [精准一次](../../concept/connector-v2-features.md) + +## 描述 + +Apache Pulsar 的接收连接器。 + +## 支持的数据源信息 + +| 数据源 | 支持的版本 | +|--------|-----------| +| Pulsar | Universal | + +## 输出选项 + +| 名称 | 类型 | 是否必须 | 默认值 | 描述 | +|----------------------|--------|------|---------------------|-----------------------------------------| +| topic | String | Yes | - | 输出到Pulsar主题名称. | +| client.service-url | String | Yes | - | Pulsar 服务的服务 URL 提供者. | +| admin.service-url | String | Yes | - | 管理端点的 Pulsar 服务 HTTP URL. | +| auth.plugin-class | String | No | - | 身份验证插件的名称. | +| auth.params | String | No | - | 身份验证插件的参数. | +| format | String | No | json | 数据格式。默认格式为 json。可选的文本格式. | +| field_delimiter | String | No | , | 自定义数据格式的字段分隔符. | +| semantics | Enum | No | AT_LEAST_ONCE | 写入 pulsar 的一致性语义. | +| transaction_timeout | Int | No | 600 | 默认情况下,事务超时指定为 10 分钟. | +| pulsar.config | Map | No | - | 除了上述必须由 Pulsar 生产者客户端指定的参数外. | +| message.routing.mode | Enum | No | RoundRobinPartition | 要分区的消息的默认路由模式. | +| partition_key_fields | array | No | - | 配置哪些字段用作 pulsar 消息的键. | +| common-options | config | no | - | 源插件常用参数,详见源码 [常用选项](common-options.md). | + +## 参数解释 + +### client.service-url [String] + +Pulsar 服务的 Service URL 提供程序。要使用客户端库连接到 Pulsar, +您需要指定一个 Pulsar 协议 URL。您可以将 Pulsar 协议 URL 分配给特定集群并使用 Pulsar 方案。 + +例如, `localhost`: `pulsar://localhost:6650,localhost:6651`. + +### admin.service-url [String] + +管理端点的 Pulsar 服务 HTTP URL. + +例如, `http://my-broker.example.com:8080`, or `https://my-broker.example.com:8443` for TLS. + +### auth.plugin-class [String] + +身份验证插件的名称。 + +### auth.params [String] + +身份验证插件的参数。 + +例如, `key1:val1,key2:val2` + +### format [String] + +数据格式。默认格式为 json。可选的文本格式。默认字段分隔符为","。如果自定义分隔符,请添加"field_delimiter"选项。 + +### field_delimiter [String] + +自定义数据格式的字段分隔符。默认field_delimiter为','。 + +### semantics [Enum] + +写入 pulsar 的一致性语义。可用选项包括 EXACTLY_ONCE、NON、AT_LEAST_ONCE、默认AT_LEAST_ONCE。 +如果语义被指定为 EXACTLY_ONCE,我们将使用 2pc 来保证消息被准确地发送到 pulsar 一次。 +如果语义指定为 NON,我们将直接将消息发送到 pulsar,如果作业重启/重试或网络错误,数据可能会重复/丢失。 + +### transaction_timeout [Int] + +默认情况下,事务超时指定为 10 分钟。如果事务未在指定的超时时间内提交,则事务将自动中止。因此,您需要确保超时大于检查点间隔。 + +### pulsar.config [Map] + +除了上述 Pulsar 生产者客户端必须指定的参数外,用户还可以为生产者客户端指定多个非强制性参数, +涵盖 Pulsar 官方文档中指定的所有生产者参数。 + +### message.routing.mode [Enum] + +要分区的消息的默认路由模式。可用选项包括 SinglePartition、RoundRobinPartition。 +如果选择 SinglePartition,如果未提供密钥,分区生产者将随机选择一个分区并将所有消息发布到该分区中,如果消息上提供了密钥,则分区生产者将对密钥进行哈希处理并将消息分配给特定分区。 +如果选择 RoundRobinPartition,则如果未提供密钥,则生产者将以循环方式跨所有分区发布消息,以实现最大吞吐量。请注意,轮询不是按单个消息完成的,而是设置为相同的批处理延迟边界,以确保批处理有效。 + +### partition_key_fields [String] + +配置哪些字段用作 pulsar 消息的键。 + +例如,如果要使用上游数据中的字段值作为键,则可以为此属性分配字段名称。 + +上游数据如下: + +| name | age | data | +|------|-----|---------------| +| Jack | 16 | data-example1 | +| Mary | 23 | data-example2 | + +如果将 name 设置为键,则 name 列的哈希值将确定消息发送到哪个分区。 + +如果未设置分区键字段,则将向 null 消息键发送至。 + +消息键的格式为 json,如果 name 设置为键,例如 '{“name”:“Jack”}'。 + +所选字段必须是上游的现有字段。 + +### 常见选项 + +源插件常用参数,详见源码[常用选项](common-options.md) . + +## 任务示例 + +### 简单: + +> 该示例定义了一个 SeaTunnel 同步任务,该任务通过 FakeSource 自动生成数据并将其发送到 Pulsar Sink。FakeSource 总共生成 16 行数据 (row.num=16),每行有两个字段,name(字符串类型)和 age(int 类型)。最终目标主题是test_topic主题中还将有 16 行数据。 如果您尚未安装和部署 SeaTunnel,则需要按照[安装Seatunnel](../../start-v2/locally/deployment.md) SeaTunnel 中的说明安装和部署 SeaTunnel。然后按照 [SeaTunnel 引擎快速入门](../../start-v2/locally/quick-start-seatunnel-engine.md)中的说明运行此作业。 + +```hocon +# Defining the runtime environment +env { + # You can set flink configuration here + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + parallelism = 1 + result_table_name = "fake" + row.num = 16 + schema = { + fields { + name = "string" + age = "int" + } + } + } +} + +sink { + Pulsar { + topic = "example" + client.service-url = "localhost:pulsar://localhost:6650" + admin.service-url = "http://my-broker.example.com:8080" + result_table_name = "test" + pulsar.config = { + sendTimeoutMs = 30000 + } + } +} +``` + +## 更改日志 + +### 下一个版本 + +- 添加 Pulsar Sink 连接器 + From 4cefb573c3eca4305634be3bf60581e990fa1ad2 Mon Sep 17 00:00:00 2001 From: seckiller <455482064@qq.com> Date: Wed, 15 May 2024 10:01:25 +0800 Subject: [PATCH 06/14] [Improve] Optimize log display during startup command (#6816) Co-authored-by: JohnZp --- .../seatunnel-starter/src/main/bin/seatunnel-cluster.cmd | 2 +- .../seatunnel-starter/src/main/bin/seatunnel-cluster.sh | 2 +- seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.cmd | 2 +- seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.sh | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.cmd b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.cmd index e94a4bb482f..067646730d5 100644 --- a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.cmd +++ b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.cmd @@ -62,7 +62,7 @@ REM port in your IDE. After that, you can happily debug your code. REM set "JAVA_OPTS=%JAVA_OPTS% -Xdebug -Xrunjdwp:server=y,transport=dt_socket,address=5001,suspend=y" if exist "%CONF_DIR%\log4j2.properties" ( - set "JAVA_OPTS=%JAVA_OPTS% -Dlog4j2.configurationFile=%CONF_DIR%\log4j2.properties" + set "JAVA_OPTS=%JAVA_OPTS% -Dhazelcast.logging.type=log4j2 -Dlog4j2.configurationFile=%CONF_DIR%\log4j2.properties" set "JAVA_OPTS=%JAVA_OPTS% -Dseatunnel.logs.path=%APP_DIR%\logs" set "JAVA_OPTS=%JAVA_OPTS% -Dseatunnel.logs.file_name=seatunnel-engine-server" ) diff --git a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.sh b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.sh index 1eda5a15e1e..e6e48123c71 100755 --- a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.sh +++ b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.sh @@ -84,7 +84,7 @@ JAVA_OPTS="${JAVA_OPTS} -Dhazelcast.config=${HAZELCAST_CONFIG}" # Log4j2 Config JAVA_OPTS="${JAVA_OPTS} -Dlog4j2.contextSelector=org.apache.logging.log4j.core.async.AsyncLoggerContextSelector" if [ -e "${CONF_DIR}/log4j2.properties" ]; then - JAVA_OPTS="${JAVA_OPTS} -Dlog4j2.configurationFile=${CONF_DIR}/log4j2.properties" + JAVA_OPTS="${JAVA_OPTS} -Dhazelcast.logging.type=log4j2 -Dlog4j2.configurationFile=${CONF_DIR}/log4j2.properties" JAVA_OPTS="${JAVA_OPTS} -Dseatunnel.logs.path=${APP_DIR}/logs" JAVA_OPTS="${JAVA_OPTS} -Dseatunnel.logs.file_name=seatunnel-engine-server" fi diff --git a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.cmd b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.cmd index cf9258e9d50..20e253847ad 100644 --- a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.cmd +++ b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.cmd @@ -77,7 +77,7 @@ REM set "JAVA_OPTS=%JAVA_OPTS% -Xdebug -Xrunjdwp:transport=dt_socket,server=y,ad REM Log4j2 Config if exist "%CONF_DIR%\log4j2_client.properties" ( - set "JAVA_OPTS=%JAVA_OPTS% -Dlog4j2.configurationFile=%CONF_DIR%\log4j2_client.properties" + set "JAVA_OPTS=%JAVA_OPTS% -Dhazelcast.logging.type=log4j2 -Dlog4j2.configurationFile=%CONF_DIR%\log4j2_client.properties" set "JAVA_OPTS=%JAVA_OPTS% -Dseatunnel.logs.path=%APP_DIR%\logs" for %%i in (%args%) do ( set "arg=%%i" diff --git a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.sh b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.sh index e7754e39759..644ec02eae0 100755 --- a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.sh +++ b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.sh @@ -89,7 +89,7 @@ JAVA_OPTS="${JAVA_OPTS} -Dhazelcast.config=${HAZELCAST_CONFIG}" # Log4j2 Config if [ -e "${CONF_DIR}/log4j2_client.properties" ]; then - JAVA_OPTS="${JAVA_OPTS} -Dlog4j2.configurationFile=${CONF_DIR}/log4j2_client.properties" + JAVA_OPTS="${JAVA_OPTS} -Dhazelcast.logging.type=log4j2 -Dlog4j2.configurationFile=${CONF_DIR}/log4j2_client.properties" JAVA_OPTS="${JAVA_OPTS} -Dseatunnel.logs.path=${APP_DIR}/logs" if [[ $args == *" -m local"* || $args == *" --master local"* || $args == *" -e local"* || $args == *" --deploy-mode local"* ]]; then ntime=$(echo `date "+%N"`|sed -r 's/^0+//') From 68149acad60b7ac59d7cfec04adb10b2bd371d5d Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 15 May 2024 13:49:03 +0800 Subject: [PATCH 07/14] [Hotfix][Zeta] Fix savepoint execute result (#6850) --- .../engine/server/CoordinatorService.java | 7 ++++++- .../seatunnel/engine/server/master/JobMaster.java | 14 ++++++++++++-- .../server/checkpoint/CheckpointStorageTest.java | 2 +- .../engine/server/checkpoint/SavePointTest.java | 3 +-- 4 files changed, 20 insertions(+), 6 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java index 0bf2c0de4dc..bef6e34a760 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java @@ -516,7 +516,12 @@ public PassiveCompletableFuture savePoint(long jobId) { CompletableFuture.supplyAsync( () -> { JobMaster runningJobMaster = runningJobMasterMap.get(jobId); - runningJobMaster.savePoint().join(); + if (!runningJobMaster.savePoint().join()) { + throw new SavePointFailedException( + "The job with id '" + + jobId + + "' save point failed"); + } return null; }, executorService)); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java index 6b90fce8e29..01b2af48a89 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java @@ -668,7 +668,7 @@ public void updateTaskExecutionState(TaskExecutionState taskExecutionState) { } /** Execute savePoint, which will cause the job to end. */ - public CompletableFuture savePoint() { + public CompletableFuture savePoint() { LOGGER.info( String.format( "Begin do save point for Job %s (%s) ", @@ -677,7 +677,17 @@ public CompletableFuture savePoint() { physicalPlan.savepointJob(); PassiveCompletableFuture[] passiveCompletableFutures = checkpointManager.triggerSavePoints(); - return CompletableFuture.allOf(passiveCompletableFutures); + return CompletableFuture.supplyAsync( + () -> + Arrays.stream(passiveCompletableFutures) + .allMatch( + future -> { + try { + return future.get() != null; + } catch (Exception e) { + throw new SeaTunnelEngineException(e); + } + })); } public void setOwnedSlotProfiles( diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointStorageTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointStorageTest.java index 4e72114e259..2a334fe3849 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointStorageTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointStorageTest.java @@ -77,7 +77,7 @@ public void testGenerateFileWhenSavepoint() .getJobStatus(jobId) .equals(JobStatus.RUNNING))); Thread.sleep(1000); - CompletableFuture future1 = + CompletableFuture future1 = server.getCoordinatorService().getJobMaster(jobId).savePoint(); future1.join(); await().atMost(120000, TimeUnit.MILLISECONDS) diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/checkpoint/SavePointTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/checkpoint/SavePointTest.java index c062a95941d..ab3c447f837 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/checkpoint/SavePointTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/checkpoint/SavePointTest.java @@ -82,8 +82,7 @@ public void testSavePointButJobGoingToFail() throws InterruptedException { } catch (Exception e) { errorCount++; } - // we should make sure only one savepoint success in the same time - Assertions.assertEquals(2, errorCount); + Assertions.assertEquals(3, errorCount); await().atMost(120, TimeUnit.SECONDS) .untilAsserted( () -> From 454c339b9c03da0654e743149e0fc8616577d864 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 15 May 2024 13:49:41 +0800 Subject: [PATCH 08/14] [Improve][CDC] Close idle subtasks gorup(reader/writer) in increment phase (#6526) --- .../base/dialect/JdbcDataSourceDialect.java | 4 +- .../enumerator/HybridSplitAssigner.java | 3 +- .../IncrementalSourceEnumerator.java | 12 ++- .../enumerator/IncrementalSplitAssigner.java | 4 + .../reader/IncrementalSourceReader.java | 8 +- .../cdc/mysql/source/MySqlDialect.java | 19 +++-- .../source/MysqlPooledDataSourceFactory.java | 36 -------- .../fetch/binlog/MySqlBinlogFetchTask.java | 20 +++++ .../cdc/oracle/source/OracleDialect.java | 6 -- .../source/OraclePooledDataSourceFactory.java | 43 ---------- .../cdc/postgres/source/PostgresDialect.java | 6 -- .../PostgresPooledDataSourceFactory.java | 35 -------- .../source/source/SqlServerDialect.java | 6 -- .../SqlServerPooledDataSourceFactory.java | 35 -------- .../SqlServerSourceFetchTaskContext.java | 21 ++++- .../source/reader/SourceReaderBase.java | 19 +++-- .../http/source/HttpSourceReader.java | 2 +- .../seatunnel/cdc/mysql/MysqlCDCIT.java | 24 +++++- .../src/test/resources/docker/setup.sql | 11 ++- .../src/test/resources/mysqlcdc_to_mysql.conf | 8 +- ...lcdc_to_mysql_with_custom_primary_key.conf | 4 +- ...dc_to_mysql_with_disable_exactly_once.conf | 8 +- ...mysql_with_multi_table_mode_one_table.conf | 11 ++- ...mysql_with_multi_table_mode_two_table.conf | 11 ++- ...mysqlcdc_to_mysql_with_no_primary_key.conf | 8 +- .../seatunnel/e2e/connector/http/HttpIT.java | 1 + .../server/checkpoint/CheckpointBarrier.java | 43 ++++++++-- .../checkpoint/CheckpointCoordinator.java | 82 ++++++++++++++++++- .../server/checkpoint/CheckpointManager.java | 9 ++ .../server/dag/physical/PhysicalVertex.java | 8 ++ .../engine/server/master/JobMaster.java | 47 +++++++++++ .../server/serializable/RecordSerializer.java | 8 +- .../serializable/TaskDataSerializerHook.java | 5 ++ .../engine/server/task/SeaTunnelTask.java | 3 +- .../task/SinkAggregatedCommitterTask.java | 11 ++- .../task/SourceSplitEnumeratorTask.java | 28 +++++-- .../task/flow/ShuffleSinkFlowLifeCycle.java | 2 +- .../task/flow/ShuffleSourceFlowLifeCycle.java | 2 +- .../server/task/flow/SinkFlowLifeCycle.java | 2 +- .../server/task/flow/SourceFlowLifeCycle.java | 2 +- .../task/flow/TransformFlowLifeCycle.java | 2 +- .../queue/IntermediateBlockingQueue.java | 2 +- .../queue/disruptor/RecordEventHandler.java | 2 +- .../queue/disruptor/RecordEventProducer.java | 3 +- .../source/CloseIdleReaderOperation.java | 72 ++++++++++++++++ .../source/SourceNoMoreElementOperation.java | 2 +- .../engine/server/task/record/Barrier.java | 21 +++++ .../engine/server/master/JobMasterTest.java | 70 ++++++++++++++++ .../resources/stream_fakesource_to_file.conf | 3 +- 49 files changed, 541 insertions(+), 253 deletions(-) delete mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MysqlPooledDataSourceFactory.java delete mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OraclePooledDataSourceFactory.java delete mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresPooledDataSourceFactory.java delete mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerPooledDataSourceFactory.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/CloseIdleReaderOperation.java diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/dialect/JdbcDataSourceDialect.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/dialect/JdbcDataSourceDialect.java index 712328b5f90..4a497daf718 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/dialect/JdbcDataSourceDialect.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/dialect/JdbcDataSourceDialect.java @@ -71,7 +71,9 @@ default JdbcConnection openJdbcConnection(JdbcSourceConfig sourceConfig) { } /** Get a connection pool factory to create connection pool. */ - JdbcConnectionPoolFactory getPooledDataSourceFactory(); + default JdbcConnectionPoolFactory getPooledDataSourceFactory() { + throw new UnsupportedOperationException(); + } /** Query and build the schema of table. */ TableChanges.TableChange queryTableSchema(JdbcConnection jdbc, TableId tableId); 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 d6b0bdb96cb..4acd0924788 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 @@ -114,7 +114,8 @@ public Optional getNext() { @Override public boolean waitingForCompletedSplits() { - return snapshotSplitAssigner.waitingForCompletedSplits(); + return snapshotSplitAssigner.waitingForCompletedSplits() + || incrementalSplitAssigner.waitingForAssignedSplits(); } @Override 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 b17b910e5d8..87ca7dc507a 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 @@ -177,8 +177,16 @@ private void assignSplits() { awaitingReader.remove(); LOG.debug("Assign split {} to subtask {}", sourceSplit, nextAwaiting); } else { - // there is no available splits by now, skip assigning - break; + if (splitAssigner.waitingForCompletedSplits()) { + // there is no available splits by now, skip assigning + break; + } else { + LOG.info( + "No more splits available, signal no more splits to subtask {}", + nextAwaiting); + context.signalNoMoreSplits(nextAwaiting); + awaitingReader.remove(); + } } } } 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 7b45ee1ef61..1accc47af23 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 @@ -278,4 +278,8 @@ public boolean completedSnapshotPhase(List tableIds) { return context.getAssignedSnapshotSplit().isEmpty() && context.getSplitCompletedOffsets().isEmpty(); } + + public boolean waitingForAssignedSplits() { + return !(splitAssigned && noMoreSplits()); + } } 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 b5fc443310f..abfccdeb75e 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 @@ -111,7 +111,13 @@ public void pollNext(Collector output) throws Exception { context.sendSplitRequest(); needSendSplitRequest.compareAndSet(true, false); } - super.pollNext(output); + + if (isNoMoreSplitsAssignment() && isNoMoreElement()) { + log.info("Reader {} send NoMoreElement event", context.getIndexOfSubtask()); + context.signalNoMoreElement(); + } else { + super.pollNext(output); + } } @Override diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlDialect.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlDialect.java index c43b819f066..b450ab84ae1 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlDialect.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlDialect.java @@ -23,7 +23,6 @@ import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; -import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionPoolFactory; import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkSplitter; import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; @@ -34,6 +33,7 @@ import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.source.reader.fetch.MySqlSourceFetchTaskContext; import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.source.reader.fetch.binlog.MySqlBinlogFetchTask; import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.source.reader.fetch.scan.MySqlSnapshotFetchTask; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.utils.MySqlConnectionUtils; import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.utils.MySqlSchema; import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.utils.TableDiscoveryUtils; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; @@ -70,20 +70,24 @@ public String getName() { @Override public boolean isDataCollectionIdCaseSensitive(JdbcSourceConfig sourceConfig) { try (JdbcConnection jdbcConnection = openJdbcConnection(sourceConfig)) { - return isTableIdCaseSensitive(jdbcConnection); + return isDataCollectionIdCaseSensitive(jdbcConnection); } catch (SQLException e) { throw new SeaTunnelException("Error reading MySQL variables: " + e.getMessage(), e); } } + private boolean isDataCollectionIdCaseSensitive(JdbcConnection jdbcConnection) { + return isTableIdCaseSensitive(jdbcConnection); + } + @Override - public ChunkSplitter createChunkSplitter(JdbcSourceConfig sourceConfig) { - return new MySqlChunkSplitter(sourceConfig, this); + public JdbcConnection openJdbcConnection(JdbcSourceConfig sourceConfig) { + return MySqlConnectionUtils.createMySqlConnection(sourceConfig.getDbzConfiguration()); } @Override - public JdbcConnectionPoolFactory getPooledDataSourceFactory() { - return new MysqlPooledDataSourceFactory(); + public ChunkSplitter createChunkSplitter(JdbcSourceConfig sourceConfig) { + return new MySqlChunkSplitter(sourceConfig, this); } @Override @@ -101,8 +105,7 @@ public List discoverDataCollections(JdbcSourceConfig sourceConfig) { public TableChanges.TableChange queryTableSchema(JdbcConnection jdbc, TableId tableId) { if (mySqlSchema == null) { mySqlSchema = - new MySqlSchema( - sourceConfig, isDataCollectionIdCaseSensitive(sourceConfig), tableMap); + new MySqlSchema(sourceConfig, isDataCollectionIdCaseSensitive(jdbc), tableMap); } return mySqlSchema.getTableSchema(jdbc, tableId); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MysqlPooledDataSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MysqlPooledDataSourceFactory.java deleted file mode 100644 index 9a0ae21b209..00000000000 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MysqlPooledDataSourceFactory.java +++ /dev/null @@ -1,36 +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.connectors.seatunnel.cdc.mysql.source; - -import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; -import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionPoolFactory; - -/** A MySQL datasource factory. */ -public class MysqlPooledDataSourceFactory extends JdbcConnectionPoolFactory { - - public static final String JDBC_URL_PATTERN = - "jdbc:mysql://%s:%s/?useInformationSchema=true&nullCatalogMeansCurrent=false&useUnicode=true&characterEncoding=UTF-8&characterSetResults=UTF-8&zeroDateTimeBehavior=CONVERT_TO_NULL"; - - @Override - public String getJdbcUrl(JdbcSourceConfig sourceConfig) { - String hostName = sourceConfig.getHostname(); - int port = sourceConfig.getPort(); - - return String.format(JDBC_URL_PATTERN, hostName, port); - } -} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/reader/fetch/binlog/MySqlBinlogFetchTask.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/reader/fetch/binlog/MySqlBinlogFetchTask.java index f858864e78a..c61cef79afb 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/reader/fetch/binlog/MySqlBinlogFetchTask.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/reader/fetch/binlog/MySqlBinlogFetchTask.java @@ -29,6 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.github.shyiko.mysql.binlog.BinaryLogClient; import com.github.shyiko.mysql.binlog.event.Event; import io.debezium.DebeziumException; import io.debezium.connector.mysql.MySqlConnection; @@ -40,12 +41,15 @@ import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.source.spi.ChangeEventSource; import io.debezium.util.Clock; +import lombok.extern.slf4j.Slf4j; +import java.sql.SQLException; import java.util.HashMap; import java.util.Map; import static org.apache.seatunnel.connectors.seatunnel.cdc.mysql.source.offset.BinlogOffset.NO_STOPPING_OFFSET; +@Slf4j public class MySqlBinlogFetchTask implements FetchTask { private final IncrementalSplit split; private volatile boolean taskRunning = false; @@ -72,6 +76,22 @@ public void execute(FetchTask.Context context) throws Exception { BinlogSplitChangeEventSourceContext changeEventSourceContext = new BinlogSplitChangeEventSourceContext(); + sourceFetchContext + .getBinaryLogClient() + .registerLifecycleListener( + new BinaryLogClient.AbstractLifecycleListener() { + @Override + public void onConnect(BinaryLogClient client) { + try { + sourceFetchContext.getConnection().close(); + log.info( + "Binlog client connected, closed idle jdbc connection."); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + }); + mySqlStreamingChangeEventSource.execute( changeEventSourceContext, sourceFetchContext.getOffsetContext()); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleDialect.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleDialect.java index 5ffef4cc3cc..d1908badc0b 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleDialect.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OracleDialect.java @@ -23,7 +23,6 @@ import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; -import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionPoolFactory; import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkSplitter; import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; @@ -90,11 +89,6 @@ public ChunkSplitter createChunkSplitter(JdbcSourceConfig sourceConfig) { return new OracleChunkSplitter(sourceConfig, this); } - @Override - public JdbcConnectionPoolFactory getPooledDataSourceFactory() { - return new OraclePooledDataSourceFactory(); - } - @Override public List discoverDataCollections(JdbcSourceConfig sourceConfig) { OracleSourceConfig oracleSourceConfig = (OracleSourceConfig) sourceConfig; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OraclePooledDataSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OraclePooledDataSourceFactory.java deleted file mode 100644 index a17ac423496..00000000000 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-oracle/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/oracle/source/OraclePooledDataSourceFactory.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.cdc.oracle.source; - -import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; -import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionPoolFactory; -import org.apache.seatunnel.connectors.seatunnel.cdc.oracle.config.OracleSourceConfig; - -import org.apache.commons.lang3.StringUtils; - -/** A Oracle datasource factory. */ -public class OraclePooledDataSourceFactory extends JdbcConnectionPoolFactory { - - public static final String JDBC_URL_PATTERN = "jdbc:oracle:thin:@%s:%s:%s"; - - @Override - public String getJdbcUrl(JdbcSourceConfig sourceConfig) { - OracleSourceConfig oracleSourceConfig = (OracleSourceConfig) sourceConfig; - if (StringUtils.isNotBlank(oracleSourceConfig.getOriginUrl())) { - return oracleSourceConfig.getOriginUrl(); - } else { - String hostName = sourceConfig.getHostname(); - int port = sourceConfig.getPort(); - String database = sourceConfig.getDatabaseList().get(0); - return String.format(JDBC_URL_PATTERN, hostName, port, database); - } - } -} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresDialect.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresDialect.java index cf6b624db95..72e8f6724e6 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresDialect.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresDialect.java @@ -23,7 +23,6 @@ import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; -import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionPoolFactory; import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkSplitter; import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; @@ -98,11 +97,6 @@ public ChunkSplitter createChunkSplitter(JdbcSourceConfig sourceConfig) { return new PostgresChunkSplitter(sourceConfig, this); } - @Override - public JdbcConnectionPoolFactory getPooledDataSourceFactory() { - return new PostgresPooledDataSourceFactory(); - } - @Override public List discoverDataCollections(JdbcSourceConfig sourceConfig) { PostgresSourceConfig postgresSourceConfig = (PostgresSourceConfig) sourceConfig; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresPooledDataSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresPooledDataSourceFactory.java deleted file mode 100644 index e1cfa4e912e..00000000000 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresPooledDataSourceFactory.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source; - -import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; -import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionPoolFactory; - -/** Factory to create {@link JdbcConnectionPoolFactory} for Postgre SQL. */ -public class PostgresPooledDataSourceFactory extends JdbcConnectionPoolFactory { - - private static final String URL_PATTERN = "jdbc:postgresql://%s:%s/%s"; - - @Override - public String getJdbcUrl(JdbcSourceConfig sourceConfig) { - String hostName = sourceConfig.getHostname(); - int port = sourceConfig.getPort(); - String database = sourceConfig.getDatabaseList().get(0); - return String.format(URL_PATTERN, hostName, port, database); - } -} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerDialect.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerDialect.java index e667412378c..aa82024b71a 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerDialect.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerDialect.java @@ -23,7 +23,6 @@ import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; -import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionPoolFactory; import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkSplitter; import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; @@ -85,11 +84,6 @@ public ChunkSplitter createChunkSplitter(JdbcSourceConfig sourceConfig) { return new SqlServerChunkSplitter(sourceConfig, this); } - @Override - public JdbcConnectionPoolFactory getPooledDataSourceFactory() { - return new SqlServerPooledDataSourceFactory(); - } - @Override public List discoverDataCollections(JdbcSourceConfig sourceConfig) { SqlServerSourceConfig sqlServerSourceConfig = (SqlServerSourceConfig) sourceConfig; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerPooledDataSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerPooledDataSourceFactory.java deleted file mode 100644 index 911b9af1473..00000000000 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerPooledDataSourceFactory.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.cdc.sqlserver.source.source; - -import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; -import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionPoolFactory; - -/** Factory to create {@link JdbcConnectionPoolFactory} for SQL Server. */ -public class SqlServerPooledDataSourceFactory extends JdbcConnectionPoolFactory { - - private static final String URL_PATTERN = "jdbc:sqlserver://%s:%s;databaseName=%s"; - - @Override - public String getJdbcUrl(JdbcSourceConfig sourceConfig) { - String hostName = sourceConfig.getHostname(); - int port = sourceConfig.getPort(); - String database = sourceConfig.getDatabaseList().get(0); - return String.format(URL_PATTERN, hostName, port, database); - } -} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/reader/fetch/SqlServerSourceFetchTaskContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/reader/fetch/SqlServerSourceFetchTaskContext.java index 8178c9f30af..6aadf1aca95 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/reader/fetch/SqlServerSourceFetchTaskContext.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/reader/fetch/SqlServerSourceFetchTaskContext.java @@ -74,7 +74,7 @@ public class SqlServerSourceFetchTaskContext extends JdbcSourceFetchTaskContext private final SqlServerConnection dataConnection; - private final SqlServerConnection metadataConnection; + private SqlServerConnection metadataConnection; private final SqlServerEventMetadataProvider metadataProvider; private SqlServerDatabaseSchema databaseSchema; @@ -92,7 +92,6 @@ public SqlServerSourceFetchTaskContext( super(sourceConfig, dataSourceDialect); this.dataConnection = createSqlServerConnection(sourceConfig.getDbzConfiguration()); - this.metadataConnection = createSqlServerConnection(sourceConfig.getDbzConfiguration()); this.metadataProvider = new SqlServerEventMetadataProvider(); } @@ -162,13 +161,29 @@ public void configure(SourceSplitBase sourceSplitBase) { taskContext, queue, metadataProvider); this.errorHandler = new SqlServerErrorHandler(connectorConfig.getLogicalName(), queue); + if (sourceSplitBase.isIncrementalSplit() || isExactlyOnce()) { + initMetadataConnection(); + } + } + + private void initMetadataConnection() { + if (this.metadataConnection == null) { + synchronized (this) { + if (this.metadataConnection == null) { + this.metadataConnection = + createSqlServerConnection(sourceConfig.getDbzConfiguration()); + } + } + } } @Override public void close() { try { this.dataConnection.close(); - this.metadataConnection.close(); + if (this.metadataConnection != null) { + this.metadataConnection.close(); + } } catch (SQLException e) { log.warn("Failed to close connection", e); } 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 29dd2ff6f5e..7ec6cc83e9e 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 @@ -65,7 +65,7 @@ public abstract class SourceReaderBase currentFetch; protected SplitContext currentSplitContext; private Collector currentSplitOutput; - private boolean noMoreSplitsAssignment; + @Getter private volatile boolean noMoreSplitsAssignment; public SourceReaderBase( BlockingQueue> elementsQueue, @@ -94,10 +94,11 @@ public void pollNext(Collector output) throws Exception { if (recordsWithSplitId == null) { if (Boundedness.BOUNDED.equals(context.getBoundedness()) && noMoreSplitsAssignment - && splitFetcherManager.maybeShutdownFinishedFetchers() - && elementsQueue.isEmpty()) { + && isNoMoreElement()) { context.signalNoMoreElement(); - log.info("Send NoMoreElement event"); + log.info( + "Reader {} into idle state, send NoMoreElement event", + context.getIndexOfSubtask()); } return; } @@ -137,7 +138,7 @@ public void addSplits(List splits) { @Override public void handleNoMoreSplits() { - log.info("Reader received NoMoreSplits event."); + log.info("Reader {} received NoMoreSplits event.", context.getIndexOfSubtask()); noMoreSplitsAssignment = true; } @@ -146,9 +147,15 @@ public void handleSourceEvent(SourceEvent sourceEvent) { log.info("Received unhandled source event: {}", sourceEvent); } + protected boolean isNoMoreElement() { + return splitFetcherManager.maybeShutdownFinishedFetchers() + && elementsQueue.isEmpty() + && currentFetch == null; + } + @Override public void close() { - log.info("Closing Source Reader."); + log.info("Closing Source Reader {}.", context.getIndexOfSubtask()); try { splitFetcherManager.close(options.getSourceReaderCloseTimeout()); } catch (Exception e) { diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSourceReader.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSourceReader.java index c2c3da69681..104d769ef5f 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSourceReader.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSourceReader.java @@ -130,7 +130,7 @@ public void pollAndCollectData(Collector output) throws Exception collect(output, content); } } - log.info( + log.debug( "http client execute success request param:[{}], http response status code:[{}], content:[{}]", httpParameter.getParams(), response.getCode(), diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCIT.java index 62dc3f077d0..7fab60f9fc8 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCIT.java @@ -63,8 +63,8 @@ public class MysqlCDCIT extends TestSuiteBase implements TestResource { // mysql private static final String MYSQL_HOST = "mysql_cdc_e2e"; - private static final String MYSQL_USER_NAME = "st_user"; - private static final String MYSQL_USER_PASSWORD = "seatunnel"; + private static final String MYSQL_USER_NAME = "mysqluser"; + private static final String MYSQL_USER_PASSWORD = "mysqlpw"; private static final String MYSQL_DATABASE = "mysql_cdc"; private static final String MYSQL_DATABASE2 = "mysql_cdc2"; private static final MySqlContainer MYSQL_CONTAINER = createMySqlContainer(MySqlVersion.V8_0); @@ -357,6 +357,12 @@ public void testMultiTableWithRestore(TestContainer container) getSourceQuerySQL( MYSQL_DATABASE2, SOURCE_TABLE_1))))); + await().atMost(60000, TimeUnit.MILLISECONDS) + .pollInterval(1000, TimeUnit.MILLISECONDS) + .until(() -> getConnectionStatus("st_user_source").size() == 1); + await().atMost(60000, TimeUnit.MILLISECONDS) + .pollInterval(1000, TimeUnit.MILLISECONDS) + .until(() -> getConnectionStatus("st_user_sink").size() == 1); Pattern jobIdPattern = Pattern.compile( @@ -413,6 +419,13 @@ public void testMultiTableWithRestore(TestContainer container) MYSQL_DATABASE2, SOURCE_TABLE_2))))); + await().atMost(60000, TimeUnit.MILLISECONDS) + .pollInterval(1000, TimeUnit.MILLISECONDS) + .until(() -> getConnectionStatus("st_user_source").size() == 1); + await().atMost(60000, TimeUnit.MILLISECONDS) + .pollInterval(1000, TimeUnit.MILLISECONDS) + .until(() -> getConnectionStatus("st_user_sink").size() == 1); + log.info("****************** container logs start ******************"); String containerLogs = container.getServerLogs(); log.info(containerLogs); @@ -479,6 +492,13 @@ private Connection getJdbcConnection() throws SQLException { MYSQL_CONTAINER.getPassword()); } + private List> getConnectionStatus(String user) { + return query( + "select USER,HOST,DB,COMMAND,TIME,STATE from information_schema.processlist where USER = '" + + user + + "'"); + } + private List> query(String sql) { try (Connection connection = getJdbcConnection()) { ResultSet resultSet = connection.createStatement().executeQuery(sql); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/docker/setup.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/docker/setup.sql index 0a54c2c2822..079b8f1d95f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/docker/setup.sql +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/docker/setup.sql @@ -19,13 +19,16 @@ -- to prevent other clients accessing the log from other machines. For example, 'replicator'@'follower.acme.com'. -- However, in this database we'll grant 2 users different privileges: -- --- 1) 'st_user' - all privileges required by the snapshot reader AND binlog reader (used for testing) --- 2) 'mysqluser' - all privileges +-- 1) 'mysqluser' - all privileges +-- 2) 'st_user_source' - all privileges required by the snapshot reader AND binlog reader (used for testing) +-- 3) 'st_user_sink' - all privileges required by the write data (used for testing) -- -GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT, DROP, LOCK TABLES ON *.* TO 'st_user'@'%'; -CREATE USER 'mysqluser' IDENTIFIED BY 'mysqlpw'; GRANT ALL PRIVILEGES ON *.* TO 'mysqluser'@'%'; +CREATE USER 'st_user_source' IDENTIFIED BY 'mysqlpw'; +GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT, DROP, LOCK TABLES ON *.* TO 'st_user_source'@'%'; +CREATE USER 'st_user_sink' IDENTIFIED BY 'mysqlpw'; +GRANT SELECT, INSERT, UPDATE, DELETE, CREATE, DROP, INDEX, ALTER ON *.* TO 'st_user_sink'@'%'; -- ---------------------------------------------------------------------------------------------------------------- -- DATABASE: emptydb -- ---------------------------------------------------------------------------------------------------------------- diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql.conf index a5874db62d3..4a352455d9c 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql.conf @@ -31,8 +31,8 @@ source { MySQL-CDC { result_table_name = "customers_mysql_cdc" server-id = 5652 - username = "st_user" - password = "seatunnel" + username = "st_user_source" + password = "mysqlpw" table-names = ["mysql_cdc.mysql_cdc_e2e_source_table"] base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" } @@ -55,8 +55,8 @@ sink { source_table_name = "trans_mysql_cdc" url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" driver = "com.mysql.cj.jdbc.Driver" - user = "st_user" - password = "seatunnel" + user = "st_user_sink" + password = "mysqlpw" generate_sink_sql = true # You need to configure both database and table diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_custom_primary_key.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_custom_primary_key.conf index ba3e94855fd..1d1c1c80c7e 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_custom_primary_key.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_custom_primary_key.conf @@ -30,7 +30,7 @@ source { result_table_name = "customers_mysql_cdc" server-id = 5652 base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" - username = "mysqluser" + username = "st_user_source" password = "mysqlpw" exactly_once = true table-names = ["mysql_cdc.mysql_cdc_e2e_source_table_1_custom_primary_key", "mysql_cdc.mysql_cdc_e2e_source_table_2_custom_primary_key"] @@ -52,7 +52,7 @@ sink { source_table_name = "customers_mysql_cdc" url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc2" driver = "com.mysql.cj.jdbc.Driver" - user = "mysqluser" + user = "st_user_sink" password = "mysqlpw" database = "mysql_cdc2" generate_sink_sql = true diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_disable_exactly_once.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_disable_exactly_once.conf index 4b91a877d41..fdc4302662f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_disable_exactly_once.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_disable_exactly_once.conf @@ -32,8 +32,8 @@ source { MySQL-CDC { result_table_name = "customers_mysql_cdc" server-id = 5652 - username = "st_user" - password = "seatunnel" + username = "st_user_source" + password = "mysqlpw" table-names = ["mysql_cdc.mysql_cdc_e2e_source_table"] base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" @@ -48,8 +48,8 @@ sink { source_table_name = "customers_mysql_cdc" url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" driver = "com.mysql.cj.jdbc.Driver" - user = "st_user" - password = "seatunnel" + user = "st_user_sink" + password = "mysqlpw" generate_sink_sql = true # You need to configure both database and table diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_multi_table_mode_one_table.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_multi_table_mode_one_table.conf index 180cedcda85..c382c1c5867 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_multi_table_mode_one_table.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_multi_table_mode_one_table.conf @@ -20,7 +20,7 @@ env { # You can set engine configuration here - parallelism = 1 + parallelism = 3 job.mode = "STREAMING" checkpoint.interval = 5000 } @@ -28,11 +28,14 @@ env { source { MySQL-CDC { result_table_name = "customers_mysql_cdc" - server-id = 5652 - username = "mysqluser" + server-id = 5652-5660 + username = "st_user_source" password = "mysqlpw" table-names = ["mysql_cdc.mysql_cdc_e2e_source_table"] base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" + + snapshot.split.size = 1 + snapshot.fetch.size = 1 } } @@ -44,7 +47,7 @@ sink { source_table_name = "customers_mysql_cdc" url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc2" driver = "com.mysql.cj.jdbc.Driver" - user = "mysqluser" + user = "st_user_sink" password = "mysqlpw" database = "mysql_cdc2" generate_sink_sql = true diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_multi_table_mode_two_table.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_multi_table_mode_two_table.conf index f9bc9b1e014..cb10cf26447 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_multi_table_mode_two_table.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_multi_table_mode_two_table.conf @@ -20,7 +20,7 @@ env { # You can set engine configuration here - parallelism = 1 + parallelism = 3 job.mode = "STREAMING" checkpoint.interval = 5000 } @@ -28,11 +28,14 @@ env { source { MySQL-CDC { result_table_name = "customers_mysql_cdc" - server-id = 5652 - username = "mysqluser" + server-id = 5652-5660 + username = "st_user_source" password = "mysqlpw" table-names = ["mysql_cdc.mysql_cdc_e2e_source_table", "mysql_cdc.mysql_cdc_e2e_source_table2"] base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" + + snapshot.split.size = 1 + snapshot.fetch.size = 1 } } @@ -44,7 +47,7 @@ sink { source_table_name = "customers_mysql_cdc" url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc2" driver = "com.mysql.cj.jdbc.Driver" - user = "mysqluser" + user = "st_user_sink" password = "mysqlpw" database = "mysql_cdc2" generate_sink_sql = true diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_no_primary_key.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_no_primary_key.conf index 46df806ae7d..e35403ba233 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_no_primary_key.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_no_primary_key.conf @@ -29,8 +29,8 @@ source { MySQL-CDC { result_table_name = "customers_mysql_cdc" server-id = 5652 - username = "st_user" - password = "seatunnel" + username = "st_user_source" + password = "mysqlpw" table-names = ["mysql_cdc.mysql_cdc_e2e_source_table_no_primary_key"] base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" @@ -43,8 +43,8 @@ sink { source_table_name = "customers_mysql_cdc" url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" driver = "com.mysql.cj.jdbc.Driver" - user = "st_user" - password = "seatunnel" + user = "st_user_sink" + password = "mysqlpw" generate_sink_sql = true # You need to configure both database and table diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/src/test/java/org/apache/seatunnel/e2e/connector/http/HttpIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/src/test/java/org/apache/seatunnel/e2e/connector/http/HttpIT.java index 9dc38cbd1ce..f53d8c1d458 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/src/test/java/org/apache/seatunnel/e2e/connector/http/HttpIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-http-e2e/src/test/java/org/apache/seatunnel/e2e/connector/http/HttpIT.java @@ -75,6 +75,7 @@ public void startUp() { .withEnv( "MOCKSERVER_INITIALIZATION_JSON_PATH", TMP_DIR + getMockServerConfig()) + .withEnv("MOCKSERVER_LOG_LEVEL", "WARN") .withLogConsumer(new Slf4jLogConsumer(DockerLoggerFactory.getLogger(IMAGE))) .waitingFor(new HttpWaitStrategy().forPath("/").forStatusCode(404)); Startables.deepStart(Stream.of(mockserverContainer)).join(); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointBarrier.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointBarrier.java index 7179cc8cb35..1cf1f8bdfd5 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointBarrier.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointBarrier.java @@ -18,27 +18,46 @@ package org.apache.seatunnel.engine.server.checkpoint; import org.apache.seatunnel.engine.core.checkpoint.CheckpointType; +import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.task.record.Barrier; import com.google.common.base.Objects; +import lombok.Getter; import java.io.Serializable; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; import static com.google.common.base.Preconditions.checkNotNull; +@Getter public class CheckpointBarrier implements Barrier, Serializable { private final long id; private final long timestamp; private final CheckpointType checkpointType; + private final Set prepareCloseTasks; + private final Set closedTasks; public CheckpointBarrier(long id, long timestamp, CheckpointType checkpointType) { + this(id, timestamp, checkpointType, Collections.emptySet(), Collections.emptySet()); + } + + public CheckpointBarrier( + long id, + long timestamp, + CheckpointType checkpointType, + Set prepareCloseTasks, + Set closedTasks) { this.id = id; this.timestamp = timestamp; this.checkpointType = checkNotNull(checkpointType); - } - - public long getId() { - return id; + this.prepareCloseTasks = prepareCloseTasks; + this.closedTasks = closedTasks; + if (new HashSet(prepareCloseTasks).removeAll(closedTasks)) { + throw new IllegalArgumentException( + "The prepareCloseTasks collection should not contain elements of the closedTasks collection"); + } } @Override @@ -51,12 +70,17 @@ public boolean prepareClose() { return checkpointType.isFinalCheckpoint(); } - public long getTimestamp() { - return timestamp; + @Override + public boolean prepareClose(TaskLocation task) { + if (prepareClose()) { + return true; + } + return prepareCloseTasks.contains(task); } - public CheckpointType getCheckpointType() { - return checkpointType; + @Override + public Set closedTasks() { + return Collections.unmodifiableSet(closedTasks); } @Override @@ -81,7 +105,8 @@ public boolean equals(Object other) { @Override public String toString() { return String.format( - "CheckpointBarrier %d @ %d Options: %s", id, timestamp, checkpointType); + "CheckpointBarrier %d @ %d type: %s, prepareClose: %s, closed: %s", + id, timestamp, checkpointType, prepareCloseTasks, closedTasks); } public boolean isAuto() { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java index 47392f7a317..12f5acd597d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java @@ -55,6 +55,7 @@ import java.time.Instant; import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -111,6 +112,8 @@ public class CheckpointCoordinator { private final CheckpointPlan plan; private final Set readyToCloseStartingTask; + private final Set readyToCloseIdleTask; + @Getter private final Set closedIdleTask; private final ConcurrentHashMap pendingCheckpoints; private final ArrayDeque completedCheckpointIds; @@ -189,6 +192,8 @@ public CheckpointCoordinator( this.pipelineTaskStatus = new ConcurrentHashMap<>(); this.checkpointIdCounter = checkpointIdCounter; this.readyToCloseStartingTask = new CopyOnWriteArraySet<>(); + this.readyToCloseIdleTask = new CopyOnWriteArraySet<>(); + this.closedIdleTask = new CopyOnWriteArraySet<>(); LOG.info( "Create CheckpointCoordinator for job({}@{}) with plan({})", @@ -309,7 +314,11 @@ private void restoreTaskState(TaskLocation taskLocation) { for (int i = tuple.f1(); i < actionState.getParallelism(); i += currentParallelism) { - states.add(actionState.getSubtaskStates().get(i)); + ActionSubtaskState subtaskState = + actionState.getSubtaskStates().get(i); + if (subtaskState != null) { + states.add(subtaskState); + } } }); } @@ -397,6 +406,64 @@ protected void readyToClose(TaskLocation taskLocation) { } } + protected void readyToCloseIdleTask(TaskLocation taskLocation) { + if (plan.getStartingSubtasks().contains(taskLocation)) { + throw new UnsupportedOperationException("Unsupported close starting task"); + } + + LOG.info( + "Received close idle task[{}]({}/{}). {}", + taskLocation.getTaskID(), + taskLocation.getPipelineId(), + taskLocation.getJobId(), + taskLocation); + synchronized (readyToCloseIdleTask) { + if (readyToCloseIdleTask.contains(taskLocation) + || closedIdleTask.contains(taskLocation)) { + LOG.warn( + "task[{}]({}/{}) already in closed. {}", + taskLocation.getTaskID(), + taskLocation.getPipelineId(), + taskLocation.getJobId(), + taskLocation); + return; + } + + List subTaskList = new ArrayList<>(); + for (TaskLocation subTask : plan.getPipelineSubtasks()) { + if (subTask.getTaskGroupLocation().equals(taskLocation.getTaskGroupLocation())) { + // close all subtask in the same task group + subTaskList.add(subTask); + LOG.info( + "Add task[{}]({}/{}) to prepare close list", + subTask.getTaskID(), + subTask.getPipelineId(), + subTask.getJobId()); + } + } + if (subTaskList.size() != 2) { + throw new UnsupportedOperationException( + "Unsupported close not reader/writer task group: " + subTaskList); + } + readyToCloseIdleTask.addAll(subTaskList); + tryTriggerPendingCheckpoint(CheckpointType.CHECKPOINT_TYPE); + } + } + + protected void completedCloseIdleTask(TaskLocation taskLocation) { + synchronized (readyToCloseIdleTask) { + if (readyToCloseIdleTask.contains(taskLocation)) { + readyToCloseIdleTask.remove(taskLocation); + closedIdleTask.add(taskLocation); + LOG.info( + "Completed close task[{}]({}/{})", + taskLocation.getTaskID(), + taskLocation.getPipelineId(), + taskLocation.getJobId()); + } + } + } + protected void restoreCoordinator(boolean alreadyStarted) { LOG.info("received restore CheckpointCoordinator with alreadyStarted= " + alreadyStarted); errorByPhysicalVertex = new AtomicReference<>(); @@ -553,7 +620,9 @@ private void startTriggerPendingCheckpoint( pendingCheckpoint.getCheckpointId(), pendingCheckpoint .getCheckpointTimestamp(), - pendingCheckpoint.getCheckpointType()), + pendingCheckpoint.getCheckpointType(), + new HashSet<>(readyToCloseIdleTask), + new HashSet<>(closedIdleTask)), executorService) .thenApplyAsync(this::triggerCheckpoint, executorService); @@ -664,8 +733,8 @@ private CompletableFuture triggerPendingCheckpoint( } private Set getNotYetAcknowledgedTasks() { - // TODO: some tasks have completed and don't need to be ack return plan.getPipelineSubtasks().stream() + .filter(e -> !closedIdleTask.contains(e)) .map(TaskLocation::getTaskID) .collect(Collectors.toCollection(CopyOnWriteArraySet::new)); } @@ -715,6 +784,8 @@ protected void cleanPendingCheckpoint(CheckpointCloseReason closedReason) { } pipelineTaskStatus.clear(); readyToCloseStartingTask.clear(); + readyToCloseIdleTask.clear(); + closedIdleTask.clear(); pendingCounter.set(0); schemaChanging.set(false); scheduler.shutdownNow(); @@ -752,6 +823,11 @@ protected void acknowledgeTask(TaskAcknowledgeOperation ackOperation) { pendingCheckpoint.getCheckpointType().isSavepoint() ? SubtaskStatus.SAVEPOINT_PREPARE_CLOSE : SubtaskStatus.RUNNING); + + if (ackOperation.getBarrier().getCheckpointType().notFinalCheckpoint() + && ackOperation.getBarrier().prepareClose(location)) { + completedCloseIdleTask(location); + } } public synchronized void completePendingCheckpoint(CompletedCheckpoint completedCheckpoint) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java index 9ef2e6623be..21c2b90df57 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java @@ -216,6 +216,15 @@ public void readyToClose(TaskLocation taskLocation) { getCheckpointCoordinator(taskLocation).readyToClose(taskLocation); } + /** + * Called by the {@link SourceSplitEnumeratorTask}.
+ * used by SourceSplitEnumeratorTask to tell CheckpointCoordinator pipeline will trigger close + * barrier of idle task by SourceSplitEnumeratorTask. + */ + public void readyToCloseIdleTask(TaskLocation taskLocation) { + getCheckpointCoordinator(taskLocation).readyToCloseIdleTask(taskLocation); + } + /** * Called by the JobMaster.
* Listen to the {@link PipelineStatus} of the {@link Pipeline}, which is used to shut down the diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalVertex.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalVertex.java index f16b0cc6e4e..4fbcfa4fa3f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalVertex.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalVertex.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.engine.server.dag.physical; +import org.apache.seatunnel.shade.com.google.common.annotations.VisibleForTesting; + import org.apache.seatunnel.common.utils.ExceptionUtils; import org.apache.seatunnel.common.utils.RetryUtils; import org.apache.seatunnel.engine.common.Constant; @@ -31,6 +33,7 @@ import org.apache.seatunnel.engine.server.execution.ExecutionState; import org.apache.seatunnel.engine.server.execution.TaskDeployState; import org.apache.seatunnel.engine.server.execution.TaskExecutionState; +import org.apache.seatunnel.engine.server.execution.TaskGroup; import org.apache.seatunnel.engine.server.execution.TaskGroupDefaultImpl; import org.apache.seatunnel.engine.server.execution.TaskGroupLocation; import org.apache.seatunnel.engine.server.master.JobMaster; @@ -350,6 +353,11 @@ private TaskGroupImmutableInformation getTaskGroupImmutableInformation() { this.connectorJarIdentifiers); } + @VisibleForTesting + public TaskGroup getTaskGroup() { + return taskGroup; + } + public synchronized void updateTaskState(@NonNull ExecutionState targetState) { try { ExecutionState current = (ExecutionState) runningJobStateIMap.get(taskGroupLocation); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java index 01b2af48a89..ece0c18c93f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java @@ -389,6 +389,46 @@ public JobDAGInfo getJobDAGInfo() { return jobDAGInfo; } + public void releaseTaskGroupResource( + PipelineLocation pipelineLocation, TaskGroupLocation taskGroupLocation) { + Map taskGroupLocationSlotProfileMap = + ownedSlotProfilesIMap.get(pipelineLocation); + if (taskGroupLocationSlotProfileMap == null) { + return; + } + SlotProfile taskGroupSlotProfile = taskGroupLocationSlotProfileMap.get(taskGroupLocation); + if (taskGroupSlotProfile == null) { + return; + } + + try { + RetryUtils.retryWithException( + () -> { + LOGGER.info( + String.format( + "release the task group resource %s", taskGroupLocation)); + + resourceManager + .releaseResources( + jobImmutableInformation.getJobId(), + Collections.singletonList(taskGroupSlotProfile)) + .join(); + + return null; + }, + new RetryUtils.RetryMaterial( + Constant.OPERATION_RETRY_TIME, + true, + exception -> ExceptionUtil.isOperationNeedRetryException(exception), + Constant.OPERATION_RETRY_SLEEP)); + } catch (Exception e) { + LOGGER.warning( + String.format( + "release the task group resource failed %s, with exception: %s ", + taskGroupLocation, ExceptionUtils.getMessage(e))); + } + } + public void releasePipelineResource(SubPlan subPlan) { try { Map taskGroupLocationSlotProfileMap = @@ -663,6 +703,13 @@ public void updateTaskExecutionState(TaskExecutionState taskExecutionState) { task.updateStateByExecutionService( taskExecutionState); + if (taskExecutionState + .getExecutionState() + .isEndState()) { + releaseTaskGroupResource( + pipeline.getPipelineLocation(), + task.getTaskGroupLocation()); + } }); }); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/serializable/RecordSerializer.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/serializable/RecordSerializer.java index 90caab3c188..975810693d9 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/serializable/RecordSerializer.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/serializable/RecordSerializer.java @@ -45,6 +45,8 @@ public void write(ObjectDataOutput out, Record record) throws IOException { out.writeLong(checkpointBarrier.getId()); out.writeLong(checkpointBarrier.getTimestamp()); out.writeString(checkpointBarrier.getCheckpointType().getName()); + out.writeObject(checkpointBarrier.getPrepareCloseTasks()); + out.writeObject(checkpointBarrier.getClosedTasks()); } else if (data instanceof SeaTunnelRow) { SeaTunnelRow row = (SeaTunnelRow) data; out.writeByte(RecordDataType.SEATUNNEL_ROW.ordinal()); @@ -67,7 +69,11 @@ public Record read(ObjectDataInput in) throws IOException { if (dataType == RecordDataType.CHECKPOINT_BARRIER.ordinal()) { data = new CheckpointBarrier( - in.readLong(), in.readLong(), CheckpointType.fromName(in.readString())); + in.readLong(), + in.readLong(), + CheckpointType.fromName(in.readString()), + in.readObject(), + in.readObject()); } else if (dataType == RecordDataType.SEATUNNEL_ROW.ordinal()) { String tableId = in.readString(); byte rowKind = in.readByte(); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/serializable/TaskDataSerializerHook.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/serializable/TaskDataSerializerHook.java index 7c298272c7d..a9267675761 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/serializable/TaskDataSerializerHook.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/serializable/TaskDataSerializerHook.java @@ -37,6 +37,7 @@ import org.apache.seatunnel.engine.server.task.operation.sink.SinkPrepareCommitOperation; import org.apache.seatunnel.engine.server.task.operation.sink.SinkRegisterOperation; import org.apache.seatunnel.engine.server.task.operation.source.AssignSplitOperation; +import org.apache.seatunnel.engine.server.task.operation.source.CloseIdleReaderOperation; import org.apache.seatunnel.engine.server.task.operation.source.LastCheckpointNotifyOperation; import org.apache.seatunnel.engine.server.task.operation.source.RequestSplitOperation; import org.apache.seatunnel.engine.server.task.operation.source.RestoredSplitOperation; @@ -101,6 +102,8 @@ public class TaskDataSerializerHook implements DataSerializerHook { public static final int REPORT_JOB_EVENT = 25; + public static final int CLOSE_READER_OPERATION = 26; + public static final int FACTORY_ID = FactoryIdHelper.getFactoryId( SeaTunnelFactoryIdConstant.SEATUNNEL_TASK_DATA_SERIALIZER_FACTORY, @@ -171,6 +174,8 @@ public IdentifiedDataSerializable create(int typeId) { return new DeleteConnectorJarInExecutionNode(); case REPORT_JOB_EVENT: return new JobEventReportOperation(); + case CLOSE_READER_OPERATION: + return new CloseIdleReaderOperation(); default: throw new IllegalArgumentException("Unknown type id " + typeId); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java index 3ee636bbc58..d66921c7423 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java @@ -339,7 +339,8 @@ public void ack(Barrier barrier) { Integer ackSize = cycleAcks.compute(barrier.getId(), (id, count) -> count == null ? 1 : ++count); if (ackSize == allCycles.size()) { - if (barrier.prepareClose()) { + cycleAcks.remove(barrier.getId()); + if (barrier.prepareClose(this.taskLocation)) { this.prepareCloseStatus = true; this.prepareCloseBarrierId.set(barrier.getId()); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SinkAggregatedCommitterTask.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SinkAggregatedCommitterTask.java index 48c3ca197d2..34a14cf5850 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SinkAggregatedCommitterTask.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SinkAggregatedCommitterTask.java @@ -216,6 +216,12 @@ public void close() throws IOException { } } + private long getClosedWriters(Barrier barrier) { + return barrier.closedTasks().stream() + .filter(task -> writerAddressMap.containsKey(task.getTaskID())) + .count(); + } + @Override public void triggerBarrier(Barrier barrier) throws Exception { long startTime = System.currentTimeMillis(); @@ -224,10 +230,11 @@ public void triggerBarrier(Barrier barrier) throws Exception { Integer count = checkpointBarrierCounter.compute( barrier.getId(), (id, num) -> num == null ? 1 : ++num); - if (count != maxWriterSize) { + + if (count != (maxWriterSize - getClosedWriters(barrier))) { return; } - if (barrier.prepareClose()) { + if (barrier.prepareClose(this.taskLocation)) { this.prepareCloseStatus = true; this.prepareCloseBarrierId.set(barrier.getId()); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSplitEnumeratorTask.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSplitEnumeratorTask.java index e95684c1c51..2672dab382e 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSplitEnumeratorTask.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSplitEnumeratorTask.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.engine.server.task; import org.apache.seatunnel.api.serialization.Serializer; +import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SourceEvent; import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.source.SourceSplitEnumerator; @@ -32,12 +33,12 @@ import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.task.context.SeaTunnelSplitEnumeratorContext; import org.apache.seatunnel.engine.server.task.operation.checkpoint.BarrierFlowOperation; +import org.apache.seatunnel.engine.server.task.operation.source.CloseIdleReaderOperation; import org.apache.seatunnel.engine.server.task.operation.source.LastCheckpointNotifyOperation; import org.apache.seatunnel.engine.server.task.record.Barrier; import org.apache.seatunnel.engine.server.task.statemachine.SeaTunnelTaskState; import com.hazelcast.cluster.Address; -import com.hazelcast.spi.impl.operationservice.Operation; import com.hazelcast.spi.impl.operationservice.impl.InvocationFuture; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; @@ -57,7 +58,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArraySet; import java.util.concurrent.ExecutionException; -import java.util.function.Function; import java.util.stream.Collectors; import static org.apache.seatunnel.engine.common.utils.ExceptionUtil.sneaky; @@ -144,7 +144,7 @@ public void triggerBarrier(Barrier barrier) throws Exception { long startTime = System.currentTimeMillis(); log.debug("split enumer trigger barrier [{}]", barrier); - if (barrier.prepareClose()) { + if (barrier.prepareClose(this.taskLocation)) { this.prepareCloseTriggered = true; this.prepareCloseBarrierId.set(barrier.getId()); } @@ -158,7 +158,7 @@ public void triggerBarrier(Barrier barrier) throws Exception { serialize = enumeratorStateSerializer.serialize(snapshotState); } log.debug("source split enumerator send state [{}] to master", snapshotState); - sendToAllReader(location -> new BarrierFlowOperation(barrier, location)); + sendToActiveReader(barrier); } if (barrier.snapshot()) { this.getExecutionContext() @@ -276,10 +276,18 @@ private SourceSplitEnumerator getEnumerator() return enumerator; } - public void readerFinished(long taskID) { - unfinishedReaders.remove(taskID); + public void readerFinished(TaskLocation taskLocation) { + unfinishedReaders.remove(taskLocation.getTaskID()); if (unfinishedReaders.isEmpty()) { prepareCloseStatus = true; + } else if (Boundedness.UNBOUNDED.equals(this.source.getSource().getBoundedness())) { + log.info( + "Send close idle reader {} operation of unbounded job. {}", + taskLocation.getTaskIndex(), + taskLocation); + this.getExecutionContext() + .sendToMaster(new CloseIdleReaderOperation(jobID, taskLocation)) + .join(); } } @@ -348,10 +356,13 @@ public Set getRegisteredReaders() { .collect(Collectors.toSet()); } - private void sendToAllReader(Function function) { + private void sendToActiveReader(Barrier barrier) { List> futures = new ArrayList<>(); taskMemberMapping.forEach( (location, address) -> { + if (barrier.closedTasks().contains(location)) { + return; + } log.debug( "split enumerator send to read--size: {}, location: {}, address: {}", taskMemberMapping.size(), @@ -359,7 +370,8 @@ private void sendToAllReader(Function function) { address.toString()); futures.add( this.getExecutionContext() - .sendToMember(function.apply(location), address)); + .sendToMember( + new BarrierFlowOperation(barrier, location), address)); }); futures.forEach(InvocationFuture::join); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/ShuffleSinkFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/ShuffleSinkFlowLifeCycle.java index 32ec5cb8f4d..9bdbd37f80a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/ShuffleSinkFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/ShuffleSinkFlowLifeCycle.java @@ -78,7 +78,7 @@ public void received(Record record) throws IOException { shuffleFlush(); Barrier barrier = (Barrier) record.getData(); - if (barrier.prepareClose()) { + if (barrier.prepareClose(runningTask.getTaskLocation())) { prepareClose = true; } if (barrier.snapshot()) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/ShuffleSourceFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/ShuffleSourceFlowLifeCycle.java index 2f14c677011..87b29eab21d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/ShuffleSourceFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/ShuffleSourceFlowLifeCycle.java @@ -107,7 +107,7 @@ public void collect(Collector> collector) throws Exception { // publish barrier if (alignedBarriersCounter == shuffles.length) { - if (barrier.prepareClose()) { + if (barrier.prepareClose(runningTask.getTaskLocation())) { prepareClose = true; } if (barrier.snapshot()) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java index 1a2143b0271..48c530a0c36 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java @@ -183,7 +183,7 @@ public void received(Record record) { long startTime = System.currentTimeMillis(); Barrier barrier = (Barrier) record.getData(); - if (barrier.prepareClose()) { + if (barrier.prepareClose(this.taskLocation)) { prepareClose = true; } if (barrier.snapshot()) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java index 64e5bfd22b8..83675575b4b 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java @@ -268,7 +268,7 @@ public void triggerBarrier(Barrier barrier) throws Exception { // Block the reader from adding barrier to the collector. synchronized (collector.getCheckpointLock()) { - if (barrier.prepareClose()) { + if (barrier.prepareClose(this.currentTaskLocation)) { this.prepareClose = true; } if (barrier.snapshot()) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index 187aa3659be..0447513b5ff 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -75,7 +75,7 @@ public void open() throws Exception { public void received(Record record) { if (record.getData() instanceof Barrier) { CheckpointBarrier barrier = (CheckpointBarrier) record.getData(); - if (barrier.prepareClose()) { + if (barrier.prepareClose(this.runningTask.getTaskLocation())) { prepareClose = true; } if (barrier.snapshot()) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/group/queue/IntermediateBlockingQueue.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/group/queue/IntermediateBlockingQueue.java index f7cd7b0c234..b8e53faabd4 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/group/queue/IntermediateBlockingQueue.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/group/queue/IntermediateBlockingQueue.java @@ -64,7 +64,7 @@ private void handleRecord(Record record, ConsumerWithException> con if (record.getData() instanceof Barrier) { CheckpointBarrier barrier = (CheckpointBarrier) record.getData(); getRunningTask().ack(barrier); - if (barrier.prepareClose()) { + if (barrier.prepareClose(this.getRunningTask().getTaskLocation())) { getIntermediateQueueFlowLifeCycle().setPrepareClose(true); } consumer.accept(record); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/group/queue/disruptor/RecordEventHandler.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/group/queue/disruptor/RecordEventHandler.java index d525a0b257b..6cc5195de56 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/group/queue/disruptor/RecordEventHandler.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/group/queue/disruptor/RecordEventHandler.java @@ -54,7 +54,7 @@ private void handleRecord(Record record, Collector> collector) thro if (record.getData() instanceof Barrier) { CheckpointBarrier barrier = (CheckpointBarrier) record.getData(); runningTask.ack(barrier); - if (barrier.prepareClose()) { + if (barrier.prepareClose(this.runningTask.getTaskLocation())) { this.intermediateQueueFlowLifeCycle.setPrepareClose(true); } } else { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/group/queue/disruptor/RecordEventProducer.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/group/queue/disruptor/RecordEventProducer.java index 021bb8d2f08..ea47f83a797 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/group/queue/disruptor/RecordEventProducer.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/group/queue/disruptor/RecordEventProducer.java @@ -34,7 +34,8 @@ public static void onData( if (record.getData() instanceof Barrier) { CheckpointBarrier barrier = (CheckpointBarrier) record.getData(); intermediateQueueFlowLifeCycle.getRunningTask().ack(barrier); - if (barrier.prepareClose()) { + if (barrier.prepareClose( + intermediateQueueFlowLifeCycle.getRunningTask().getTaskLocation())) { intermediateQueueFlowLifeCycle.setPrepareClose(true); } } else { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/CloseIdleReaderOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/CloseIdleReaderOperation.java new file mode 100644 index 00000000000..abedf1a4993 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/CloseIdleReaderOperation.java @@ -0,0 +1,72 @@ +/* + * 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.task.operation.source; + +import org.apache.seatunnel.engine.server.SeaTunnelServer; +import org.apache.seatunnel.engine.server.execution.TaskLocation; +import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; + +import com.hazelcast.nio.ObjectDataInput; +import com.hazelcast.nio.ObjectDataOutput; +import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; + +import java.io.IOException; + +public class CloseIdleReaderOperation extends Operation implements IdentifiedDataSerializable { + private long jobId; + private TaskLocation taskLocation; + + public CloseIdleReaderOperation() {} + + public CloseIdleReaderOperation(long jobId, TaskLocation taskLocation) { + this.jobId = jobId; + this.taskLocation = taskLocation; + } + + @Override + public void run() throws Exception { + SeaTunnelServer server = getService(); + server.getCoordinatorService() + .getJobMaster(jobId) + .getCheckpointManager() + .readyToCloseIdleTask(taskLocation); + } + + @Override + protected void writeInternal(ObjectDataOutput out) throws IOException { + out.writeLong(jobId); + out.writeObject(taskLocation); + } + + @Override + protected void readInternal(ObjectDataInput in) throws IOException { + jobId = in.readLong(); + taskLocation = in.readObject(); + } + + @Override + public int getFactoryId() { + return TaskDataSerializerHook.FACTORY_ID; + } + + @Override + public int getClassId() { + return TaskDataSerializerHook.CLOSE_READER_OPERATION; + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceNoMoreElementOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceNoMoreElementOperation.java index fec6afcb91e..60b67c0c83b 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceNoMoreElementOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceNoMoreElementOperation.java @@ -57,7 +57,7 @@ public void run() throws Exception { Thread.currentThread().setContextClassLoader(classLoader); SourceSplitEnumeratorTask task = server.getTaskExecutionService().getTask(enumeratorTaskID); - task.readerFinished(currentTaskID.getTaskID()); + task.readerFinished(currentTaskID); Thread.currentThread().setContextClassLoader(oldClassLoader); return null; }, diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/record/Barrier.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/record/Barrier.java index 67042247b62..4350c633727 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/record/Barrier.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/record/Barrier.java @@ -17,6 +17,10 @@ package org.apache.seatunnel.engine.server.task.record; +import org.apache.seatunnel.engine.server.execution.TaskLocation; + +import java.util.Set; + /** barrier flowing in data flow */ public interface Barrier { Long PREPARE_CLOSE_BARRIER_ID = Long.MAX_VALUE; @@ -32,4 +36,21 @@ public interface Barrier { /** Barrier indicating that the task should prepare to close. */ boolean prepareClose(); + + /** + * Barrier indicating that the task should prepare to close. + * + * @param task task location + * @return If the task is included, the return true + */ + default boolean prepareClose(TaskLocation task) { + return prepareClose(); + } + + /** + * Indicates a list of tasks that have been closed. + * + * @return + */ + Set closedTasks(); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/master/JobMasterTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/master/JobMasterTest.java index dd886d923e5..cbfde91b370 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/master/JobMasterTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/master/JobMasterTest.java @@ -28,9 +28,15 @@ import org.apache.seatunnel.engine.server.AbstractSeaTunnelServerTest; import org.apache.seatunnel.engine.server.TestUtils; import org.apache.seatunnel.engine.server.checkpoint.CheckpointCloseReason; +import org.apache.seatunnel.engine.server.checkpoint.CheckpointCoordinator; +import org.apache.seatunnel.engine.server.dag.physical.PhysicalVertex; import org.apache.seatunnel.engine.server.dag.physical.PipelineLocation; +import org.apache.seatunnel.engine.server.dag.physical.SubPlan; import org.apache.seatunnel.engine.server.execution.TaskGroupLocation; import org.apache.seatunnel.engine.server.resourcemanager.resource.SlotProfile; +import org.apache.seatunnel.engine.server.service.slot.SlotService; +import org.apache.seatunnel.engine.server.task.CoordinatorTask; +import org.apache.seatunnel.engine.server.task.SeaTunnelTask; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; @@ -219,7 +225,70 @@ public void testCommitFailedWillRestore() throws Exception { Assertions.assertTrue(jobMaster.isNeedRestore()); } + @Test + public void testCloseIdleTask() throws InterruptedException { + long jobId = instance.getFlakeIdGenerator(Constant.SEATUNNEL_ID_GENERATOR_NAME).newId(); + JobMaster jobMaster = newJobInstanceWithRunningState(jobId); + Assertions.assertEquals(JobStatus.RUNNING, jobMaster.getJobStatus()); + + assertCloseIdleTask(jobMaster); + + server.getCoordinatorService().savePoint(jobId); + server.getCoordinatorService().getJobStatus(jobId); + await().atMost(60, TimeUnit.SECONDS) + .untilAsserted( + () -> { + JobStatus jobStatus = + server.getCoordinatorService().getJobStatus(jobId); + Assertions.assertEquals(JobStatus.SAVEPOINT_DONE, jobStatus); + }); + jobMaster = newJobInstanceWithRunningState(jobId, true); + Assertions.assertEquals(JobStatus.RUNNING, jobMaster.getJobStatus()); + + assertCloseIdleTask(jobMaster); + } + + private void assertCloseIdleTask(JobMaster jobMaster) { + SlotService slotService = server.getSlotService(); + Assertions.assertEquals(4, slotService.getWorkerProfile().getAssignedSlots().length); + + Assertions.assertEquals(1, jobMaster.getPhysicalPlan().getPipelineList().size()); + SubPlan subPlan = jobMaster.getPhysicalPlan().getPipelineList().get(0); + try { + PhysicalVertex coordinatorVertex1 = subPlan.getCoordinatorVertexList().get(0); + CoordinatorTask coordinatorTask = + (CoordinatorTask) + coordinatorVertex1.getTaskGroup().getTasks().stream().findFirst().get(); + jobMaster + .getCheckpointManager() + .readyToCloseIdleTask(coordinatorTask.getTaskLocation()); + Assertions.fail("should throw UnsupportedOperationException"); + } catch (UnsupportedOperationException e) { + // ignore + } + + Assertions.assertEquals(2, subPlan.getPhysicalVertexList().size()); + PhysicalVertex taskGroup1 = subPlan.getPhysicalVertexList().get(0); + SeaTunnelTask seaTunnelTask = + (SeaTunnelTask) taskGroup1.getTaskGroup().getTasks().stream().findFirst().get(); + jobMaster.getCheckpointManager().readyToCloseIdleTask(seaTunnelTask.getTaskLocation()); + + CheckpointCoordinator checkpointCoordinator = + jobMaster + .getCheckpointManager() + .getCheckpointCoordinator(seaTunnelTask.getTaskLocation().getPipelineId()); + await().atMost(60, TimeUnit.SECONDS) + .until(() -> checkpointCoordinator.getClosedIdleTask().size() == 2); + await().atMost(60, TimeUnit.SECONDS) + .until(() -> slotService.getWorkerProfile().getAssignedSlots().length == 3); + } + private JobMaster newJobInstanceWithRunningState(long jobId) throws InterruptedException { + return newJobInstanceWithRunningState(jobId, false); + } + + private JobMaster newJobInstanceWithRunningState(long jobId, boolean restore) + throws InterruptedException { LogicalDag testLogicalDag = TestUtils.createTestLogicalPlan( "stream_fakesource_to_file.conf", "test_clear_coordinator_service", jobId); @@ -228,6 +297,7 @@ private JobMaster newJobInstanceWithRunningState(long jobId) throws InterruptedE new JobImmutableInformation( jobId, "Test", + restore, nodeEngine.getSerializationService().toData(testLogicalDag), testLogicalDag.getJobConfig(), Collections.emptyList(), diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fakesource_to_file.conf b/seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fakesource_to_file.conf index 7a2b07a2a37..2cbcf14bd9a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fakesource_to_file.conf +++ b/seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fakesource_to_file.conf @@ -19,7 +19,7 @@ ###### env { - parallelism = 1 + parallelism = 2 job.mode = "STREAMING" checkpoint.interval = 5000 } @@ -34,7 +34,6 @@ source { age = "int" } } - parallelism = 1 } } From f108a5e658b261aa049e688264dcc50787611761 Mon Sep 17 00:00:00 2001 From: TaoZex <45089228+TaoZex@users.noreply.github.com> Date: Wed, 15 May 2024 15:48:08 +0800 Subject: [PATCH 09/14] [Connector]Add hbase source connector (#6348) --- .../Error-Quick-Reference-Manual.md | 6 + docs/en/connector-v2/source/Hbase.md | 91 ++++++++ plugin-mapping.properties | 1 + .../seatunnel/hbase/config/HbaseConfig.java | 6 + .../hbase/config/HbaseParameters.java | 18 ++ .../exception/HbaseConnectorErrorCode.java | 43 ++++ .../format/HBaseDeserializationFormat.java | 93 +++++++++ .../seatunnel/hbase/source/HbaseSource.java | 114 ++++++++++ .../hbase/source/HbaseSourceFactory.java | 64 ++++++ .../hbase/source/HbaseSourceReader.java | 195 ++++++++++++++++++ .../hbase/source/HbaseSourceSplit.java | 56 +++++ .../source/HbaseSourceSplitEnumerator.java | 185 +++++++++++++++++ .../hbase/source/HbaseSourceState.java | 36 ++++ .../hbase/utils/HbaseConnectionUtil.java | 48 +++++ .../e2e/connector/hbase/HbaseIT.java | 6 + .../src/test/resources/hbase-to-assert.conf | 119 +++++++++++ 16 files changed, 1081 insertions(+) create mode 100644 docs/en/connector-v2/source/Hbase.md create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/exception/HbaseConnectorErrorCode.java create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/format/HBaseDeserializationFormat.java create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSource.java create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceFactory.java create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceReader.java create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceSplit.java create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceSplitEnumerator.java create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceState.java create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/utils/HbaseConnectionUtil.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-to-assert.conf diff --git a/docs/en/connector-v2/Error-Quick-Reference-Manual.md b/docs/en/connector-v2/Error-Quick-Reference-Manual.md index 2519bfbe655..960bddc0ebd 100644 --- a/docs/en/connector-v2/Error-Quick-Reference-Manual.md +++ b/docs/en/connector-v2/Error-Quick-Reference-Manual.md @@ -256,6 +256,12 @@ problems encountered by users. |--------------|-------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------| | FIRESTORE-01 | Close Firestore client failed | When users encounter this error code, it is usually there are some problems with closing the Firestore client, please check the Firestore is work | +## Hbase Connector Error Codes + +| code | description | solution | +|----------|-------------------------------|---------------------------------------------------------------------------------------------------------------------------------| +| Hbase-01 | Build hbase connection failed | When users create Hbase database connection, the connection failed. Check the Hbase configuration parameters used and try again | + ## FilterFieldTransform Error Codes | code | description | solution | diff --git a/docs/en/connector-v2/source/Hbase.md b/docs/en/connector-v2/source/Hbase.md new file mode 100644 index 00000000000..677b827fb29 --- /dev/null +++ b/docs/en/connector-v2/source/Hbase.md @@ -0,0 +1,91 @@ +# Hbase + +> Hbase source connector + +## Description + +Read data from Apache Hbase. + +## Key features + +- [x] [batch](../../concept/connector-v2-features.md) +- [ ] [stream](../../concept/connector-v2-features.md) +- [ ] [exactly-once](../../concept/connector-v2-features.md) +- [x] [schema projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [ ] [support user-defined split](../../concept/connector-v2-features.md) + +## Options + +| name | type | required | default value | +|--------------------|--------|----------|---------------| +| zookeeper_quorum | string | yes | - | +| table | string | yes | - | +| query_columns | list | yes | - | +| schema | config | yes | - | +| hbase_extra_config | string | no | - | +| common-options | | no | - | + +### zookeeper_quorum [string] + +The zookeeper cluster host of hbase, example: "hadoop001:2181,hadoop002:2181,hadoop003:2181" + +### table [string] + +The table name you want to write, example: "seatunnel" + +### query_columns [list] + +The column name which you want to query in the table. If you want to query the rowkey column, please set "rowkey" in query_columns. +Other column format should be: columnFamily:columnName, example: ["rowkey", "columnFamily1:column1", "columnFamily1:column1", "columnFamily2:column1"] + +### schema [config] + +Hbase uses byte arrays for storage. Therefore, you need to configure data types for each column in a table. For more information, see: [guide](../../concept/schema-feature.md#how-to-declare-type-supported). + +### hbase_extra_config [config] + +The extra configuration of hbase + +### common options + +Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details + +## Examples + +```bash +source { + Hbase { + zookeeper_quorum = "hadoop001:2181,hadoop002:2181,hadoop003:2181" + table = "seatunnel_test" + query_columns=["rowkey", "columnFamily1:column1", "columnFamily1:column1", "columnFamily2:column1"] + schema = { + columns = [ + { + name = rowkey + type = string + }, + { + name = "columnFamily1:column1" + type = boolean + }, + { + name = "columnFamily1:column1" + type = double + }, + { + name = "columnFamily2:column1" + type = bigint + } + ] + } + } +} +``` + +## Changelog + +### next version + +- Add Hbase Source Connector + diff --git a/plugin-mapping.properties b/plugin-mapping.properties index c880a8fdf22..314d453ffc7 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -110,6 +110,7 @@ seatunnel.source.TDengine = connector-tdengine seatunnel.sink.TDengine = connector-tdengine seatunnel.source.Persistiq = connector-http-persistiq seatunnel.sink.SelectDBCloud = connector-selectdb-cloud +seatunnel.source.Hbase = connector-hbase seatunnel.sink.Hbase = connector-hbase seatunnel.source.StarRocks = connector-starrocks seatunnel.source.Rocketmq = connector-rocketmq diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseConfig.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseConfig.java index 605a5291fc5..565f1b4b48e 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseConfig.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseConfig.java @@ -42,6 +42,12 @@ public class HbaseConfig { .noDefaultValue() .withDescription("Hbase rowkey column"); + public static final Option> QUERY_COLUMNS = + Options.key("query_columns") + .listType() + .noDefaultValue() + .withDescription("query Hbase columns"); + public static final Option ROWKEY_DELIMITER = Options.key("rowkey_delimiter") .stringType() diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseParameters.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseParameters.java index fe32301f6b5..858030fe2ac 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseParameters.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/config/HbaseParameters.java @@ -32,6 +32,7 @@ import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.FAMILY_NAME; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.HBASE_EXTRA_CONFIG; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.NULL_MODE; +import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.QUERY_COLUMNS; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ROWKEY_COLUMNS; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ROWKEY_DELIMITER; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.TABLE; @@ -50,6 +51,8 @@ public class HbaseParameters implements Serializable { private List rowkeyColumns; + private List columns; + private Map familyNames; private String versionColumn; @@ -103,4 +106,19 @@ public static HbaseParameters buildWithConfig(Config pluginConfig) { } return builder.build(); } + + public static HbaseParameters buildWithSinkConfig(Config pluginConfig) { + HbaseParametersBuilder builder = HbaseParameters.builder(); + + // required parameters + builder.zookeeperQuorum(pluginConfig.getString(ZOOKEEPER_QUORUM.key())); + builder.table(pluginConfig.getString(TABLE.key())); + builder.columns(pluginConfig.getStringList(QUERY_COLUMNS.key())); + + if (pluginConfig.hasPath(HBASE_EXTRA_CONFIG.key())) { + Config extraConfig = pluginConfig.getConfig(HBASE_EXTRA_CONFIG.key()); + builder.hbaseExtraConfig(TypesafeConfigUtils.configToMap(extraConfig)); + } + return builder.build(); + } } diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/exception/HbaseConnectorErrorCode.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/exception/HbaseConnectorErrorCode.java new file mode 100644 index 00000000000..5717c933b0d --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/exception/HbaseConnectorErrorCode.java @@ -0,0 +1,43 @@ +/* + * 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.hbase.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; + +public enum HbaseConnectorErrorCode implements SeaTunnelErrorCode { + CONNECTION_FAILED("Hbase-01", "Build Hbase connection failed"); + + private final String code; + private final String description; + + HbaseConnectorErrorCode(String code, String description) { + this.code = code; + this.description = description; + } + + @Override + public String getCode() { + return code; + } + + @Override + public String getDescription() { + return description; + } +} diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/format/HBaseDeserializationFormat.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/format/HBaseDeserializationFormat.java new file mode 100644 index 00000000000..8d7a1bcbe1d --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/format/HBaseDeserializationFormat.java @@ -0,0 +1,93 @@ +/* + * 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.hbase.format; + +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.common.utils.DateTimeUtils; +import org.apache.seatunnel.common.utils.DateUtils; +import org.apache.seatunnel.common.utils.TimeUtils; +import org.apache.seatunnel.connectors.seatunnel.hbase.exception.HbaseConnectorException; + +import org.apache.hadoop.hbase.util.Bytes; + +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.format.DateTimeFormatter; + +import static org.apache.seatunnel.common.utils.DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS; + +public class HBaseDeserializationFormat { + + private final DateUtils.Formatter dateFormat = DateUtils.Formatter.YYYY_MM_DD; + private final DateTimeUtils.Formatter datetimeFormat = YYYY_MM_DD_HH_MM_SS; + private final TimeUtils.Formatter timeFormat = TimeUtils.Formatter.HH_MM_SS; + + public SeaTunnelRow deserialize(byte[][] rowCell, SeaTunnelRowType seaTunnelRowType) { + SeaTunnelRow row = new SeaTunnelRow(seaTunnelRowType.getTotalFields()); + for (int i = 0; i < row.getArity(); i++) { + SeaTunnelDataType fieldType = seaTunnelRowType.getFieldType(i); + row.setField(i, deserializeValue(fieldType, rowCell[i])); + } + return row; + } + + private Object deserializeValue(SeaTunnelDataType typeInfo, byte[] cell) { + if (cell == null) { + return null; + } + + switch (typeInfo.getSqlType()) { + case TINYINT: + case SMALLINT: + case INT: + return Bytes.toInt(cell); + case BOOLEAN: + return Bytes.toBoolean(cell); + case BIGINT: + return Bytes.toLong(cell); + case FLOAT: + case DECIMAL: + return Bytes.toFloat(cell); + case DOUBLE: + return Bytes.toDouble(cell); + case BYTES: + return cell; + case DATE: + return LocalDate.parse( + Bytes.toString(cell), DateTimeFormatter.ofPattern(dateFormat.getValue())); + case TIME: + return LocalTime.parse( + Bytes.toString(cell), DateTimeFormatter.ofPattern(timeFormat.getValue())); + case TIMESTAMP: + return LocalDateTime.parse( + Bytes.toString(cell), + DateTimeFormatter.ofPattern(datetimeFormat.getValue())); + case STRING: + return Bytes.toString(cell); + default: + throw new HbaseConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + "Unsupported data type " + typeInfo.getSqlType()); + } + } +} diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSource.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSource.java new file mode 100644 index 00000000000..869e33f6235 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSource.java @@ -0,0 +1,114 @@ +/* + * 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.hbase.source; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.source.SupportColumnProjection; +import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.config.CheckConfigUtil; +import org.apache.seatunnel.common.config.CheckResult; +import org.apache.seatunnel.common.constants.PluginType; +import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; +import org.apache.seatunnel.connectors.seatunnel.hbase.exception.HbaseConnectorException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Lists; + +import java.util.List; + +import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.QUERY_COLUMNS; +import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.TABLE; +import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ZOOKEEPER_QUORUM; + +public class HbaseSource + implements SeaTunnelSource, + SupportParallelism, + SupportColumnProjection { + private static final Logger LOG = LoggerFactory.getLogger(HbaseSource.class); + public static final String PLUGIN_NAME = "Hbase"; + private Config pluginConfig; + private SeaTunnelRowType seaTunnelRowType; + private HbaseParameters hbaseParameters; + + private CatalogTable catalogTable; + + @Override + public String getPluginName() { + return PLUGIN_NAME; + } + + HbaseSource(Config pluginConfig) { + this.pluginConfig = pluginConfig; + CheckResult result = + CheckConfigUtil.checkAllExists( + pluginConfig, ZOOKEEPER_QUORUM.key(), TABLE.key(), QUERY_COLUMNS.key()); + if (!result.isSuccess()) { + throw new HbaseConnectorException( + SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, + String.format( + "PluginName: %s, PluginType: %s, Message: %s", + getPluginName(), PluginType.SOURCE, result.getMsg())); + } + this.hbaseParameters = HbaseParameters.buildWithSinkConfig(pluginConfig); + this.catalogTable = CatalogTableUtil.buildWithConfig(pluginConfig); + this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public List getProducedCatalogTables() { + return Lists.newArrayList(catalogTable); + } + + @Override + public SourceReader createReader( + SourceReader.Context readerContext) throws Exception { + return new HbaseSourceReader(hbaseParameters, readerContext, seaTunnelRowType); + } + + @Override + public SourceSplitEnumerator createEnumerator( + SourceSplitEnumerator.Context enumeratorContext) throws Exception { + return new HbaseSourceSplitEnumerator(enumeratorContext, hbaseParameters); + } + + @Override + public SourceSplitEnumerator restoreEnumerator( + SourceSplitEnumerator.Context enumeratorContext, + HbaseSourceState checkpointState) + throws Exception { + return new HbaseSourceSplitEnumerator(enumeratorContext, hbaseParameters, checkpointState); + } +} diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceFactory.java new file mode 100644 index 00000000000..4eec3e00482 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceFactory.java @@ -0,0 +1,64 @@ +/* + * 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.hbase.source; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.connector.TableSource; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; +import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig; + +import com.google.auto.service.AutoService; + +import java.io.Serializable; + +import static org.apache.seatunnel.connectors.seatunnel.hbase.sink.HbaseSinkFactory.IDENTIFIER; + +@AutoService(Factory.class) +public class HbaseSourceFactory implements TableSourceFactory { + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required(HbaseConfig.ZOOKEEPER_QUORUM) + .required(HbaseConfig.TABLE) + .required(HbaseConfig.QUERY_COLUMNS) + .build(); + } + + @Override + public Class getSourceClass() { + return HbaseSource.class; + } + + @Override + public + TableSource createSource(TableSourceFactoryContext context) { + return () -> + (SeaTunnelSource) + new HbaseSource(context.getOptions().toConfig()); + } +} diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceReader.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceReader.java new file mode 100644 index 00000000000..556374844e9 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceReader.java @@ -0,0 +1,195 @@ +/* + * 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.hbase.source; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; +import org.apache.seatunnel.connectors.seatunnel.hbase.format.HBaseDeserializationFormat; +import org.apache.seatunnel.connectors.seatunnel.hbase.utils.HbaseConnectionUtil; + +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Deque; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedDeque; + +@Slf4j +public class HbaseSourceReader implements SourceReader { + private static final String ROW_KEY = "rowkey"; + private final Deque sourceSplits = new ConcurrentLinkedDeque<>(); + + private final transient Map namesMap; + + private final Set columnFamilies = new LinkedHashSet<>(); + private final SourceReader.Context context; + private final SeaTunnelRowType seaTunnelRowType; + private volatile boolean noMoreSplit = false; + + private HbaseParameters hbaseParameters; + private final List columnNames; + private final transient Connection connection; + + private HBaseDeserializationFormat hbaseDeserializationFormat = + new HBaseDeserializationFormat(); + private ResultScanner currentScanner; + + public HbaseSourceReader( + HbaseParameters hbaseParameters, Context context, SeaTunnelRowType seaTunnelRowType) { + this.hbaseParameters = hbaseParameters; + this.context = context; + this.seaTunnelRowType = seaTunnelRowType; + this.namesMap = Maps.newConcurrentMap(); + + this.columnNames = hbaseParameters.getColumns(); + // Check if input column names are in format: [ columnFamily:column ]. + this.columnNames.stream() + .peek( + column -> + Preconditions.checkArgument( + (column.contains(":") && column.split(":").length == 2) + || this.ROW_KEY.equalsIgnoreCase(column), + "Invalid column names, it should be [ColumnFamily:Column] format")) + .forEach(column -> this.columnFamilies.add(column.split(":")[0])); + + connection = HbaseConnectionUtil.getHbaseConnection(hbaseParameters); + } + + @Override + public void open() throws Exception { + // do nothing + } + + @Override + public void close() throws IOException { + if (this.currentScanner != null) { + try { + this.currentScanner.close(); + } catch (Exception e) { + throw new IOException("Failed to close HBase Scanner.", e); + } + } + if (this.connection != null) { + try { + this.connection.close(); + } catch (Exception e) { + throw new IOException("Failed to close HBase connection.", e); + } + log.info("Current HBase connection is closed."); + } + } + + @Override + public void pollNext(Collector output) throws Exception { + synchronized (output.getCheckpointLock()) { + final HbaseSourceSplit split = sourceSplits.poll(); + if (Objects.nonNull(split)) { + // read logic + if (this.currentScanner == null) { + Scan scan = new Scan(); + scan.withStartRow(split.getStartRow(), true); + scan.withStopRow(split.getEndRow(), true); + this.currentScanner = + this.connection + .getTable(TableName.valueOf(hbaseParameters.getTable())) + .getScanner(scan); + } + for (Result result : currentScanner) { + SeaTunnelRow seaTunnelRow = + hbaseDeserializationFormat.deserialize( + convertRawRow(result), seaTunnelRowType); + output.collect(seaTunnelRow); + } + } else if (noMoreSplit && sourceSplits.isEmpty()) { + // signal to the source that we have reached the end of the data. + log.info("Closed the bounded Hbase source"); + context.signalNoMoreElement(); + } else { + log.warn("Waiting for Hbase split, sleeping 1s"); + Thread.sleep(1000L); + } + } + } + + private byte[][] convertRawRow(Result result) { + String[] fieldNames = seaTunnelRowType.getFieldNames(); + byte[][] rawRow = new byte[fieldNames.length][]; + for (int i = 0; i < fieldNames.length; ++i) { + String columnName = fieldNames[i]; + byte[] bytes; + try { + // handle rowkey column + if (this.ROW_KEY.equals(columnName)) { + bytes = result.getRow(); + } else { + byte[][] arr = this.namesMap.get(columnName); + // Deduplicate + if (Objects.isNull(arr)) { + arr = new byte[2][]; + String[] arr1 = columnName.split(":"); + arr[0] = arr1[0].trim().getBytes(StandardCharsets.UTF_8); + arr[1] = arr1[1].trim().getBytes(StandardCharsets.UTF_8); + this.namesMap.put(columnName, arr); + } + bytes = result.getValue(arr[0], arr[1]); + } + rawRow[i] = bytes; + } catch (Exception e) { + log.error( + "Cannot read data from {}, reason: \n", this.hbaseParameters.getTable(), e); + } + } + return rawRow; + } + + @Override + public List snapshotState(long checkpointId) { + return new ArrayList<>(sourceSplits); + } + + @Override + public void addSplits(List splits) { + sourceSplits.addAll(splits); + } + + @Override + public void handleNoMoreSplits() { + noMoreSplit = true; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) {} +} diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceSplit.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceSplit.java new file mode 100644 index 00000000000..1d38ddd116e --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceSplit.java @@ -0,0 +1,56 @@ +/* + * 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.hbase.source; + +import org.apache.seatunnel.api.source.SourceSplit; + +public class HbaseSourceSplit implements SourceSplit { + public static final String HBASE_SOURCE_SPLIT_PREFIX = "hbase_source_split_"; + private String splitId; + private byte[] startRow; + private byte[] endRow; + + public HbaseSourceSplit(int splitId) { + this.splitId = HBASE_SOURCE_SPLIT_PREFIX + splitId; + } + + public HbaseSourceSplit(int splitId, byte[] startRow, byte[] endRow) { + this.splitId = HBASE_SOURCE_SPLIT_PREFIX + splitId; + this.startRow = startRow; + this.endRow = endRow; + } + + @Override + public String toString() { + return String.format("{\"split_id\":\"%s\"}", splitId); + } + + @Override + public String splitId() { + return splitId; + } + + public byte[] getStartRow() { + return startRow; + } + + public byte[] getEndRow() { + return endRow; + } +} diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceSplitEnumerator.java new file mode 100644 index 00000000000..094128b1747 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceSplitEnumerator.java @@ -0,0 +1,185 @@ +/* + * 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.hbase.source; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; +import org.apache.seatunnel.connectors.seatunnel.hbase.utils.HbaseConnectionUtil; + +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.RegionLocator; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +@Slf4j +public class HbaseSourceSplitEnumerator + implements SourceSplitEnumerator { + /** Source split enumerator context */ + private final Context context; + + private Config pluginConfig; + /** The splits that has assigned */ + private final Set assignedSplit; + + /** The splits that have not assigned */ + private Set pendingSplit; + + private HbaseParameters hbaseParameters; + private Connection connection; + + public HbaseSourceSplitEnumerator( + Context context, HbaseParameters hbaseParameters) { + this.context = context; + this.hbaseParameters = hbaseParameters; + this.assignedSplit = new HashSet<>(); + connection = HbaseConnectionUtil.getHbaseConnection(hbaseParameters); + } + + public HbaseSourceSplitEnumerator( + Context context, + HbaseParameters hbaseParameters, + HbaseSourceState sourceState) { + this.context = context; + this.hbaseParameters = hbaseParameters; + this.assignedSplit = sourceState.getAssignedSplits(); + connection = HbaseConnectionUtil.getHbaseConnection(hbaseParameters); + } + + @Override + public void open() { + this.pendingSplit = new HashSet<>(); + } + + @Override + public void run() throws Exception { + // do nothing + } + + @Override + public void close() throws IOException { + // do nothing + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + if (!splits.isEmpty()) { + pendingSplit.addAll(splits); + assignSplit(subtaskId); + } + } + + @Override + public int currentUnassignedSplitSize() { + return pendingSplit.size(); + } + + @Override + public void registerReader(int subtaskId) { + pendingSplit = getTableSplits(); + assignSplit(subtaskId); + } + + @Override + public HbaseSourceState snapshotState(long checkpointId) throws Exception { + return new HbaseSourceState(assignedSplit); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + // do nothing + } + + @Override + public void handleSplitRequest(int subtaskId) { + // do nothing + } + + /** Assign split by reader task id */ + private void assignSplit(int taskId) { + ArrayList currentTaskSplits = new ArrayList<>(); + if (context.currentParallelism() == 1) { + // if parallelism == 1, we should assign all the splits to reader + currentTaskSplits.addAll(pendingSplit); + } else { + // if parallelism > 1, according to hashCode of split's id to determine whether to + // allocate the current task + for (HbaseSourceSplit sourceSplit : pendingSplit) { + final int splitOwner = + getSplitOwner(sourceSplit.splitId(), context.currentParallelism()); + if (splitOwner == taskId) { + currentTaskSplits.add(sourceSplit); + } + } + } + // assign splits + context.assignSplit(taskId, currentTaskSplits); + // save the state of assigned splits + assignedSplit.addAll(currentTaskSplits); + // remove the assigned splits from pending splits + currentTaskSplits.forEach(split -> pendingSplit.remove(split)); + log.info( + "SubTask {} is assigned to [{}]", + taskId, + currentTaskSplits.stream() + .map(HbaseSourceSplit::splitId) + .collect(Collectors.joining(","))); + context.signalNoMoreSplits(taskId); + } + + /** Get all splits of table */ + private Set getTableSplits() { + List splits = new ArrayList<>(); + + try { + RegionLocator regionLocator = + connection.getRegionLocator(TableName.valueOf(hbaseParameters.getTable())); + byte[][] startKeys = regionLocator.getStartKeys(); + byte[][] endKeys = regionLocator.getEndKeys(); + if (startKeys.length != endKeys.length) { + throw new IOException( + "Failed to create Splits for HBase table {}. HBase start keys and end keys not equal." + + hbaseParameters.getTable()); + } + + int i = 0; + while (i < startKeys.length) { + splits.add(new HbaseSourceSplit(i, startKeys[i], endKeys[i])); + i++; + } + return new HashSet<>(splits); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** Hash algorithm for assigning splits to readers */ + private static int getSplitOwner(String tp, int numReaders) { + return (tp.hashCode() & Integer.MAX_VALUE) % numReaders; + } +} diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceState.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceState.java new file mode 100644 index 00000000000..f2a79dd5689 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceState.java @@ -0,0 +1,36 @@ +/* + * 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.hbase.source; + +import java.io.Serializable; +import java.util.Set; + +public class HbaseSourceState implements Serializable { + private static final long serialVersionUID = 1L; + + private final Set assignedSplits; + + public HbaseSourceState(Set assignedSplits) { + this.assignedSplits = assignedSplits; + } + + public Set getAssignedSplits() { + return assignedSplits; + } +} diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/utils/HbaseConnectionUtil.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/utils/HbaseConnectionUtil.java new file mode 100644 index 00000000000..f006986e660 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/utils/HbaseConnectionUtil.java @@ -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. + */ + +package org.apache.seatunnel.connectors.seatunnel.hbase.utils; + +import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; +import org.apache.seatunnel.connectors.seatunnel.hbase.exception.HbaseConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.hbase.exception.HbaseConnectorException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; + +import java.io.IOException; + +public class HbaseConnectionUtil { + public static Connection getHbaseConnection(HbaseParameters hbaseParameters) { + Configuration hbaseConfiguration = HBaseConfiguration.create(); + hbaseConfiguration.set("hbase.zookeeper.quorum", hbaseParameters.getZookeeperQuorum()); + if (hbaseParameters.getHbaseExtraConfig() != null) { + hbaseParameters.getHbaseExtraConfig().forEach(hbaseConfiguration::set); + } + // initialize hbase connection + try { + Connection connection = ConnectionFactory.createConnection(hbaseConfiguration); + return connection; + } catch (IOException e) { + String errorMsg = "Build Hbase connection failed."; + throw new HbaseConnectorException(HbaseConnectorErrorCode.CONNECTION_FAILED, errorMsg); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java index 4c7a9587ea2..e27e5c715ed 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java @@ -147,6 +147,12 @@ public void testHbaseSink(TestContainer container) throws IOException, Interrupt Assertions.assertEquals(results.size(), 5); } + @TestTemplate + public void testHbaseSource(TestContainer container) throws IOException, InterruptedException { + Container.ExecResult execResult = container.executeJob("/hbase-to-assert.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } + @TestTemplate public void testHbaseSinkWithArray(TestContainer container) throws IOException, InterruptedException { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-to-assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-to-assert.conf new file mode 100644 index 00000000000..f209875745b --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-to-assert.conf @@ -0,0 +1,119 @@ +# +# 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 { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Hbase { + zookeeper_quorum = "hbase-e2e:2181" + table = "seatunnel_test" + query_columns=["rowkey", "cf1:col1", "cf1:col2", "cf2:col1", "cf2:col2"] + schema = { + columns = [ + { + name = rowkey + type = string + }, + { + name = "cf1:col1" + type = boolean + }, + { + name = "cf1:col2" + type = double + }, + { + name = "cf2:col1" + type = bigint + }, + { + name = "cf2:col2" + type = int + } + ] + } + result_table_name = hbase_source + } +} + +sink { + Assert { + source_table_name = hbase_source + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 10000 + }, + { + rule_type = MIN_ROW + rule_value = 10000 + } + ], + field_rules = [ + { + field_name = rowkey + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = "cf1:col1" + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = "cf1:col2" + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = "cf2:col1" + field_type = bigint + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = "cf2:col2" + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file From 8f2049b2f18219ed374cd60913afd618d532d7cf Mon Sep 17 00:00:00 2001 From: Eric Date: Wed, 15 May 2024 16:22:52 +0800 Subject: [PATCH 10/14] [Feature][S3 File] Make S3 File Connector support multiple table write (#6698) --- docs/en/connector-v2/sink/S3File.md | 2 +- .../connector-file/connector-file-s3/pom.xml | 2 +- .../file/s3/catalog/S3FileCatalogFactory.java | 4 +- .../config/{S3Conf.java => S3HadoopConf.java} | 39 ++-- .../seatunnel/file/s3/sink/S3FileSink.java | 4 +- .../file/s3/source/S3FileSource.java | 106 +---------- .../file/s3/source/S3FileSourceFactory.java | 10 ++ .../MultipleTableS3FileSourceConfig.java | 34 ++++ .../s3/source/config/S3FileSourceConfig.java | 46 +++++ .../seatunnel/hive/config/HiveOnS3Conf.java | 6 +- .../seatunnel/hive/storage/S3StorageTest.java | 8 +- .../redshift/sink/S3RedshiftSink.java | 5 +- seatunnel-dist/pom.xml | 2 +- .../connector-file-s3-e2e/pom.xml | 77 ++++++++ .../e2e/connector/file/s3/S3FileIT.java | 166 ++++++++++++++++++ .../file/s3/S3FileWithMultipleTableIT.java | 148 ++++++++++++++++ .../e2e/connector/file/s3/S3Utils.java | 98 +++++++++++ .../src/test/resources/excel/e2e.xlsx | Bin 0 -> 5823 bytes .../resources/excel/fake_to_s3_excel.conf | 81 +++++++++ .../excel/s3_excel_projection_to_assert.conf | 109 ++++++++++++ .../resources/excel/s3_excel_to_assert.conf | 135 ++++++++++++++ ...s3_excel_to_assert_with_multipletable.conf | 134 ++++++++++++++ .../excel/s3_filter_excel_to_assert.conf | 136 ++++++++++++++ .../src/test/resources/json/e2e.json | 5 + .../src/test/resources/json/e2e.json.lzo | Bin 0 -> 3466 bytes .../resources/json/fake_to_s3_file_json.conf | 86 +++++++++ .../json/s3_file_json_lzo_to_console.conf | 144 +++++++++++++++ .../json/s3_file_json_to_assert.conf | 133 ++++++++++++++ ...ile_json_to_assert_with_multipletable.conf | 130 ++++++++++++++ .../resources/json/s3_file_to_console.conf | 46 +++++ .../src/test/resources/orc/e2e.orc | Bin 0 -> 5730 bytes .../resources/orc/fake_to_s3_file_orc.conf | 87 +++++++++ .../orc/s3_file_orc_projection_to_assert.conf | 82 +++++++++ .../resources/orc/s3_file_orc_to_assert.conf | 81 +++++++++ ...file_orc_to_assert_with_multipletable.conf | 66 +++++++ .../src/test/resources/parquet/e2e.parquet | Bin 0 -> 9730 bytes .../parquet/fake_to_s3_file_parquet.conf | 87 +++++++++ .../s3_file_parquet_projection_to_assert.conf | 82 +++++++++ .../parquet/s3_file_parquet_to_assert.conf | 99 +++++++++++ ..._parquet_to_assert_with_multipletable.conf | 66 +++++++ .../resources/parquet/s3_file_to_console.conf | 42 +++++ .../src/test/resources/text/e2e.txt | 5 + .../src/test/resources/text/e2e.txt.lzo | Bin 0 -> 2720 bytes .../src/test/resources/text/e2e_delimiter.txt | 5 + .../test/resources/text/e2e_time_format.txt | 5 + .../resources/text/fake_to_s3_file_text.conf | 87 +++++++++ .../fake_to_s3_file_with_multiple_table.conf | 128 ++++++++++++++ .../text/s3_file_delimiter_assert.conf | 109 ++++++++++++ .../text/s3_file_text_lzo_to_assert.conf | 143 +++++++++++++++ .../s3_file_text_projection_to_assert.conf | 134 ++++++++++++++ .../text/s3_file_text_skip_headers.conf | 134 ++++++++++++++ .../text/s3_file_text_to_assert.conf | 133 ++++++++++++++ ...ile_text_to_assert_with_multipletable.conf | 130 ++++++++++++++ .../text/s3_file_time_format_assert.conf | 100 +++++++++++ .../seatunnel-connector-v2-e2e/pom.xml | 1 + .../seatunnel-hadoop3-3.1.4-uber/pom.xml | 1 + 56 files changed, 3566 insertions(+), 137 deletions(-) rename seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/config/{S3Conf.java => S3HadoopConf.java} (67%) create mode 100644 seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/config/MultipleTableS3FileSourceConfig.java create mode 100644 seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/config/S3FileSourceConfig.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3FileIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3FileWithMultipleTableIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3Utils.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/e2e.xlsx create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/fake_to_s3_excel.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_excel_projection_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_excel_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_excel_to_assert_with_multipletable.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_filter_excel_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/e2e.json create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/e2e.json.lzo create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/fake_to_s3_file_json.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_json_lzo_to_console.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_json_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_json_to_assert_with_multipletable.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_to_console.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/orc/e2e.orc create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/orc/fake_to_s3_file_orc.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/orc/s3_file_orc_projection_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/orc/s3_file_orc_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/orc/s3_file_orc_to_assert_with_multipletable.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/e2e.parquet create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/fake_to_s3_file_parquet.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_parquet_projection_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_parquet_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_parquet_to_assert_with_multipletable.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_to_console.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e.txt create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e.txt.lzo create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e_delimiter.txt create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e_time_format.txt create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/fake_to_s3_file_text.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/fake_to_s3_file_with_multiple_table.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_delimiter_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_lzo_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_projection_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_skip_headers.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_to_assert_with_multipletable.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_time_format_assert.conf diff --git a/docs/en/connector-v2/sink/S3File.md b/docs/en/connector-v2/sink/S3File.md index 322263b1ede..508524c5115 100644 --- a/docs/en/connector-v2/sink/S3File.md +++ b/docs/en/connector-v2/sink/S3File.md @@ -474,7 +474,7 @@ transform { sink { S3File { bucket = "s3a://seatunnel-test" - tmp_path = "/tmp/seatunnel" + tmp_path = "/tmp/seatunnel/${table_name}" path="/test/${table_name}" fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" fs.s3a.aws.credentials.provider="org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" diff --git a/seatunnel-connectors-v2/connector-file/connector-file-s3/pom.xml b/seatunnel-connectors-v2/connector-file/connector-file-s3/pom.xml index 261f5fdbb95..fbf0016fced 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-s3/pom.xml +++ b/seatunnel-connectors-v2/connector-file/connector-file-s3/pom.xml @@ -30,7 +30,7 @@ SeaTunnel : Connectors V2 : File : S3 - 2.6.5 + 3.1.4 27.0-jre diff --git a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/catalog/S3FileCatalogFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/catalog/S3FileCatalogFactory.java index add4b0ac208..53f350e10ca 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/catalog/S3FileCatalogFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/catalog/S3FileCatalogFactory.java @@ -24,7 +24,7 @@ import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf; import org.apache.seatunnel.connectors.seatunnel.file.hadoop.HadoopFileSystemProxy; -import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3Conf; +import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3HadoopConf; import com.google.auto.service.AutoService; @@ -32,7 +32,7 @@ public class S3FileCatalogFactory implements CatalogFactory { @Override public Catalog createCatalog(String catalogName, ReadonlyConfig options) { - HadoopConf hadoopConf = S3Conf.buildWithReadOnlyConfig(options); + HadoopConf hadoopConf = S3HadoopConf.buildWithReadOnlyConfig(options); HadoopFileSystemProxy fileSystemUtils = new HadoopFileSystemProxy(hadoopConf); return new S3FileCatalog(fileSystemUtils, options); } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/config/S3Conf.java b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/config/S3HadoopConf.java similarity index 67% rename from seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/config/S3Conf.java rename to seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/config/S3HadoopConf.java index 2680ce151c7..557403b28ac 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/config/S3Conf.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/config/S3HadoopConf.java @@ -17,16 +17,13 @@ package org.apache.seatunnel.connectors.seatunnel.file.s3.config; -import org.apache.seatunnel.shade.com.typesafe.config.Config; - import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf; import java.util.HashMap; import java.util.Map; -public class S3Conf extends HadoopConf { +public class S3HadoopConf extends HadoopConf { private static final String HDFS_S3N_IMPL = "org.apache.hadoop.fs.s3native.NativeS3FileSystem"; private static final String HDFS_S3A_IMPL = "org.apache.hadoop.fs.s3a.S3AFileSystem"; protected static final String S3A_SCHEMA = "s3a"; @@ -47,39 +44,33 @@ public void setSchema(String schema) { this.schema = schema; } - protected S3Conf(String hdfsNameKey) { + public S3HadoopConf(String hdfsNameKey) { super(hdfsNameKey); } - public static HadoopConf buildWithConfig(Config config) { + public static HadoopConf buildWithReadOnlyConfig(ReadonlyConfig config) { - String bucketName = config.getString(S3ConfigOptions.S3_BUCKET.key()); - S3Conf hadoopConf = new S3Conf(bucketName); + String bucketName = config.get(S3ConfigOptions.S3_BUCKET); + S3HadoopConf hadoopConf = new S3HadoopConf(bucketName); if (bucketName.startsWith(S3A_SCHEMA)) { hadoopConf.setSchema(S3A_SCHEMA); } HashMap s3Options = new HashMap<>(); hadoopConf.putS3SK(s3Options, config); - if (CheckConfigUtil.isValidParam(config, S3ConfigOptions.S3_PROPERTIES.key())) { - config.getObject(S3ConfigOptions.S3_PROPERTIES.key()) - .forEach((key, value) -> s3Options.put(key, String.valueOf(value.unwrapped()))); + if (config.getOptional(S3ConfigOptions.S3_PROPERTIES).isPresent()) { + config.get(S3ConfigOptions.S3_PROPERTIES) + .forEach((key, value) -> s3Options.put(key, String.valueOf(value))); } s3Options.put( S3ConfigOptions.S3A_AWS_CREDENTIALS_PROVIDER.key(), - config.getString(S3ConfigOptions.S3A_AWS_CREDENTIALS_PROVIDER.key())); + config.get(S3ConfigOptions.S3A_AWS_CREDENTIALS_PROVIDER).getProvider()); s3Options.put( - S3ConfigOptions.FS_S3A_ENDPOINT.key(), - config.getString(S3ConfigOptions.FS_S3A_ENDPOINT.key())); + S3ConfigOptions.FS_S3A_ENDPOINT.key(), config.get(S3ConfigOptions.FS_S3A_ENDPOINT)); hadoopConf.setExtraOptions(s3Options); return hadoopConf; } - public static HadoopConf buildWithReadOnlyConfig(ReadonlyConfig readonlyConfig) { - Config config = readonlyConfig.toConfig(); - return buildWithConfig(config); - } - protected String switchHdfsImpl() { switch (this.schema) { case S3A_SCHEMA: @@ -89,13 +80,13 @@ protected String switchHdfsImpl() { } } - private void putS3SK(Map s3Options, Config config) { - if (!CheckConfigUtil.isValidParam(config, S3ConfigOptions.S3_ACCESS_KEY.key()) - && !CheckConfigUtil.isValidParam(config, S3ConfigOptions.S3_SECRET_KEY.key())) { + private void putS3SK(Map s3Options, ReadonlyConfig config) { + if (!config.getOptional(S3ConfigOptions.S3_ACCESS_KEY).isPresent() + && config.getOptional(S3ConfigOptions.S3_SECRET_KEY).isPresent()) { return; } - String accessKey = config.getString(S3ConfigOptions.S3_ACCESS_KEY.key()); - String secretKey = config.getString(S3ConfigOptions.S3_SECRET_KEY.key()); + String accessKey = config.get(S3ConfigOptions.S3_ACCESS_KEY); + String secretKey = config.get(S3ConfigOptions.S3_SECRET_KEY); if (S3A_SCHEMA.equals(this.schema)) { s3Options.put("fs.s3a.access.key", accessKey); s3Options.put("fs.s3a.secret.key", secretKey); diff --git a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/sink/S3FileSink.java b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/sink/S3FileSink.java index b7583569523..2a636bcbcc9 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/sink/S3FileSink.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/sink/S3FileSink.java @@ -34,8 +34,8 @@ import org.apache.seatunnel.common.constants.PluginType; import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; -import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3Conf; import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3ConfigOptions; +import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3HadoopConf; import org.apache.seatunnel.connectors.seatunnel.file.sink.BaseMultipleTableFileSink; import java.util.Optional; @@ -55,7 +55,7 @@ public String getPluginName() { } public S3FileSink(CatalogTable catalogTable, ReadonlyConfig readonlyConfig) { - super(S3Conf.buildWithConfig(readonlyConfig.toConfig()), readonlyConfig, catalogTable); + super(S3HadoopConf.buildWithReadOnlyConfig(readonlyConfig), readonlyConfig, catalogTable); this.catalogTable = catalogTable; this.readonlyConfig = readonlyConfig; Config pluginConfig = readonlyConfig.toConfig(); diff --git a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSource.java b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSource.java index f36e935818c..a2918062ecb 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSource.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSource.java @@ -17,111 +17,19 @@ package org.apache.seatunnel.connectors.seatunnel.file.s3.source; -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; -import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; -import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.config.CheckConfigUtil; -import org.apache.seatunnel.common.config.CheckResult; -import org.apache.seatunnel.common.constants.PluginType; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; -import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; -import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorErrorCode; -import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; -import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3Conf; -import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3ConfigOptions; -import org.apache.seatunnel.connectors.seatunnel.file.source.BaseFileSource; -import org.apache.seatunnel.connectors.seatunnel.file.source.reader.ReadStrategyFactory; +import org.apache.seatunnel.connectors.seatunnel.file.s3.source.config.MultipleTableS3FileSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.file.source.BaseMultipleTableFileSource; -import com.google.auto.service.AutoService; +public class S3FileSource extends BaseMultipleTableFileSource { -import java.io.IOException; + public S3FileSource(ReadonlyConfig readonlyConfig) { + super(new MultipleTableS3FileSourceConfig(readonlyConfig)); + } -@AutoService(SeaTunnelSource.class) -public class S3FileSource extends BaseFileSource { @Override public String getPluginName() { return FileSystemType.S3.getFileSystemPluginName(); } - - @Override - public void prepare(Config pluginConfig) throws PrepareFailException { - CheckResult result = - CheckConfigUtil.checkAllExists( - pluginConfig, - S3ConfigOptions.FILE_PATH.key(), - S3ConfigOptions.FILE_FORMAT_TYPE.key(), - S3ConfigOptions.S3_BUCKET.key()); - if (!result.isSuccess()) { - throw new FileConnectorException( - SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, - String.format( - "PluginName: %s, PluginType: %s, Message: %s", - getPluginName(), PluginType.SOURCE, result.getMsg())); - } - String path = pluginConfig.getString(S3ConfigOptions.FILE_PATH.key()); - hadoopConf = S3Conf.buildWithConfig(pluginConfig); - readStrategy = - ReadStrategyFactory.of( - pluginConfig.getString(S3ConfigOptions.FILE_FORMAT_TYPE.key())); - readStrategy.setPluginConfig(pluginConfig); - readStrategy.init(hadoopConf); - try { - filePaths = readStrategy.getFileNamesByPath(path); - } catch (IOException e) { - String errorMsg = String.format("Get file list from this path [%s] failed", path); - throw new FileConnectorException( - FileConnectorErrorCode.FILE_LIST_GET_FAILED, errorMsg, e); - } - // support user-defined schema - FileFormat fileFormat = - FileFormat.valueOf( - pluginConfig - .getString(S3ConfigOptions.FILE_FORMAT_TYPE.key()) - .toUpperCase()); - // only json text csv type support user-defined schema now - if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { - switch (fileFormat) { - case CSV: - case TEXT: - case JSON: - case EXCEL: - case XML: - SeaTunnelRowType userDefinedSchema = - CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); - readStrategy.setSeaTunnelRowTypeInfo(userDefinedSchema); - rowType = readStrategy.getActualSeaTunnelRowTypeInfo(); - break; - case ORC: - case PARQUET: - throw new FileConnectorException( - CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, - "SeaTunnel does not support user-defined schema for [parquet, orc] files"); - default: - // never got in there - throw new FileConnectorException( - CommonErrorCodeDeprecated.ILLEGAL_ARGUMENT, - "SeaTunnel does not supported this file format"); - } - } else { - if (filePaths.isEmpty()) { - // When the directory is empty, distribute default behavior schema - rowType = CatalogTableUtil.buildSimpleTextSchema(); - return; - } - try { - rowType = readStrategy.getSeaTunnelRowTypeInfo(filePaths.get(0)); - } catch (FileConnectorException e) { - String errorMsg = - String.format("Get table schema from file [%s] failed", filePaths.get(0)); - throw new FileConnectorException( - CommonErrorCodeDeprecated.TABLE_SCHEMA_GET_FAILED, errorMsg, e); - } - } - } } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSourceFactory.java index ebd752fbf09..93d879e559c 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSourceFactory.java @@ -19,9 +19,12 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; +import org.apache.seatunnel.api.table.connector.TableSource; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfigOptions; import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; @@ -29,6 +32,7 @@ import com.google.auto.service.AutoService; +import java.io.Serializable; import java.util.Arrays; @AutoService(Factory.class) @@ -38,6 +42,12 @@ public String factoryIdentifier() { return FileSystemType.S3.getFileSystemPluginName(); } + @Override + public + TableSource createSource(TableSourceFactoryContext context) { + return () -> (SeaTunnelSource) new S3FileSource(context.getOptions()); + } + @Override public OptionRule optionRule() { return OptionRule.builder() diff --git a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/config/MultipleTableS3FileSourceConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/config/MultipleTableS3FileSourceConfig.java new file mode 100644 index 00000000000..ac9114d9e46 --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/config/MultipleTableS3FileSourceConfig.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.seatunnel.file.s3.source.config; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseFileSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseMultipleTableFileSourceConfig; + +public class MultipleTableS3FileSourceConfig extends BaseMultipleTableFileSourceConfig { + + public MultipleTableS3FileSourceConfig(ReadonlyConfig s3FileSourceRootConfig) { + super(s3FileSourceRootConfig); + } + + @Override + public BaseFileSourceConfig getBaseSourceConfig(ReadonlyConfig readonlyConfig) { + return new S3FileSourceConfig(readonlyConfig); + } +} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/config/S3FileSourceConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/config/S3FileSourceConfig.java new file mode 100644 index 00000000000..e9767bf18a5 --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/config/S3FileSourceConfig.java @@ -0,0 +1,46 @@ +/* + * 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.file.s3.source.config; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseFileSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; +import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf; +import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3HadoopConf; + +import lombok.Getter; + +@Getter +public class S3FileSourceConfig extends BaseFileSourceConfig { + + private static final long serialVersionUID = 1L; + + @Override + public HadoopConf getHadoopConfig() { + return S3HadoopConf.buildWithReadOnlyConfig(getBaseFileSourceConfig()); + } + + @Override + public String getPluginName() { + return FileSystemType.S3.getFileSystemPluginName(); + } + + public S3FileSourceConfig(ReadonlyConfig readonlyConfig) { + super(readonlyConfig); + } +} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/HiveOnS3Conf.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/HiveOnS3Conf.java index 01fede7517c..be58932f9cd 100644 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/HiveOnS3Conf.java +++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/HiveOnS3Conf.java @@ -19,10 +19,10 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf; -import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3Conf; import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3ConfigOptions; +import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3HadoopConf; -public class HiveOnS3Conf extends S3Conf { +public class HiveOnS3Conf extends S3HadoopConf { protected static final String S3_SCHEMA = "s3"; // The emr of amazon on s3 use this EmrFileSystem as the file system protected static final String HDFS_S3_IMPL = "com.amazon.ws.emr.hadoop.fs.EmrFileSystem"; @@ -43,7 +43,7 @@ protected String switchHdfsImpl() { } public static HadoopConf buildWithReadOnlyConfig(ReadonlyConfig readonlyConfig) { - S3Conf s3Conf = (S3Conf) S3Conf.buildWithReadOnlyConfig(readonlyConfig); + S3HadoopConf s3Conf = (S3HadoopConf) S3HadoopConf.buildWithReadOnlyConfig(readonlyConfig); String bucketName = readonlyConfig.get(S3ConfigOptions.S3_BUCKET); if (bucketName.startsWith(DEFAULT_SCHEMA)) { s3Conf.setSchema(DEFAULT_SCHEMA); diff --git a/seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/storage/S3StorageTest.java b/seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/storage/S3StorageTest.java index 52edf0fb4d9..ed475f74ed1 100644 --- a/seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/storage/S3StorageTest.java +++ b/seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/storage/S3StorageTest.java @@ -44,7 +44,9 @@ public class S3StorageTest { put( S3ConfigOptions.S3A_AWS_CREDENTIALS_PROVIDER .key(), - "provider"); + S3ConfigOptions.S3aAwsCredentialsProvider + .InstanceProfileCredentialsProvider + .getProvider()); put( S3ConfigOptions.FS_S3A_ENDPOINT.key(), "http://s3.ap-northeast-1.amazonaws.com"); @@ -65,7 +67,9 @@ public class S3StorageTest { put( S3ConfigOptions.S3A_AWS_CREDENTIALS_PROVIDER .key(), - "testProvider"); + S3ConfigOptions.S3aAwsCredentialsProvider + .InstanceProfileCredentialsProvider + .getProvider()); put(S3ConfigOptions.FS_S3A_ENDPOINT.key(), "test"); } }); diff --git a/seatunnel-connectors-v2/connector-s3-redshift/src/main/java/org/apache/seatunnel/connectors/seatunnel/redshift/sink/S3RedshiftSink.java b/seatunnel-connectors-v2/connector-s3-redshift/src/main/java/org/apache/seatunnel/connectors/seatunnel/redshift/sink/S3RedshiftSink.java index 6b954b63c48..28a8e250429 100644 --- a/seatunnel-connectors-v2/connector-s3-redshift/src/main/java/org/apache/seatunnel/connectors/seatunnel/redshift/sink/S3RedshiftSink.java +++ b/seatunnel-connectors-v2/connector-s3-redshift/src/main/java/org/apache/seatunnel/connectors/seatunnel/redshift/sink/S3RedshiftSink.java @@ -21,14 +21,15 @@ import org.apache.seatunnel.api.common.PrepareFailException; import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SinkAggregatedCommitter; import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.common.config.CheckResult; import org.apache.seatunnel.common.constants.PluginType; import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.BaseHdfsFileSink; -import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3Conf; import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3ConfigOptions; +import org.apache.seatunnel.connectors.seatunnel.file.s3.config.S3HadoopConf; import org.apache.seatunnel.connectors.seatunnel.file.sink.commit.FileAggregatedCommitInfo; import org.apache.seatunnel.connectors.seatunnel.file.sink.commit.FileCommitInfo; import org.apache.seatunnel.connectors.seatunnel.redshift.commit.S3RedshiftSinkAggregatedCommitter; @@ -66,7 +67,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { getPluginName(), PluginType.SINK, checkResult.getMsg())); } this.pluginConfig = pluginConfig; - hadoopConf = S3Conf.buildWithConfig(pluginConfig); + hadoopConf = S3HadoopConf.buildWithReadOnlyConfig(ReadonlyConfig.fromConfig(pluginConfig)); } @Override diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index 41cb15cd63b..28b987b4016 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -99,7 +99,7 @@ 3.13.29 - 3.0.0 + 3.1.4 2.4.7 3.1.4 1.11.271 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/pom.xml new file mode 100644 index 00000000000..e33b6273a89 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/pom.xml @@ -0,0 +1,77 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + + connector-file-s3-e2e + SeaTunnel : E2E : Connector V2 : File S3 + + 3.1.4 + + + + + org.apache.seatunnel + connector-file-s3 + ${project.version} + test + + + org.apache.seatunnel + connector-fake + ${project.version} + test + + + org.apache.hadoop + hadoop-aws + ${hadoop-aws.version} + test + + + com.amazonaws + aws-java-sdk-bundle + 1.11.271 + test + + + org.apache.seatunnel + seatunnel-hadoop3-3.1.4-uber + ${project.version} + optional + test + + + org.apache.avro + avro + + + + + org.apache.seatunnel + connector-assert + ${project.version} + test + + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3FileIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3FileIT.java new file mode 100644 index 00000000000..b2ba903518e --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3FileIT.java @@ -0,0 +1,166 @@ +/* + * 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.e2e.connector.file.s3; + +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.container.TestHelper; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; +import org.apache.seatunnel.e2e.common.util.ContainerUtil; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; + +import io.airlift.compress.lzo.LzopCodec; + +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; + +@Disabled("have no s3 environment to run this test") +public class S3FileIT extends TestSuiteBase { + + public static final String S3_SDK_DOWNLOAD = + "https://repo1.maven.org/maven2/com/amazonaws/aws-java-sdk-bundle/1.11.271/aws-java-sdk-bundle-1.11.271.jar"; + public static final String HADOOP_S3_DOWNLOAD = + "https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-aws/3.1.4/hadoop-aws-3.1.4.jar"; + + @TestContainerExtension + private final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/s3/lib && cd /tmp/seatunnel/plugins/s3/lib && curl -O " + + S3_SDK_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", + "-c", + "cd /tmp/seatunnel/plugins/s3/lib && curl -O " + + HADOOP_S3_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + }; + + /** Copy data files to s3 */ + @TestTemplate + public void testS3FileReadAndWrite(TestContainer container) + throws IOException, InterruptedException { + // Copy test files to s3 + S3Utils s3Utils = new S3Utils(); + try { + s3Utils.uploadTestFiles( + "/json/e2e.json", + "test/seatunnel/read/json/name=tyrantlucifer/hobby=coding/e2e.json", + true); + Path jsonLzo = convertToLzoFile(ContainerUtil.getResourcesFile("/json/e2e.json")); + s3Utils.uploadTestFiles( + jsonLzo.toString(), "test/seatunnel/read/lzo_json/e2e.json", false); + s3Utils.uploadTestFiles( + "/text/e2e.txt", + "test/seatunnel/read/text/name=tyrantlucifer/hobby=coding/e2e.txt", + true); + s3Utils.uploadTestFiles( + "/text/e2e_delimiter.txt", "test/seatunnel/read/text_delimiter/e2e.txt", true); + s3Utils.uploadTestFiles( + "/text/e2e_time_format.txt", + "test/seatunnel/read/text_time_format/e2e.txt", + true); + Path txtLzo = convertToLzoFile(ContainerUtil.getResourcesFile("/text/e2e.txt")); + s3Utils.uploadTestFiles( + txtLzo.toString(), "test/seatunnel/read/lzo_text/e2e.txt", false); + s3Utils.uploadTestFiles( + "/excel/e2e.xlsx", + "test/seatunnel/read/excel/name=tyrantlucifer/hobby=coding/e2e.xlsx", + true); + s3Utils.uploadTestFiles( + "/orc/e2e.orc", + "test/seatunnel/read/orc/name=tyrantlucifer/hobby=coding/e2e.orc", + true); + s3Utils.uploadTestFiles( + "/parquet/e2e.parquet", + "test/seatunnel/read/parquet/name=tyrantlucifer/hobby=coding/e2e.parquet", + true); + s3Utils.uploadTestFiles( + "/excel/e2e.xlsx", + "test/seatunnel/read/excel_filter/name=tyrantlucifer/hobby=coding/e2e_filter.xlsx", + true); + s3Utils.createDir("tmp/fake_empty"); + } finally { + s3Utils.close(); + } + + TestHelper helper = new TestHelper(container); + + helper.execute("/excel/fake_to_s3_excel.conf"); + helper.execute("/excel/s3_excel_to_assert.conf"); + helper.execute("/excel/s3_excel_projection_to_assert.conf"); + // test write s3 text file + helper.execute("/text/fake_to_s3_file_text.conf"); + helper.execute("/text/s3_file_text_lzo_to_assert.conf"); + helper.execute("/text/s3_file_delimiter_assert.conf"); + helper.execute("/text/s3_file_time_format_assert.conf"); + // test read skip header + helper.execute("/text/s3_file_text_skip_headers.conf"); + // test read s3 text file + helper.execute("/text/s3_file_text_to_assert.conf"); + // test read s3 text file with projection + helper.execute("/text/s3_file_text_projection_to_assert.conf"); + // test write s3 json file + helper.execute("/json/fake_to_s3_file_json.conf"); + // test read s3 json file + helper.execute("/json/s3_file_json_to_assert.conf"); + helper.execute("/json/s3_file_json_lzo_to_console.conf"); + // test write s3 orc file + helper.execute("/orc/fake_to_s3_file_orc.conf"); + // test read s3 orc file + helper.execute("/orc/s3_file_orc_to_assert.conf"); + // test read s3 orc file with projection + helper.execute("/orc/s3_file_orc_projection_to_assert.conf"); + // test write s3 parquet file + helper.execute("/parquet/fake_to_s3_file_parquet.conf"); + // test read s3 parquet file + helper.execute("/parquet/s3_file_parquet_to_assert.conf"); + // test read s3 parquet file with projection + helper.execute("/parquet/s3_file_parquet_projection_to_assert.conf"); + // test read filtered s3 file + helper.execute("/excel/s3_filter_excel_to_assert.conf"); + + // test read empty directory + helper.execute("/json/s3_file_to_console.conf"); + helper.execute("/parquet/s3_file_to_console.conf"); + } + + private Path convertToLzoFile(File file) throws IOException { + LzopCodec lzo = new LzopCodec(); + Path path = Paths.get(file.getAbsolutePath() + ".lzo"); + OutputStream outputStream = lzo.createOutputStream(Files.newOutputStream(path)); + outputStream.write(Files.readAllBytes(file.toPath())); + outputStream.close(); + return path; + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3FileWithMultipleTableIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3FileWithMultipleTableIT.java new file mode 100644 index 00000000000..34fd443146c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3FileWithMultipleTableIT.java @@ -0,0 +1,148 @@ +/* + * 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.e2e.connector.file.s3; + +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.container.TestHelper; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; + +import java.io.IOException; + +@Disabled("have no s3 environment to run this test") +public class S3FileWithMultipleTableIT extends TestSuiteBase { + + public static final String S3_SDK_DOWNLOAD = + "https://repo1.maven.org/maven2/com/amazonaws/aws-java-sdk-bundle/1.11.271/aws-java-sdk-bundle-1.11.271.jar"; + public static final String HADOOP_S3_DOWNLOAD = + "https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-aws/3.1.4/hadoop-aws-3.1.4.jar"; + + @TestContainerExtension + private final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/s3/lib && cd /tmp/seatunnel/plugins/s3/lib && curl -O " + + S3_SDK_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", + "-c", + "cd /tmp/seatunnel/plugins/s3/lib && curl -O " + + HADOOP_S3_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", + "-c", + "cd /tmp/seatunnel/lib && curl -O " + S3_SDK_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + + extraCommands = + container.execInContainer( + "bash", + "-c", + "cd /tmp/seatunnel/lib && curl -O " + HADOOP_S3_DOWNLOAD); + Assertions.assertEquals(0, extraCommands.getExitCode()); + }; + + /** Copy data files to s3 */ + @TestTemplate + public void addTestFiles(TestContainer container) throws IOException, InterruptedException { + // Copy test files to s3 + S3Utils s3Utils = new S3Utils(); + try { + s3Utils.uploadTestFiles( + "/json/e2e.json", + "test/seatunnel/read/json/name=tyrantlucifer/hobby=coding/e2e.json", + true); + s3Utils.uploadTestFiles( + "/text/e2e.txt", + "test/seatunnel/read/text/name=tyrantlucifer/hobby=coding/e2e.txt", + true); + s3Utils.uploadTestFiles( + "/excel/e2e.xlsx", + "test/seatunnel/read/excel/name=tyrantlucifer/hobby=coding/e2e.xlsx", + true); + s3Utils.uploadTestFiles( + "/orc/e2e.orc", + "test/seatunnel/read/orc/name=tyrantlucifer/hobby=coding/e2e.orc", + true); + s3Utils.uploadTestFiles( + "/parquet/e2e.parquet", + "test/seatunnel/read/parquet/name=tyrantlucifer/hobby=coding/e2e.parquet", + true); + s3Utils.createDir("tmp/fake_empty"); + } finally { + s3Utils.close(); + } + } + + @TestTemplate + public void testFakeToS3FileInMultipleTableMode_text(TestContainer testContainer) + throws IOException, InterruptedException { + TestHelper helper = new TestHelper(testContainer); + helper.execute("/text/fake_to_s3_file_with_multiple_table.conf"); + } + + @TestTemplate + public void testS3FileReadAndWriteInMultipleTableMode_excel(TestContainer container) + throws IOException, InterruptedException { + TestHelper helper = new TestHelper(container); + helper.execute("/excel/s3_excel_to_assert_with_multipletable.conf"); + } + + @TestTemplate + public void testS3FileReadAndWriteInMultipleTableMode_json(TestContainer container) + throws IOException, InterruptedException { + TestHelper helper = new TestHelper(container); + helper.execute("/json/s3_file_json_to_assert_with_multipletable.conf"); + } + + @TestTemplate + public void testS3FileReadAndWriteInMultipleTableMode_orc(TestContainer container) + throws IOException, InterruptedException { + TestHelper helper = new TestHelper(container); + helper.execute("/orc/s3_file_orc_to_assert_with_multipletable.conf"); + } + + @TestTemplate + public void testS3FileReadAndWriteInMultipleTableMode_parquet(TestContainer container) + throws IOException, InterruptedException { + TestHelper helper = new TestHelper(container); + helper.execute("/parquet/s3_file_parquet_to_assert_with_multipletable.conf"); + } + + @TestTemplate + public void testS3FileReadAndWriteInMultipleTableMode_text(TestContainer container) + throws IOException, InterruptedException { + TestHelper helper = new TestHelper(container); + helper.execute("/text/s3_file_text_to_assert_with_multipletable.conf"); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3Utils.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3Utils.java new file mode 100644 index 00000000000..63789b0d281 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3Utils.java @@ -0,0 +1,98 @@ +/* + * 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.e2e.connector.file.s3; + +import org.apache.seatunnel.e2e.common.util.ContainerUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.amazonaws.auth.AWSStaticCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.client.builder.AwsClientBuilder; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3ClientBuilder; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.PutObjectRequest; + +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.InputStream; + +public class S3Utils { + private static Logger logger = LoggerFactory.getLogger(S3Utils.class); + private static final String ACCESS_KEY = "XXXXXX"; + private static final String SECRET_KEY = "AWS_XXXX"; + private static final String REGION = "cn-north-1"; + private static final String ENDPOINT = + "s3.cn-north-1.amazonaws.com.cn"; // For example, "https://s3.amazonaws.com" + private String bucket = "ws-package"; + + private final AmazonS3 s3Client; + + public S3Utils() { + BasicAWSCredentials credentials = new BasicAWSCredentials(ACCESS_KEY, SECRET_KEY); + + this.s3Client = + AmazonS3ClientBuilder.standard() + .withCredentials(new AWSStaticCredentialsProvider(credentials)) + .withEndpointConfiguration( + new AwsClientBuilder.EndpointConfiguration(ENDPOINT, REGION)) + .build(); + } + + public void uploadTestFiles( + String filePath, String targetFilePath, boolean isFindFromResource) { + File resourcesFile = null; + if (isFindFromResource) { + resourcesFile = ContainerUtil.getResourcesFile(filePath); + } else { + resourcesFile = new File(filePath); + } + s3Client.putObject(bucket, targetFilePath, resourcesFile); + } + + public void createDir(String dir) { + ObjectMetadata metadata = new ObjectMetadata(); + metadata.setContentLength(0); + InputStream emptyContent = new ByteArrayInputStream(new byte[0]); + PutObjectRequest putObjectRequest = + new PutObjectRequest(bucket, dir, emptyContent, metadata); + s3Client.putObject(putObjectRequest); + } + + public void close() { + if (s3Client != null) { + s3Client.shutdown(); + } + } + + public static void main(String[] args) { + S3Utils s3Utils = new S3Utils(); + s3Utils.uploadTestFiles( + "/Users/gaojun/workspace/seatunnel/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/e2e.xlsx", + "test/test/seatunnel/e2e.xlsx", + false); + + s3Utils.createDir("test/test1"); + s3Utils.uploadTestFiles( + "/Users/gaojun/workspace/seatunnel/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/e2e.xlsx", + "test/test1/seatunnel/e2e.xlsx", + false); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/e2e.xlsx b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/e2e.xlsx new file mode 100644 index 0000000000000000000000000000000000000000..87d363d7db3bbe23b416ef1297979146b04423ea GIT binary patch literal 5823 zcmaJ_bzGF~wxyc^1f)S41PSSu?nYve5{8DMTRKFBPDyD=k(LfA0V(NbC~093kh;V7 zoqObZ&-wPuJ2QX0d;gxjo;~kc&(csvMj=ALz`#J@seG=Ba4!hqzYV3qj&2}FHxnIi zCy=WVhnItWo%Wz(7Z?6ja{u&WewfB{;vi|7zK10+#lW;tI-J-~RM<&UZRRq5we=JL*)9UEcSJ@C%X7$l8$-|S-gEj>L zm0s)UgtuCeb<^^Br^a#2RmkYRgg?SD%6*pNzEo03#Ygf`#$XS=-C45c7_dIqs~xSE z3j)|~ybqPuoP4*N_ByRP=+)OxDi_a%9}vl86TS&9xL{n0`1n3mYE&8~JM1*3i zCP#LDko!b%p7rd-4274tSoy&-;?8ZSH3c-fZvTc?d6hEg@A z6AM0+GKc=M{jG~Tk}~5-!5GG`N_>SOwaXz?h~%Mb#>_-rbBvWL>;MQK_aT z)RC*Byz(8<@p6=9=cq~*zex40a7oY?<+bBpHZBugoniR{ej(qrv+CzL6Axc(1N{2$ zTTO7k9dob?=&m1!xOt_|Ttu<~h3lU%b&9_*i=6}gwc zYh#?Ex(1HZzolg$?!fbKIFjmc*b$UHQ~IYZasfYpU|Pds3a8BCAc7D zJ{E@f>H*&zkD1+ggeh4diP+8$Fpu zO(+sBee2YZ{DlEa|GesB{Go-3u~CEJIFYwzk$3#V+CoN=Po^8QuBh#sR;$yqjhjy& zj+<9Ex4->#`YBj~Pax+G!**h^DpdI2r3!xicMOT(7<$=rx>}pLfGo7#Tx=YzT<>6X zPqovgQzQ&)Z*LD?;d!~pSqlx14t^7i!P3P2LoNiD*X0oHbL91*Z-TQY9U8x7@V2%a zv4MelHv7oUd77F_hF3Dv$!votXSE+k_URUQ!2bNnvW=eFi<{V!+pI#Hl--f(nmt3~ z)#OulgG%V}2gLHH%IzES2K6ji*+tiyqS51Pegq>&!=`QNa%AhaUg;dV&ajl2HNgxu zHB|FWH*&_Bs|$nl%pB0x^{sk-&#aAyzlRd?Esk8-^I8!~wHjh1G0MZ?twh0eD*t%K z8l$I!G3C6`Oo{o+lvI0+-TBav0a4+nSNtYtOh4j59tN4z-eO;3TCN_R5z2VmYflIc z=Y5ZFso0!fMh{r07y%IgTz z4OV7dKU>x_Ff!iD3n{F4vMwkPmU#kO+3uzpx{jv-bU6p(Lv0o+9<5g`eBsEw%$nW@ zG&MH|+!j0$kC?RDHn?n>4t10n6Af#djhSbKZB+_Wuis+btQ-5#KhZAjc+0)SW%W_4 zvXNkcOM695m@Ul$r}?R|VJGS(bqgd>k&jd+kryC{7pl3i8pPw6ti!IMD+t1!(&-Xm zOd<+mc^ZSm?a64g@xr zC_@|Fi&e-*YALHIO%0U-OSuvaG;Byr>EZ(2^np$zG%P zMyE%g?Lf8i*?4L~89F8sfGK_iPB$U5j3%qWN|6`N7CbyUFKGvqIHk_k9+ z$~8o|k>*K_MtnvmyzP{~{`K-kPR7KjxVHQ3i**CTV3~N&I?1We2c?^*3VC?PQHsSG zaZ5!cMx}wnYjr0`*NwP(8Dcc1={#0}g-J6s9`Y%je6R3X<^r8Jy(ERdi{T?trq+!d ztPG0QqLIkWKJzxMaGne5(n({#$?c%R8c6VbmrY8TvZ5h-*om?bXj7(X;d|ImKDKop zh3i45gMd8sl^%(Gjc+2ojBe<)w__`UZPkOuNfPp%&_PNP#>NO!c}TYQF1cC^gjYvrSxMIKc{Gj&&Q_{}qS1~tNt`fZ1cZh5#0HxPR z`~ApfNm;ug71m z9?#GvI6QDPriISPCLa+B`>-)vyc%%HmL28H&j4bGWkCA)kn$67F$!Qjb_~8;`23}0 z@)~hRj*(DT)Cwverg1#E%#;@{=;*6bT|$K`%ORG{uI?A=h6I8h5pMwH^b7@E2A$8+ z!iT%!MvXPsN$HmZbNA-4LOWd^W)Bu}`KkAAmXa^|Wz~qRKJHKnk|Ij!)`|1pp_Lsb zlMM5~Ak<2;YA-q`EOJ#zP2*(sPHWoP1Znq9mCalFN%Kt{PQEXLh#yExZEUX9slOjO zEWPG9UDvHp=HOWr?*;h;=6uDTTRWAQC z!HQNexT3EIX}HG^k3UEKFjc}3OSu1LF;KWv-WTaH^|hi%7yS#Cox~quN(n@ z35>FEgEAdSX?ZMK8vShY$opvbR|j$$RwcQVxdJrx^rn{( zfWPsH^9`GG@p<>!75I)diCREfFf;^&Fr0s4P27L6rkl4t=r6jgOHx(8lbMHCZ&=Ra zx!(ns>;n?fC|8gaUexq1(j99uk^wJH1y#wi5NHv1dt#tx9}}B~tqd#ru?0CRShoj$ zq&?tDCLS-x`gLAKYBXx#opd?+xX}a|B^U{c-2}*e`*gF6&-x(N0U6EHc9$2g(T#1B z0{z7~%lI4BUMR$d1NVt*%@{V7Xvto0XmIv{6vJDJpb87CB!-0rEEemik?o%z3@9Lz z&76Q#^~w*$K=syJxwL)kh}m**-H@G~xUhIGtH$O{a*I<~yq7!6`||1-1$t9#jP`9M z7hG1mugNcj$wK(1wdz(qNe7PLp?<}ui+xFM#!EXO}IU6p8uKw}# zfcN*tdV*c-UV_1PcSY;|H^-ablDR0K%Gg?a9kNQtfj*$%##T z+z~%v%L}$Fp7rn1%!(IWx5%9%)kyO@kP-%ZEw?X;chTMO#=i({84IJD&EnTMr;#(lf zx^J_ORyL#8H8#_gF}8=peidD#-3_}O+G6T7ykoOL}mAF$$8?ckID z?DoyrE9db}(tJlw^U>X@YXoyPXKOk#<;(5%oaXMSXXqj@2FQ^wUhR_q*F^6=I{L(r zN}J9{s4?`o0OK=tfiqJXFL{wTR!-znWW_d9zcH=Bf|8XcVau!7luLh{ z^Zr!_xp8cd=D?6v${OLQF=*)Yhx4MF!uIRQ04E^@sc+ieREu@W{wT$y&5b`dvG1@6 zM;3|y2Jf8bA5T=Izvso(8U%84g)3?J7x&%ZgryF+%nRa&AD$4gFQKE1>T4%xDYA2Z zBvu{vC}y-RQOJvWxhHzV%P24H(G5z)EBFP@n>{~5rhIkvX&E%~+;w3p!nrbZsAi|? zed>I-Pv-5(+36wet>A?NOiN%nHlx&*=NnB7)Ni6*o#xG270ItS5zRKR15Hsml%Sw+Xu;TdLNPZ{>W8_%s+buw8KNY1m5XJ8d8U zGdBu2bUa9bQnJ|(_O+K9)mgK(fHP>Gry5nkcD4BecxS7D$r2v>Ij4gTbw*&IpJo*J zmtB8Z^%Lg&BL4w$h-pyN!CGQhQ11o7VGgs4jwlpAYIywyGxs2E{@%C1j`k)+QM9rQd1N!(AQ=a<6vwz( z&O~C~f9m=R{(hcCHPZePU^BNB8VWm5mXZQ`xu=L`GdxD^hLsD#h{(t2YclX{pvaSp zPFW8S-qmSyF;2K;c)iyE21RuSMe53e_I2XlwnJ?m)g~H;U@{?*j{0LIkz^Cg;s|Eq z$ti)ernKVq7nd9P0B(>>}$Me9~3@5v*fY5Kc zlG`^5y`q}VDiataDXN41A^M7M%CG4{jy3SR!tO@6H=w7$uY(} zl%M7>sOY3XjB4-e$O+7NnQRFF&YlmF&<{JBSnl;(on*sip+pcl?A`LzN;~$qhu(Q>0_9^bOnF5kfaX$C72LGB zQ`SpiV@`B*A@Na3D1$)%Zg*q-!0nK5*4&+xsF0&Cv=9Zw&*AySE)Ts^2sP1)rL8(g zRi9XKYqoPe6vo1lQIQ~EAJ9LGMANq7WhD)Acj{$B*n@PXSKvMD(;IUpR}Gd!ZS3f5 z@u9Ell^Fo*=eLep8vr!yR21IxF)@52-*2X<+fYHskB{P2=&p$0O?qvd?r3htTl5)7 zhA{=LCl29zCm0>66)HdwJBvo50Q+LycV?3flBs$f9F3R@5q#j;D-%${Glv3cyx<|# zhKA_Py%Spl@|{m*%4WPALmP8P!y{|7jSL$mByL}Y9CYPDg}#vyKG=c4H$m=n$nYpeS8HiW8LSce6MzhsdJ4d-3|E`sSd6wu3Fj__DY}oYHb*} z-h37_YB-&+b36<4yLgjxg?{(^IfYMp@Y`#=?}XiXjsG|PgY$U5 zz@1@s-!8f{%zlR-oC@yQXMZN%H)ihKq~DPSe@_1CE&Z8&-;%g9Z+?dr^1bZ;wQ~MU zzpu^ie2d>v1_$atxfp*|xGw_kG~Dl4g~M=H;opkx&(!-|d8ZP7M-#jfJoP>^{+WEA zOz!CPcYFo>mHc1G_0Kx@hvJTUe#Z`cP5)ZN|05y|WmLGLML@uWza-$>WcwF*{{u*V B1w#M; literal 0 HcmV?d00001 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/fake_to_s3_excel.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/fake_to_s3_excel.conf new file mode 100644 index 00000000000..3ff0df55d34 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/fake_to_s3_excel.conf @@ -0,0 +1,81 @@ +# +# 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 { + parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" + "shade.identifier"=aes256 +} + +source { + FakeSource { + result_table_name = "fake" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + S3File { + path="/test/seatunnel/sink" + "file_format_type"=excel + "schema_save_mode"="CREATE_SCHEMA_WHEN_NOT_EXIST" + "data_save_mode"="APPEND_DATA" + "access_key"="XXXXXXXX" + bucket="s3a://ws-package" + "secret_key"="AWS_XXXX" + "fs.s3a.endpoint"="s3.cn-north-1.amazonaws.com.cn" + "fs.s3a.aws.credentials.provider"="org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_excel_projection_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_excel_projection_to_assert.conf new file mode 100644 index 00000000000..0838c32003b --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_excel_projection_to_assert.conf @@ -0,0 +1,109 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/excel" + result_table_name = "fake" + file_format_type = excel + field_delimiter = ; + read_columns = [c_string, c_boolean] + skip_header_row_number = 1 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_excel_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_excel_to_assert.conf new file mode 100644 index 00000000000..f1b9bbb1395 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_excel_to_assert.conf @@ -0,0 +1,135 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/excel" + result_table_name = "fake" + file_format_type = excel + field_delimiter = ; + skip_header_row_number = 1 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_excel_to_assert_with_multipletable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_excel_to_assert_with_multipletable.conf new file mode 100644 index 00000000000..2c1e37a7b6d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_excel_to_assert_with_multipletable.conf @@ -0,0 +1,134 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + tables_configs = [ + { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/excel" + file_format_type = excel + field_delimiter = ; + skip_header_row_number = 1 + schema = { + table = "fake01" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + }, + { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/excel" + file_format_type = excel + field_delimiter = ; + skip_header_row_number = 1 + schema = { + table = "fake02" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } + ] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + table-names = ["fake01", "fake02"] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_filter_excel_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_filter_excel_to_assert.conf new file mode 100644 index 00000000000..d7866c21f63 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/s3_filter_excel_to_assert.conf @@ -0,0 +1,136 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/excel_filter" + result_table_name = "fake" + file_format_type = excel + field_delimiter = ; + skip_header_row_number = 1 + file_filter_pattern = "e2e_filter.*" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/e2e.json b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/e2e.json new file mode 100644 index 00000000000..aff56314e7c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/e2e.json @@ -0,0 +1,5 @@ +{"c_map":{"ccQcS":"PrhhP","ypJZu":"MsOdX","YFBJW":"iPXGR","ipjwT":"kcgPQ","EpKKR":"jgRfX"},"c_array":[887776100,1633238485,1009033208,600614572,1487972145],"c_string":"WArEB","c_boolean":false,"c_tinyint":-90,"c_smallint":15920,"c_int":1127427935,"c_bigint":4712806879122100224,"c_float":1.620476E38,"c_double":2.750908810407852E307,"c_bytes":"Q3NrVnQ=","c_date":"2022-04-27","c_decimal":88574263949141714798.835853182708550244,"c_timestamp":"2022-01-26T17:39:00","c_row":{"C_MAP":{"IVaKD":"bydeV","CnKBd":"kcZdt","RGlmG":"XuMyE","krSIr":"FPeal","IfhvE":"ReKxo"},"C_ARRAY":[86555282,967939739,1162972923,1662468723,546056811],"C_STRING":"bYjyZ","C_BOOLEAN":false,"C_TINYINT":-121,"C_SMALLINT":29252,"C_INT":977226449,"C_BIGINT":5047232039582494720,"C_FLOAT":2.5345643E38,"C_DOUBLE":1.5883424829997996E308,"C_BYTES":"TEVLTHU=","C_DATE":"2022-04-25","C_DECIMAL":55295207715324162970.316560703127334413,"C_TIMESTAMP":"2022-06-14T23:03:00"}} +{"c_map":{"AKiQx":"wIIdk","zgunZ":"qvHRy","ohVQL":"WfBPo","EzUcN":"yPhVF","qusBc":"FWbcI"},"c_array":[1837821269,980724530,2085935679,386596035,1433416218],"c_string":"LGMAw","c_boolean":false,"c_tinyint":-65,"c_smallint":25802,"c_int":1312064317,"c_bigint":4434124023629949952,"c_float":1.0186125E38,"c_double":3.0746920457833206E307,"c_bytes":"V2pjem4=","c_date":"2022-04-21","c_decimal":1943815605574160687.499688237951975681,"c_timestamp":"2022-08-09T09:32:00","c_row":{"C_MAP":{"qMdUz":"ylcLM","bcwFI":"qgkJT","lrPiD":"JRdjf","zmRix":"uqOKy","NEHDJ":"tzJbU"},"C_ARRAY":[951883741,2012849301,1709478035,1095210330,94263648],"C_STRING":"VAdKg","C_BOOLEAN":true,"C_TINYINT":-121,"C_SMALLINT":24543,"C_INT":1853224936,"C_BIGINT":6511613165105889280,"C_FLOAT":2.4886748E38,"C_DOUBLE":1.675530128024138E308,"C_BYTES":"UnNlRXo=","C_DATE":"2022-01-26","C_DECIMAL":50854841532374241314.109746688054104586,"C_TIMESTAMP":"2022-02-18T22:33:00"}} +{"c_map":{"VLlqs":"OwUpp","MWXek":"KDEYD","RAZII":"zGJSJ","wjBNl":"IPTvu","YkGPS":"ORquf"},"c_array":[1530393427,2055877022,1389865473,926021483,402841214],"c_string":"TNcNF","c_boolean":false,"c_tinyint":-93,"c_smallint":26429,"c_int":1890712921,"c_bigint":78884499049828080,"c_float":7.816842E37,"c_double":7.852574522011583E307,"c_bytes":"cHhzZVA=","c_date":"2022-06-05","c_decimal":32486229951636021942.906126821535443395,"c_timestamp":"2022-04-09T16:03:00","c_row":{"C_MAP":{"yIfRN":"gTBEL","oUnIJ":"GtmSz","IGuwP":"TyCOu","BwTUT":"HgnUn","MFrOg":"csTeq"},"C_ARRAY":[306983370,1604264996,2038631670,265692923,717846839],"C_STRING":"wavDf","C_BOOLEAN":true,"C_TINYINT":-48,"C_SMALLINT":29740,"C_INT":1691565731,"C_BIGINT":6162480816264462336,"C_FLOAT":3.3218342E38,"C_DOUBLE":9.993666902591773E307,"C_BYTES":"RnVoR0Q=","C_DATE":"2022-04-09","C_DECIMAL":81349181592680914623.14214231545254843,"C_TIMESTAMP":"2022-11-06T02:58:00"}} +{"c_map":{"OSHIu":"FlSum","MaSwp":"KYQkK","iXmjf":"zlkgq","jOBeN":"RDfwI","mNmag":"QyxeW"},"c_array":[1632475346,1988402914,1222138765,1952120146,1223582179],"c_string":"fUmcz","c_boolean":false,"c_tinyint":86,"c_smallint":2122,"c_int":798530029,"c_bigint":4622710207120546816,"c_float":2.7438526E38,"c_double":3.710018378162975E306,"c_bytes":"WWlCdWk=","c_date":"2022-10-08","c_decimal":21195432655142738238.345609599825344131,"c_timestamp":"2022-01-12T10:58:00","c_row":{"C_MAP":{"HdaHZ":"KMWIb","ETTGr":"zDkTq","kdTfa":"AyDqd","beLSj":"gCVdP","RDgtj":"YhJcx"},"C_ARRAY":[1665702810,2138839494,2129312562,1248002085,1536850903],"C_STRING":"jJotn","C_BOOLEAN":false,"C_TINYINT":90,"C_SMALLINT":5092,"C_INT":543799429,"C_BIGINT":3526775209703891968,"C_FLOAT":1.9285203E37,"C_DOUBLE":1.1956984788876983E308,"C_BYTES":"RVd4a1g=","C_DATE":"2022-09-19","C_DECIMAL":86909407361565847023.835229924753629936,"C_TIMESTAMP":"2022-09-15T18:06:00"}} +{"c_map":{"aDAzK":"sMIOi","NSyDX":"TKSoT","JLxhC":"NpeWZ","LAjup":"KmHDA","HUIPE":"yAOKq"},"c_array":[1046349188,1243865078,849372657,522012053,644827083],"c_string":"pwRSn","c_boolean":true,"c_tinyint":55,"c_smallint":14285,"c_int":290002708,"c_bigint":4717741595193431040,"c_float":3.0965473E38,"c_double":1.2984472295257766E308,"c_bytes":"TE1oUWg=","c_date":"2022-05-05","c_decimal":75406296065465000885.249652183329686608,"c_timestamp":"2022-07-05T14:40:00","c_row":{"C_MAP":{"WTqxL":"RuJsv","UXnhR":"HOjTp","EeFOQ":"PSpGy","YtxFI":"ACjTB","YAlWV":"NlOjQ"},"C_ARRAY":[1610325348,1432388472,557306114,590115029,1704913966],"C_STRING":"Pnkxe","C_BOOLEAN":false,"C_TINYINT":-15,"C_SMALLINT":8909,"C_INT":2084130154,"C_BIGINT":3344333580258222592,"C_FLOAT":3.3306473E38,"C_DOUBLE":9.233143817392184E307,"C_BYTES":"enpuUXk=","C_DATE":"2022-07-01","C_DECIMAL":87998983887293909887.925694693860636437,"C_TIMESTAMP":"2022-02-12T07:45:00"}} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/e2e.json.lzo b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/e2e.json.lzo new file mode 100644 index 0000000000000000000000000000000000000000..26c577d660596a38b15532bcc22957585ef2853b GIT binary patch literal 3466 zcmY*cOK%(38Kq=dcHC&3#z~MAAmC6=q{u`1zUSUi(uNW#k)lLWq_vo%8vJ(@VL~z_ssOd*Owf`RCs4xg5Iq^Mfa~-+$}J7nknpr9Xy3FV2KQ z0}G+h=m(L?@>Y2#;^U$+TPa5TNPcg9Js)u+ot>$rHt@`TuDSrcI59Rg54@3Im@EKq z?9>ke;LXZfeik^nlgSi->uZJ8g~)vuE0_27%AJV+p4Oh{DN2Y-73Yj=p`{CG2oTY( zB1BOkJ?2uOJ>xNe_w54vt-X!r8Wx?8?GP*KY0Y5(<^`1JGz#@dqEcTRf-jOQ_?gfJItq$f{$LoZ9zhe1Z%NX>)+K*`6K zC#G|8tj1!J3#kM@^Fz+~aw1n6o5t7CnADmJCNwj~c*dZ@M0c+}zZfKK69mb*=^#~l z=j4^1L_A0i-R$o{@uV$Yay*T*MSL(}BqN?jC1>J-1`^{`Nku%up*7Bhp!_%CzL{h( zh|lCvJV!;T2pIPX{*uW3``6xj;)G7(nT^>)?0S$+S2w}W(OSE?1iX7MRp6>Fa_1cyYzjo-O*M^=t zp_ix;TvHG!(H~UV?u=ncr8VQ8k<@rrT;klhqNBtF#Q2=4b0;*qJ5wzkSz;QM=@}5S zQaPAN1MjYFP6fb?z5Ir4jj2Mlz6yM_RoJk^w|8?HOLR7wN=yN_j;2;hE>?_SHwXo% z0BJA^VK|{K^$2vPFIXcG0fmnd*VtB8Lcf37{HonM7q4d49^4)J*lF#xzxR7qFI-M%(#x7JAI{Q zwozEvrsqzNwS$MRf!)vzR8?qU<%k2@XQ_x$Vn8{VBT~Suq&|IxoICw2N_D`P&#^>k zbZ)w_YkezsP}`--IV!M!%{;^t)D zc1^CZ+g^=g#n*9QZqQ#IXp<855C3ypL5e1CJ$9~mF-?^gFyhk>zWCAslnmmL z3}vHI^VpfwS4piOEzQO085WKbdG_ki8N`O>)rvv165(V|g*wIz0*NU!j^t9HG0gD^ zNrn4_l^a!`eauqf&}b*UTCi=n7K|mQLBn>bnYQYgY;6^ffYX!h12}Rd=#1wqO=AZ^ z36>N|tu;$c;F*cNoDH|ieo)(WyQm;1%Ahek85#Z=l36%&nOd&^Q#E*@a|h-4OL zH#5K+3tMPRD6O%%whLU(jnyp0g~aMX8h9(aRkoDPb`ERv(NiZhs8Bx-9Okgbfu_CVWeX{&Pd;I8?(qgC`L4rOQm7_7Q|h+v49{O;b+ z+n|jxj}n+XRzRj{`snK4|J9H5hd)WC9{9su=PHlk8TcHGb}%*ABatNC{x`q4-8Vnq z7_ZK65^P3ElxPSJMSh7Ap?&O@DeF0&i`$RR>?KMY;gmKvL$nEzax3myvUxexzx8B*acx@@ql zxE0#YL7~`05_Fwgk8Zv7BVV5UdHD0&J*d5Y)Z1<#7;^0JAu(0I;qu|f&ZQtpw@dR~6UF*>&}lb3DF{eGctW@m zJ7Cz1A$Jto!DcVgRHV!+;7OFMNJG^!j|TduXSPLVIfe9tbEcG}rB5pUTw( zn-0m^L~a&1U)-6rp}E*P9Q?i;x)dL;2X;nZj5p@z@O!qAtIx(BJE5D%N2_X32H8Vp zIKGQ$hucv!t2EZ4(YQ8O;VaxI<(@gc`R3+f?Ylp9P|m?;?ocY+gBUPF$Nm+_5gkHc zHMbu-185h{(b(`UjlP#pKKQhK3D$fqv5aCMb9r{~IvKDjXG~Eyu37fMlUkj4voUu|bcJJ@yBXq%0twFXYz~ L-M7E~FW3J7kMdyE literal 0 HcmV?d00001 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/fake_to_s3_file_json.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/fake_to_s3_file_json.conf new file mode 100644 index 00000000000..3a22956f540 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/fake_to_s3_file_json.conf @@ -0,0 +1,86 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + S3File { + path = "/tmp/seatunnel/json" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "json" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + "schema_save_mode"="CREATE_SCHEMA_WHEN_NOT_EXIST" + "data_save_mode"="APPEND_DATA" + "access_key"="XXXXXX" + bucket="s3a://ws-package" + "secret_key"="AWS_XXXX" + "fs.s3a.endpoint"="s3.cn-north-1.amazonaws.com.cn" + "fs.s3a.aws.credentials.provider"="org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_json_lzo_to_console.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_json_lzo_to_console.conf new file mode 100644 index 00000000000..df99cd66fb8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_json_lzo_to_console.conf @@ -0,0 +1,144 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + result_table_name = "fake" + path = "/test/seatunnel/read/lzo_json" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "json" + compress_codec = "lzo" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + C_MAP = "map" + C_ARRAY = "array" + C_STRING = string + C_BOOLEAN = boolean + C_TINYINT = tinyint + C_SMALLINT = smallint + C_INT = int + C_BIGINT = bigint + C_FLOAT = float + C_DOUBLE = double + C_BYTES = bytes + C_DATE = date + C_DECIMAL = "decimal(38, 18)" + C_TIMESTAMP = timestamp + } + } + } + } +} + +transform { + sql { + source_table_name = "fake" + result_table_name = "sqlresult" + query = "select * from fake where c_string = 'WArEB'" + } +} + +sink { + Assert { + source_table_name = "sqlresult" + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 1 + }, + { + rule_type = MIN_ROW + rule_value = 1 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + equals_to = "WArEB" + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_smallint + field_type = short + field_value = [ + { + equals_to = 15920 + } + ] + }, + { + field_name = c_date + field_type = date + field_value = [ + { + equals_to = "2022-04-27" + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_json_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_json_to_assert.conf new file mode 100644 index 00000000000..7e2491d2f6f --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_json_to_assert.conf @@ -0,0 +1,133 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/json" + file_format_type = "json" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + C_MAP = "map" + C_ARRAY = "array" + C_STRING = string + C_BOOLEAN = boolean + C_TINYINT = tinyint + C_SMALLINT = smallint + C_INT = int + C_BIGINT = bigint + C_FLOAT = float + C_DOUBLE = double + C_BYTES = bytes + C_DATE = date + C_DECIMAL = "decimal(38, 18)" + C_TIMESTAMP = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_json_to_assert_with_multipletable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_json_to_assert_with_multipletable.conf new file mode 100644 index 00000000000..741c26da8e5 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_json_to_assert_with_multipletable.conf @@ -0,0 +1,130 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + tables_configs = [ + { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/json" + file_format_type = "json" + schema = { + table = "fake01" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + C_MAP = "map" + C_ARRAY = "array" + C_STRING = string + C_BOOLEAN = boolean + C_TINYINT = tinyint + C_SMALLINT = smallint + C_INT = int + C_BIGINT = bigint + C_FLOAT = float + C_DOUBLE = double + C_BYTES = bytes + C_DATE = date + C_DECIMAL = "decimal(38, 18)" + C_TIMESTAMP = timestamp + } + } + } + }, + { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/json" + file_format_type = "json" + schema = { + table = "fake02" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + C_MAP = "map" + C_ARRAY = "array" + C_STRING = string + C_BOOLEAN = boolean + C_TINYINT = tinyint + C_SMALLINT = smallint + C_INT = int + C_BIGINT = bigint + C_FLOAT = float + C_DOUBLE = double + C_BYTES = bytes + C_DATE = date + C_DECIMAL = "decimal(38, 18)" + C_TIMESTAMP = timestamp + } + } + } + } + ] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + table-names = ["fake01", "fake02"] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_to_console.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_to_console.conf new file mode 100644 index 00000000000..e0d0019f17a --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/json/s3_file_to_console.conf @@ -0,0 +1,46 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/tmp/fake_empty" + file_format_type = "json" + # schema is needed for json type + schema { + + } + } +} + +sink { + Console {} +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/orc/e2e.orc b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/orc/e2e.orc new file mode 100644 index 0000000000000000000000000000000000000000..d50f6bb54dd6af1f3bdd521dcb829d989c503c78 GIT binary patch literal 5730 zcmc&%c~nzp7JqrkgU18m5f&d&NW|KD>?x8EvYnoG*h1Js79a_yqfJ7{0$Cs-5ZW4C z+N!11Dz2lB$4-k=M{C{MTF=yR!M0=7sdcICap~sNx=^QDZLQAtK3*7e)$P6OHm#gWb}d}h{r#eiw+_$`e3)P%+zB|w z8y5!(arTO0UTHE|tUal?Qknr2uQbhOtL{F>*Pc6KTG$s(p7wq_=Ipr=2IuIX{xCS_ z9eA<>$3zw)Qi2P&4T`aT)(u)3as}ZjRJd>H2_*~3;K-F zMx#hy;Bf)F#$Kem-)O{v8;1@KAGRq2&)}xu{TmLhWRY*Izi=ptI{5adcMj7{x_gea zMr$WX#pQ3)*F>2kdSV7^0Op7yUtm!;=o(P)9xfP%1pM?Gl-x6yP4yhUcaZvTUw$oy zy!`3vcN3}WM`v!408at*EMbumz8;f>gLKCh&By@PPaze zrqeY+461V1`zz{OiwoQ~Z(GpSZr4>dG}gK6bbha0X|eg-7OPk9RtFuvHch)lZ3z0> zJ5@H-ZVE%%bvB*b)8?!7>a~?+)w=ekPD4#~d7Y`w(@|CB(1D2BTARmi^cnS45Xr1n zB~9gxo|dwjfU7L1bJljM8|s`DDx0OTx}wxx=5w@#LdswTL_bfn+TR>>*^3Hlm4UL5 z-Ww=4m{d)T#%67!UP&i_Vz=~3ix7*lLRSGHPpfm74616S)(N2x#qRX~YGG%n9-GFw zB)FO089cc%ttZGa?wGzy&R;kF?dwWS`2MI4zbelPXO0~Dg3)_1CWw*x8Otqik!Qo{R>hr&;u{P4N(iLz&JijZl~+_?~* zo%PpUi%*;mNASwjzdC%7|7AD=WH(Oqox1gI_q^=~zPPjdXY`xn`_G-exOc(gtt&Pz z(XZODbnlWw?;N_g`egT+W$4_sv)6Zh_IXd=lApKr9%mi8y?^7b7iRtaXwQ;=xlvM8 zQROTOc4!KH>Xx9@V5zSpm3 zUGD4TFU$E91*NO6Epurh))cGUenW@XY&1Ko-6eGw#iTNK6#A5bD!P02fLiNlFDbFrch~~f4xPaoa=Y9nr(0<%@w@9iO=dTQ zI-AW=*$^smX*!x43;n9vYFkUKvcz3e=&rOn>mdvlw^+2z`UZ`r-PurHq^>Pfw^W%t zUbow?)3)kuMRl&)hK`boV6k3R9CDYH>)V;};XU>=C`FLk*_)zC(Wmu4v8Rv7eEA)R z=J{@cKuM7=DOzuRwBY5n8967r(_+8cCR8pxbxwSkdcE|An47119zOrYk^av&btV4( zz4N!9pg!*2mbYtLm)*#@*0u4CH^sX@-t(-gaPgVab?YTdpRzgXJFPB#v)uq;)oy61 zG(ltqW4iuVIlm|&Z5n66w2dn_bnYwMc{a#7+BAl@>67=jG(7WxZ~2^#%JZ>K&kWAi ztyizjnYkH3dT#O0E!>TK`-tJG>09R_zdc{Oq3qg8WX*B9c+(pSB5^TM4^7Ixj~KDlM{3iMAKR=@t)-2J`lSN**0YBzTIwR7jbJ>9)}=E}L3 z=07}tU+?Po-{_w+f9dkxALp01iC^sFxb zW`$*$YemO0WssA#LayUKTBQ(FE?y+~`0P*oj>Ny9;BO}Q4+WucC|Ohjlqe;DlFdjC zqsJ&N)W=YKC}WvDPNl z_YmVh3Fe2AED8m77Ku{N0f_?1fqOQyXTv>8a+q@%g+Y4^tPe`YF#1?V;#3T@$H8~7 zNCMu+BIBtkP@h1hLOD=B7MVz?z#ao~Ade&_ZW6O6nK;QX4n`(JJxZo9_EaX%R3`5< zW}n97oz9#a3-d-virHsSjW7=$b03eL{aMUA_{=*5%sU3^%p$X?GWd>c z#&04McOnz-Atufw<{gtLl#AjB_-Hxwi|A+>Sx4;AboYxhns3y2yfFoSr+KR0VfNS^ zW+%}XOAOEy&?gOILo86sCf=8F7*m-;kw4SX_=a&Z;d)S1!&pub z5tac825K~vVSsV~o;RFZ7^&#}97ZUH^m7=Qh|*6$B0xVQO&&KQc`!Ccntm^D_)pUT zWf*MwAYA}W2W079sxa8~C^P|deLV0Ofg;@1^pW_%VACUi1f!7y(DbGU#tV$qQMx`z z2L`)-4-qg@IGg-)25$@y-{8~_ae@;uldHa{g(_`nwS*KgP-e#NGLoWkPPsr&B@Caaco12^IHSnbcaIrY; z^a@02Z1$R=s=?c4X|~a>4K}M2xX@NhC}8vRVt~#Zu<_#Qux^__V0QcHV6EQv>=-Q~u#A-Df2u()rhuTX zV18c}qkwwE77f$~{fC0OVnk3^39ntnx9VS%NVw6AdJJQqKCxvfSA6#eaANWEII=^BI!tq8WH74DT<0ZxAK}O z0v<9VMAH<7E#97XQIx{n?e5V*TiImwo1pC}scx)=)>v0k7=YI0FKcOowy9cQS_rMr zR${4yR@YeX@MLSuJ%BTnm88@$kZTFs79Xj|%x?iNv0mR4U}>OpUO zwY3Hp>+)_f{pssZjEE+3Sqv`wJvtMZn$_OhZgwMv*61xk@XDg*t{!Ns%tmb!wAyZe z6Z-Dg8;a5A@`@r&0kjr#d$Cy>CG&MUq}d90xv|*}CKuPZ%4)z9eO;ggy6f8gdM9)@ z7uzl9Nx8kJ*8;7*#$Ss8FV^~O#W209RZ}gUr_gxRm+; zo-A*Bag`HoEiEX~w1L0FBKnGme)7;Czl{?3+cDxBwZ!(>9qF^*e|5h9_R>ke+qFC{ zqoQeYb3v{CwIw;~Q!AOc^iTKD&BJli%>_rwHyrtUxB9D-iw|sa&b;T$eay0_AFsW* zEjp>+)t|X7u0#Cb(>Ki@PoJ9K{CSsRAC*a${pn0*(zd5)FsG5W(rbW(4rExrviebsO~Mc!M2k7o$Wo+sH{d`i8X-Z8nspL0Kutr4E^f=zAz9W zz?sCbIAR#jqI2kTr{4X)$%65Jv9sglfU_8W&)&X$y9rvviJHF;}oE;Hs{M6)dUl zG~ppBvl?390B0@J6}hz->SguSJ`?A{uiOHZ;QGk*6!oHxqMG3M9sE|o59aR@(R)So zpHF_bQ7RCs8L4K2m-4}8>egpBmkbZ(PLlkIK%@-achv*Z-uvpS%9N_7&Z3D~Z^{ z^sSe7y)E1ksfDb%70q8fwMIly`5(r8B$6#3nwxUsF?c-^x$`!5Q%7c|^+5|LK(fF% zA7A)vH>~;iB#VqwU0BiD1u-+!bU4eP^|8U71KB~34mU*CUYcpCk(Dvx7nyaAo>$}a!!S=Gka^&EG3Uxg$k-ghk zsfJcxYxFk5p$j;CUTLSovdmgg4$Rcmt+pcr>52^n(t3HjqrfZuS4ClKi>p&~PnMyn z%3X%*T3YVY=~#bviRtgRzV%AvME**^-!gZ18xp3@SWxGO)~u~4#M57*ZE8jXI~xsF zl4R_fnhJPtQ){%yk)>LV6&>_Cic2xMba@QLkd)-6fU_+sTVACrE5+Q<6euJ+(Op^$ zVUbrF{Cern6!n3Ms(>g)uB&Wq0WV7{EERfqt?2G`Nb42t&8C`G5N>sqXdr=%cUgknIrz?Hp}S!OE1oIe5taGt`R!XGV&qTlPjj4H zytVHS|F$FU(#+iSbN?ZVEM4Wz@yc^Eq;t;nxK`F^^^^aL8d^V~SpU|Go_Ra8Nehqb zW7bYvQAO$Uq;zOPBR?P)7O4EBqZ3ZJTVw7 z-Wh>*w^?l3a&V~DRBXmrnoNdfh;dez*W=U^$6ST2HGrg`XVBNSZrucVgFoo9SsaoG zCS=pO^u~Qd!{9eR;htu0!f7Dai|~6FewX040e+m!v!@SV5z-$oGdPo?=4VsXt8os$I-5;7#}{$k6zoUv3tpAoVD(4~ug{e$0rG?+Hj zC;qxW=9ka?weQbg{i5Ic^TEFN==w$I?v~RFvyIVvL0NH+c z{hV-hmyOhLArzmDn_2a1)!CT9Z&y9O=J(sA2R=3QZ`xscEatOCQOdpFq?j&O?0C|0 zGrBTzwQtURtK3@M8=3jNzG}2ki>|l?#uGWI319>(;{S(BZ=)I2=1YB9@Voq7YhAo6-@n z%HRO0rC@+mc7g#3RxPoN1PdEMx0F;HI3KEzTET#bie*G1L5Bou2>`%yLfDP90(4{P zAne9k0J^a>5Om9e`97AB1yjCYSc176q%f@ux@Ey!4N}2$Ef|)PJPbatX*U!M38q(o z!)z;1^|$B1oI?FF3^ppv~Md;f7}^W%RHZ zjsX=rOLA=rI%2HhWgK1qix+#5@u3tGEncF5Y@arD!{`GfIv z0SCEm4&BT>UI{&ZqT+iG&JB5aY#!e|kPxzVFfjyu>R#TVgGrF+k0ajpkYCfOP+KA$B3IFW`t!Y>IDMoRiv37M^-!|!u?nu0JCz(m;^ ztX{9nVe!D0V&DBvPr&K%v+vO$Tl|g$xxs~|PNm~%VvF5jbAq~rXnG!PNM4SAaU7LRwl(;j zZbygT;%-ZbG4xNLg|UEmkqG*{-SjN-tZ6W+%YSq#vpxRtCDw@z)q|t)9F{6o=8H=jSYGb)Dn#$u_=_xs6ZZ&rol3bQhvUeRw`dxKxa^i zSK$si4kWvmgP=oVm`hMe{$->j9Sc#fFQysgd4hTzuBj2k(4^oAVD~&hdoDuxQbgkM zNqKC}W=H~8Qb{C(vu{G&jX8k(bTXYvC0>DYRvx=3QU-*}9~{7pRsx-D5y3cXA!dAr zEdYc_+YxCn8U4Ny4dLLBLZ^*D9&Y^c0&PpBAAkjd;$Oz;2vQht^o^-tG?r2fsjJZF(3=o&=Mpgb zTq?%(JbnlW8O^ew!07YS!02EN#_)w18Xev%)Nurj?oUIbM--v`8KhRC0=DN?5e$>< zNs2j!t?h(BrxECbG~AwRV?qnYu`d94s># zzV^fbhxZEAyo74jr(=Lm(m}c^g?$X^1*jSl%h)^!ly#CO5hn#K!`8Dx5v3h1m zDUE9zdK01#lh9}2+U}0yH(e4N6j?9}lITAp0(OL1#9C5;Gi*UF#Jh!fr!z1TgYi5y za(GdE0wsVCC<|cH1O)laQrLJKGm#+I&EYq42#m|V1kA2%hDqQxmi&+q;4gX|dJ|f? z3xRiKqSsf#fTKfcoJYQb$argD$Q1&SLvKRlK}0@7kUvq4Hkqvlc+gK&0E%}GMuHa* z1P#3jK~E#-`Ybf};9UNEgcgY{EC?=?&yc?k&LzajURfCa9s?SHI7jyiHCeTE!hiywl^GXFpV zI1s!&G5nPZ9SH9gYI-pb9N4}H9T-d;y`RAWQdbd0>~#u)Z)CWIS6xS>V3 zolo66dO0K6!i=C{mOgbagafZ{47tl;I6`ki)MFTqt=VKflXyZ6?Q5>GBfvu<y(fA|TNUPiDkKQ(u1tlU;1Um;(?)Le(bGVBm!`~(`@Zr5eW#6H) zmli|dl`kYKqZ4Jy-z=B`SIuPRCv6sAd#A&nYU%QMQ#)+U4!4EbKYgXjA82!^a#cQZ zTT&&fJQg>;kGe*|K%>*)vUjL*A6yxnLoQV?!a_W#@^rdfs$?D#Zd@=(?ocSk4XfA- zlIo$=4<)PYjz&wT%dg5s(bXWzY3D9l<71OM7YsZ74z~~-E=qSo=Jby?o40EstGIhy zm|d#iz=S?=1!cH2v3erC!TaG|h$42FuXj|)R)OJyF0ZFaHKCK_dKxasc26kD7IERC zT>pf!rv%q<|L~+z%-P|nb;=&WgY`PWVXC zp_*6?6hz}I7|k-I>VxUw+(IhB@qvB>w{X?k_}F*NEx|%94Ei777QQG87ybvhh5rT; zE`5ii9xAi$LKHrv+=UcdD1{5&iCehBCtUE3+!Fk-6E1pZZsDqxaM3$*3tO0k3H~^4 z;lDVIk2{K6gcAQ8ZV?nQ<6CvN+%iU;@X%_CncnS#-0w?ud+iQaD$*z;!P16ebX=vA z;`YUNIef^I@#(4Q8L64^N!CuM%O2mDW4Af%={Xs;EL(b`6{t7cX0v9d=PX*3x!AhM fvM}9|?XYKC?}M*&@B#Fn^pAQSzF?Qb|Ni=abvwAB literal 0 HcmV?d00001 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/fake_to_s3_file_parquet.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/fake_to_s3_file_parquet.conf new file mode 100644 index 00000000000..8738d35db50 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/fake_to_s3_file_parquet.conf @@ -0,0 +1,87 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + S3File { + "schema_save_mode"="CREATE_SCHEMA_WHEN_NOT_EXIST" + "data_save_mode"="APPEND_DATA" + "access_key"="XXXXXX" + bucket="s3a://ws-package" + "secret_key"="AWS_XXXX" + "fs.s3a.endpoint"="s3.cn-north-1.amazonaws.com.cn" + "fs.s3a.aws.credentials.provider"="org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + path = "/tmp/seatunnel/parquet" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "parquet" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + compress_codec = "gzip" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_parquet_projection_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_parquet_projection_to_assert.conf new file mode 100644 index 00000000000..6d5e7cc8850 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_parquet_projection_to_assert.conf @@ -0,0 +1,82 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/parquet" + file_format_type = "parquet" + read_columns = [c_string, c_boolean, c_double] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_parquet_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_parquet_to_assert.conf new file mode 100644 index 00000000000..db3619a33f6 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_parquet_to_assert.conf @@ -0,0 +1,99 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/parquet" + file_format_type = "parquet" + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_parquet_to_assert_with_multipletable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_parquet_to_assert_with_multipletable.conf new file mode 100644 index 00000000000..ba9a8037ab2 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_parquet_to_assert_with_multipletable.conf @@ -0,0 +1,66 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + tables_configs = [ + { + schema = { + table = "fake01" + } + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/parquet" + file_format_type = "parquet" + }, + { + schema = { + table = "fake02" + } + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/parquet" + file_format_type = "parquet" + } + ] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + table-names = ["fake01", "fake02"] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_to_console.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_to_console.conf new file mode 100644 index 00000000000..330ce7479ce --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/parquet/s3_file_to_console.conf @@ -0,0 +1,42 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/tmp/fake_empty" + file_format_type = "parquet" + } +} + +sink { + Console {} +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e.txt b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e.txt new file mode 100644 index 00000000000..9871cd85eb6 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e.txt @@ -0,0 +1,5 @@ +uDDrwsQQYONTNeUBIOnLAgunvDqLBObroRzdEdvDgRmgaeFyFH5456857591576298739157764687713794636442057612252MTDnafalse3313846190943192276641872220071936002.4798444E389.52375328387482E307vcIGF2023-06-0776258155390368615610.7646252373186602912023-05-08 16:08:51ipToEdierOAbwQfQzObWqiRhjkWYaMKdCbjurhstsWrAVlRyyR2905930362869031292782506910815576701385108050hArFutrue12631169122166306155952414159791708165.949173E372.1775762383875058E307kMlgO2023-05-2027214280267865241887.6424416000104182532023-10-20 03:49:02 +QIpzzZNFkLwARZDSdwdBzkegCdIRVYJnuXgxNXytAJxxaTzmDF16603816781145850255103997497062535321459349811xaTOktrue5327578191749099325840234439082792961.955231E381.5072154481920294E308GDWOu2023-05-0581449039533149712064.4515003874168475032023-07-06 22:34:11sfgxhqvOLzjdTSNcNaWfEnZqvQraSSuMPazCGhPmSrGuxggqGh111449466287130860562118177510004750271267350957FDhTstrue96247293946402921952995131535667203.3240283E384.473485404447698E307YFdwf2023-02-0429456519357128996647.9939318900994572132023-01-12 02:29:58 +xVJPgVlosBlTYSkmJCqKHMXzbZkNQKInuVMZeYGhsmzUmcLyPx137745493211075991209783701051546835517166168384qcYaifalse8318050110096656524405690917018449922.9617934E371.8901064340036343E307jaKMq2023-05-1275317114043170470995.9654034735914367862023-05-18 08:09:22raGGBnHsNwMZKemkFErUbedNjSllNcKOVUGdTpXcHGSVphHsNE86377304018502081846122308810391870441519757437JCRZStrue1829974183977114228752256792969205767.9090967E371.6286963710372255E308NBHUB2023-05-0732934086493941743464.6503746053883129532023-05-06 04:35:55 +dBgFeTKkCfnxCljyGfNEurEzCVgwpsHgmcOfYXiQHxeeQNjQuq1961913761867016982512369059615238191571813320BTfhbfalse652666522281866957533025299230722.1456136E381.2398422714159417E308YOiwg2023-10-2433001899362876139955.7235198795513055732023-06-23 13:46:46jsvmHLHlXCGFKwuqlTwAjdMckElrmqgBWvOuuKuWxcinFZWSky19959088245502706421265289671411088181469730839vUyULtrue952655754382886132164227350822215681.9033253E381.0966562906060974E308XFeKf2023-09-1731084757529957096723.2394423349193989032023-06-15 17:04:50 +obtYzIHOTKsABVtirEKEMYUYobsYlDJcFbpQUYvGxCcKlnswEG8096984004544201585383739017658796661353001394xchcntrue853141253976762312923177914159380482.8480754E381.055208146200822E308MSkTD2023-11-2420361788179232141281.9718823433892185262023-10-25 11:47:50gdCWZMGESyarjQPopBhDwKnOyDvaUDgQOEDRCmfUAagfnDDPqV8473436731118772451890654127233667151574025969ewJzLtrue6321769209768782446484076920790579202.7134378E381.1883616449174808E308STvOu2023-10-0821793351767634029460.2897683013563753232023-08-12 23:57:38 \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e.txt.lzo b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e.txt.lzo new file mode 100644 index 0000000000000000000000000000000000000000..3bd2782c6676b911dd0e45cdbaf2baaaa7b6998e GIT binary patch literal 2720 zcmXYzyKiGz9mnrI=dwW9m6mQbQxJ)kSm*t)&}w2|KN81w9NS6kYW;TNM`C;Y?2srX z5*-p1#THXgOhHKj1ro&+E5skLe*!|3NKo*--pxaDuAO_&`940s^K#X$2A`e0I{Ezb z)9W9<34ahAzkc<8@YkRG?)dT-Q~FI1ocsm9^Wb0WI9_h)s#a@NNu^O4G->9dI=>?4 zqxF14*VYK{tUSK7Mj2-<;nI3# zq*00phEH%Iv2N*QwLlnJ?KUP%yKUcW1&bX>zz_6 zCtc`MT1cyXd1zpY1+I z>*aX$maguW=eN=HX1~87!ZYn9lhg=j@L(>e7uE^QjHjF>>#@c%icM?$$uuM5^JRWb z-Y(Yz!i6D{a|02D;KoR1IMLc`p}6AOThA@-VTew7h1W7CEfFbhZ6d!+PDb27~t&~HS;4cEhTv&+js_XB=0kd`JxWXN!*HUujEf>tF zl+s)?tk^;ory!+7IF=T9Iu#-&_v{7RJ{zT3;5{yJ$386vhxCd8#r-P-i*TWtc`#ICqJIhzDcIQ-!4nf|Z(i zZAm^JH&%a+gI{_>gtEdDiCaJfPXtFYyw_ZE=xB_E(=sIm-gJnDBiNFa%4x;W9Bce{ z$!O*K+u{2!gI7$PF*MVo#Tq@+kimJ6_SqD)_Na(wP+e&#`j0qRF@DCyDHCbo)7qWv zZZEG#^mh8R%8+TJRiDt=Wp+^_#q#~3OWTu5twb*7>)SGI4_es%YIbPOXzyx&y+i4( zRa*JPe9k}+4-1I5PFkc9A<-a+)S3e##=s6hB3$%ZoriB;f;SG-NV*f^e;F2x^@qqZ z)2Lih5e`y#FF*}%xJ9(k6GQ{s;qX~1m~5niM8L#jr&L~qzyB(D%`~eFeZM@!Y)W+Cb~p6`3KSg7AVM`L_GN1TBkk|}^8X$d;+s7i2&kT?WGdH@k7QVyfg z35DKYW^dZ{@44aNB05F5lint)7es&sz^yrU1XTbb2z+JuHa6h~PaPSBC*%I0t=36Z zNF`HjX8x87zCcebkpk)=sBp?Fv``{?DT8KMh44z}Ksf)OmtX<$P!p!oQm3^(>1Rgy z0d15f*&&(lveQSpFATBK^)h$J(%aGYc}0q&S+7cmt^0=>DeeY?8m&Cm)(h0#pz#vp z0=mFV;Pg@n5pH6S@q$B%;s&5=^DFw+>0yRhVS5(I-lgqR)>(HI|MK1m7kKq1I! zlT?5=hD-C|CHRq$7%M{9BVVz@BQ9FihwbR=FN4AhH5Q%0;2!>SwUs|WEknqA; zyhfFQ^@N^Fj29hoCa@0R8k47CiNS8YBbE;2U8&rr|z6C=EZx zC;kjglz1QjWqjm zo9g;Q3>g?sO35v~MEVm^K?e*F!%|2ko*?0Y22wE9{~4dSWL^lF);2BOPro~e{vXl< BZsh;~ literal 0 HcmV?d00001 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e_delimiter.txt b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e_delimiter.txt new file mode 100644 index 00000000000..b87687448ca --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e_delimiter.txt @@ -0,0 +1,5 @@ +qwerqwer|1972607327106509113020400507301104442513849629249|qwer|true|108|22432|11383204|723560014108175360|3.1407707E38|1.262116635132156E308|zlmzw|2023-05-25|97236477433882034782.803540569732795689|2023-03-25 04:30:13|qwerqwer1458583961104266156763552401211382922561937221393qwertrue930925142792030530244095935039344647.838737E373.3238256808030654E307Zicjq2023-10-1918739344608215707574.2737367351403166822023-10-07 08:24:27 +qwerqwer|20734545375230101131603368534223532992574063143|qwer|true|99|21567|768189694|8504422836686883840|1.3761162E38|5.460153079423635E307|dkCwG|2023-05-19|83044404421834652395.960138696348105704|2023-03-24 10:48:12|qwerqwer2774295104069855819185865051778415509162817756qwerfalse1619571127265647324402356645454202881.8446726E381.7000909191489263E308cXxQV2023-07-2713431695514477025331.5815661990272672962023-12-22 12:26:16 +qwerqwer|11147903451235598576860383707165213199232994316|qwer|true|49|21122|1110303282|2083282743100007424|1.9729736E38|1.0399541425415623E308|muvcN|2023-08-13|68941603382218317993.487441177291093700|2023-04-06 02:40:57|qwerqwer69745783829424948385550024313502468211004949206qwertrue117227855844811138143962162044856324.844609E374.992962483991954E307pPYZS2023-05-1751345924758748590630.6631664051742477762023-12-10 19:23:26 +qwerqwer|12600145717385486047323762331460409881387559257|qwer|true|54|30782|475296705|6520650210788816896|3.253564E38|1.181636072812166E308|RxBAU|2023-03-14|94882795877228509625.376060071805770292|2023-02-25 15:29:26|qwerqwer17078206571395918506189177703116985975671620089209qwerfalse11415353139002758476082670167752366081.4806856E385.82327433457546E307ppTVu2023-10-2784302780955330822761.6237458260160280852023-08-23 09:26:16 +qwerqwer|10811140972103212018816962034437650301336224152|qwer|true|82|27637|1110251085|806786601324796928|7.711023E37|4.398648945575819E307|kGVbL|2023-04-26|80164231813502964946.202647535547152674|2023-04-15 05:22:59|qwerqwer800727634149093075168463891515323059061714847070qwertrue351280654957024134756885372412119043.0538885E384.631561190310559E306leTTG2023-11-1490016690865756655359.8578360402194859042023-08-23 10:30:18 \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e_time_format.txt b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e_time_format.txt new file mode 100644 index 00000000000..b53cbf2e995 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/e2e_time_format.txt @@ -0,0 +1,5 @@ +PgxkWerPquZxADbwRoyZXWZYjOZGvPkcRgcvBHHlSezTHszfCM9312330451016879764123434177920993828271669125781ofduatrue02080228995824162301554221532246172.5053808E386.563348638622289E306KjJjD2023-03-2048955630047163560901.2848899427908584092023-01-11T06:25:29OxqxAMLLAWTMzSvpVKDBXwVuLuVMdhJAbNYRqEmrsQBARdHLAo98774360215016679841391554731369111688804353367rsgcotrue1212280190712202440694969264535828985.9302515E371.2125301856008725E308tVuZI2023-08-0322004483923120397310.0486453397455656992023-06-10T17:15:02 +zxMhGtbuHzxGFwmfFHIUAFvvTgUvQqetaDxOzAavJELHDSdPEV101244983376266331014538704017395315171492457270otcMntrue1024860139917168158893375714893248001.4333913E381.4334353544948444E308VdcYj2023-05-1991883965802194963022.6890574501331289452023-04-26T00:46:03PRIEJkcMnYJRsURrfhCbSgtGebklCfMXxzhZOZMudVetgtUCXc7736457412116475204164682112718260472701764785855oCRKRtrue851925389193674632146772472708622432.5017376E371.4791889801142986E308KIZKN2023-09-1334541234299674175851.0304104953008357352023-08-21T23:52:24 +EIYLFVjmjZXKcbLQtzXKMzIqLccyubcQygIssDqfcwotNQDdfH1836526392121945431313063532901700703821233811949qIlEotrue92987344051191848244308123217417652.6358307E379.12573038650651E307wrQCE2023-06-1169873404793136392100.0758355471497874132023-02-25T07:13:57IRAHziGvkRHEaUmcameBKDUCNFEjmKaafwSGblGdJGGyzQivvd12711189912021715577886030065553480147504046565RpOswfalse122122444030768933777305146193430843.3350248E381.2526133143299848E308kzyBq2023-07-1557715748983349653587.0631369056378550372023-04-28T16:02:28 +tfaoRtCwuXCoiKkBcvPOoixYBZnaUlPQMFaRjxhigVLzmBrskw190529529814451211117678789994558371211783348ccGkzfalse1112829829981778213199660821898045981.2857434E383.343575138440927E307SsSaC2023-10-2658282015679301802224.6155516408553745142023-01-26T13:15:35IETWTtUXEMkdNCiBvZPKghKHXjQUvSMaMsKYCmzsLRjFhEQXyv76798692084328842150475226014007534741586287890wbzKKtrue6625604192054124846725009551245517061.307359E381.6429413197552776E308QdOjL2023-02-2257671928068543569766.1712121225441028432023-03-28T03:01:44 +hdTngggfdRvAAMngAsZUYTEQuTFQEHdIzjOIEGIoYrTYZLIvey760974310142710026829414079475220181644270624MLIllfalse3622155133605466673524332669773532608.235333E379.308989713025347E307nrzoy2023-05-2418552644397825116718.5869443937920164442023-01-08T10:11:24bzXNzJVCPXkxeiQSpYXaVJoHWTJnKJbeIiuknfLOtQAGrKUoFr5144561031691489776106356671519647880411104465196MwxgFtrue501558654910648188780747761689955443.637149E371.4784398529023391E308cZRyO2023-06-0419268168651664178359.9430267663053671912023-11-28T19:35:41 \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/fake_to_s3_file_text.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/fake_to_s3_file_text.conf new file mode 100644 index 00000000000..145081911a2 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/fake_to_s3_file_text.conf @@ -0,0 +1,87 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + S3File { + "schema_save_mode"="CREATE_SCHEMA_WHEN_NOT_EXIST" + "data_save_mode"="APPEND_DATA" + "access_key"="XXXXXX" + bucket="s3a://ws-package" + "secret_key"="AWS_XXXX" + "fs.s3a.endpoint"="s3.cn-north-1.amazonaws.com.cn" + "fs.s3a.aws.credentials.provider"="org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + path = "/tmp/seatunnel/text" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "text" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + compress_codec = "lzo" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/fake_to_s3_file_with_multiple_table.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/fake_to_s3_file_with_multiple_table.conf new file mode 100644 index 00000000000..70e90d7e50a --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/fake_to_s3_file_with_multiple_table.conf @@ -0,0 +1,128 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + tables_configs = [ + { + schema = { + table = "fake1" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + }, + { + schema = { + table = "fake2" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } + ] + } +} + +sink { + S3File { + "schema_save_mode"="CREATE_SCHEMA_WHEN_NOT_EXIST" + "data_save_mode"="APPEND_DATA" + "access_key"="XXXXXX" + bucket="s3a://ws-package" + "secret_key"="AWS_XXXX" + "fs.s3a.endpoint"="s3.cn-north-1.amazonaws.com.cn" + "fs.s3a.aws.credentials.provider"="org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + path = "/tmp/fake_empty/text/${table_name}" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "text" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + compress_codec = "lzo" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_delimiter_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_delimiter_assert.conf new file mode 100644 index 00000000000..af82ff5b644 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_delimiter_assert.conf @@ -0,0 +1,109 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/text_delimiter" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + file_format_type = "text" + read_columns = [c_string, c_boolean] + delimiter = "\\|" + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + equals_to = "qwer" + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + equals_to = true + } + ] + } + ] + } + } +} + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_lzo_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_lzo_to_assert.conf new file mode 100644 index 00000000000..44b1a690861 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_lzo_to_assert.conf @@ -0,0 +1,143 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/lzo_text" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "text" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + compress_codec = "lzo" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +transform { + sql { + source_table_name = "fake" + result_table_name = "sqlresult" + query = "select * from fake where c_string = 'MTDna'" + } +} + +sink { + Assert { + source_table_name = "sqlresult" + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 1 + }, + { + rule_type = MIN_ROW + rule_value = 1 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + equals_to = "MTDna" + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_smallint + field_type = short + field_value = [ + { + equals_to = 13846 + } + ] + }, + { + field_name = c_date + field_type = date + field_value = [ + { + equals_to = "2023-06-07" + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_projection_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_projection_to_assert.conf new file mode 100644 index 00000000000..a7de9c1c5f8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_projection_to_assert.conf @@ -0,0 +1,134 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/text" + file_format_type = "text" + read_columns = [c_string, c_boolean, c_double] + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_skip_headers.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_skip_headers.conf new file mode 100644 index 00000000000..92de5de9838 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_skip_headers.conf @@ -0,0 +1,134 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/text" + file_format_type = "text" + skip_header_row_number = 1 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 4 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_to_assert.conf new file mode 100644 index 00000000000..624f71f199f --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_to_assert.conf @@ -0,0 +1,133 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/text" + file_format_type = "text" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_to_assert_with_multipletable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_to_assert_with_multipletable.conf new file mode 100644 index 00000000000..41c2533423c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_text_to_assert_with_multipletable.conf @@ -0,0 +1,130 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + tables_configs = [ + { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/text" + file_format_type = "text" + schema = { + table = "fake01" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + }, + { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/text" + file_format_type = "text" + schema = { + table = "fake02" + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } + ] + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + table-names = ["fake01", "fake02"] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_time_format_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_time_format_assert.conf new file mode 100644 index 00000000000..cc8c814e95d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/text/s3_file_time_format_assert.conf @@ -0,0 +1,100 @@ +# +# 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 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + S3File { + fs.s3a.endpoint="s3.cn-north-1.amazonaws.com.cn" + fs.s3a.aws.credentials.provider = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider" + access_key = "XXXXXX" + secret_key = "AWS_XXXX" + bucket = "s3a://ws-package" + path = "/test/seatunnel/read/text_time_format" + file_format_type = "text" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + read_columns = [c_timestamp] + datetime_format = "yyyy-MM-dd'T'HH:mm:ss" + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_timestamp + field_type = timestamp + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index 477b0620d28..455f9e76594 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -40,6 +40,7 @@ connector-file-cos-e2e connector-file-sftp-e2e connector-file-oss-e2e + connector-file-s3-e2e connector-cassandra-e2e connector-neo4j-e2e connector-http-e2e diff --git a/seatunnel-shade/seatunnel-hadoop3-3.1.4-uber/pom.xml b/seatunnel-shade/seatunnel-hadoop3-3.1.4-uber/pom.xml index be5ced9214a..322c1ed65b5 100644 --- a/seatunnel-shade/seatunnel-hadoop3-3.1.4-uber/pom.xml +++ b/seatunnel-shade/seatunnel-hadoop3-3.1.4-uber/pom.xml @@ -97,6 +97,7 @@ com.google.common.base.* com.google.common.cache.* com.google.common.collect.* + com.google.common.util.* From 9ec971d942b4d8ace5f71502c19140f15c3abb96 Mon Sep 17 00:00:00 2001 From: dailai Date: Mon, 20 May 2024 16:39:06 +0800 Subject: [PATCH 11/14] [Hotfix][Connector-V2] Fix the batch write with paimon (#6865) --- .../paimon/sink/PaimonSinkWriter.java | 2 +- .../e2e/connector/paimon/PaimonSinkCDCIT.java | 34 +++++++++ .../fake_cdc_sink_paimon_case10.conf | 58 +++++++++++++++ .../resources/fake_cdc_sink_paimon_case9.conf | 74 +++++++++++++++++++ 4 files changed, 167 insertions(+), 1 deletion(-) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_cdc_sink_paimon_case10.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_cdc_sink_paimon_case9.conf diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkWriter.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkWriter.java index 88b3c1fa17d..164c6a0cd9d 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkWriter.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkWriter.java @@ -89,7 +89,7 @@ public PaimonSinkWriter( this.table = table; this.tableWriteBuilder = JobContextUtil.isBatchJob(jobContext) - ? this.table.newBatchWriteBuilder().withOverwrite() + ? this.table.newBatchWriteBuilder() : this.table.newStreamWriteBuilder(); this.tableWrite = tableWriteBuilder.newWrite(); this.seaTunnelRowType = seaTunnelRowType; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkCDCIT.java index 5bec4cd41cd..05fa3db4b95 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkCDCIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkCDCIT.java @@ -86,6 +86,40 @@ public void startUp() throws Exception {} @Override public void tearDown() throws Exception {} + @TestTemplate + public void testSinkWithMultipleInBatchMode(TestContainer container) throws Exception { + Container.ExecResult execOneResult = + container.executeJob("/fake_cdc_sink_paimon_case9.conf"); + Assertions.assertEquals(0, execOneResult.getExitCode()); + + Container.ExecResult execTwoResult = + container.executeJob("/fake_cdc_sink_paimon_case10.conf"); + Assertions.assertEquals(0, execTwoResult.getExitCode()); + + given().ignoreExceptions() + .await() + .atLeast(100L, TimeUnit.MILLISECONDS) + .atMost(30L, TimeUnit.SECONDS) + .untilAsserted( + () -> { + // copy paimon to local + container.executeExtraCommands(containerExtendedFactory); + List paimonRecords = + loadPaimonData("seatunnel_namespace9", TARGET_TABLE); + Assertions.assertEquals(3, paimonRecords.size()); + paimonRecords.forEach( + paimonRecord -> { + if (paimonRecord.getPkId() == 1) { + Assertions.assertEquals("A", paimonRecord.getName()); + } + if (paimonRecord.getPkId() == 2 + || paimonRecord.getPkId() == 3) { + Assertions.assertEquals("CCC", paimonRecord.getName()); + } + }); + }); + } + @TestTemplate public void testFakeCDCSinkPaimon(TestContainer container) throws Exception { Container.ExecResult execResult = container.executeJob("/fake_cdc_sink_paimon_case1.conf"); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_cdc_sink_paimon_case10.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_cdc_sink_paimon_case10.conf new file mode 100644 index 00000000000..3c2061c55bb --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_cdc_sink_paimon_case10.conf @@ -0,0 +1,58 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + pk_id = bigint + name = string + score = int + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = [2, "CCC", 100] + }, + { + kind = INSERT + fields = [3, "CCC", 100] + } + ] + } +} + +sink { + Paimon { + warehouse = "file:///tmp/paimon" + database = "seatunnel_namespace9" + table = "st_test" + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_cdc_sink_paimon_case9.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_cdc_sink_paimon_case9.conf new file mode 100644 index 00000000000..674491f90d7 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_cdc_sink_paimon_case9.conf @@ -0,0 +1,74 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 2 + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + pk_id = bigint + name = string + score = int + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = [1, "A", 100] + }, + { + kind = INSERT + fields = [2, "B", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + } + ] + } +} + +sink { + Paimon { + warehouse = "file:///tmp/paimon" + database = "seatunnel_namespace9" + table = "st_test" + } +} From afdcb3236b7b0788c666af7ad5abaeb9d18aab26 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Tue, 21 May 2024 10:25:06 +0800 Subject: [PATCH 12/14] [Fix][Transform] Fix sql transform lose source type and options (#6855) --- .../seatunnel/transform/sql/SQLTransform.java | 6 ++-- .../transform/sql/SQLTransformTest.java | 29 +++++++++++++++++-- 2 files changed, 30 insertions(+), 5 deletions(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java index ac573f29ce4..a9d04b07396 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java @@ -163,14 +163,16 @@ protected TableSchema transformTableSchema() { Column column; if (simpleColumn != null) { column = - PhysicalColumn.of( + new PhysicalColumn( fieldNames[i], fieldTypes[i], simpleColumn.getColumnLength(), simpleColumn.getScale(), simpleColumn.isNullable(), simpleColumn.getDefaultValue(), - simpleColumn.getComment()); + simpleColumn.getComment(), + simpleColumn.getSourceType(), + simpleColumn.getOptions()); } else { column = PhysicalColumn.of(fieldNames[i], fieldTypes[i], 0, true, null, null); } diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java index afafa57514b..c1002b59689 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java @@ -43,7 +43,7 @@ public class SQLTransformTest { private static final String GENERATE_PARTITION_KEY = "dt"; private static final ReadonlyConfig READONLY_CONFIG = ReadonlyConfig.fromMap( - new HashMap() { + new HashMap() { { put( "query", @@ -69,6 +69,23 @@ public void testScaleSupport() { }); } + @Test + public void testNotLoseSourceTypeAndOptions() { + SQLTransform sqlTransform = new SQLTransform(READONLY_CONFIG, getCatalogTable()); + TableSchema tableSchema = sqlTransform.transformTableSchema(); + tableSchema + .getColumns() + .forEach( + column -> { + if (!column.getName().equals(GENERATE_PARTITION_KEY)) { + Assertions.assertEquals( + "source_" + column.getDataType(), column.getSourceType()); + Assertions.assertEquals( + "testInSQL", column.getOptions().get("context")); + } + }); + } + private CatalogTable getCatalogTable() { SeaTunnelRowType rowType = new SeaTunnelRowType( @@ -89,14 +106,20 @@ private CatalogTable getCatalogTable() { columnLength = 3L; } PhysicalColumn column = - PhysicalColumn.of( + new PhysicalColumn( rowType.getFieldName(i), rowType.getFieldType(i), columnLength, scale, true, null, - null); + null, + "source_" + rowType.getFieldType(i), + new HashMap() { + { + put("context", "testInSQL"); + } + }); schemaBuilder.column(column); } return CatalogTable.of( From 4969c91dc41b16331111a8b9cc49e27298e41295 Mon Sep 17 00:00:00 2001 From: dailai Date: Wed, 22 May 2024 09:40:41 +0800 Subject: [PATCH 13/14] [Improve][Connector-V2] Support hive catalog for paimon sink (#6833) --- docs/en/connector-v2/sink/Paimon.md | 105 +++++++++++++++++ docs/zh/connector-v2/sink/Paimon.md | 107 +++++++++++++++++- .../common/exception/CommonError.java | 9 ++ .../common/exception/CommonErrorCode.java | 4 +- .../connector-paimon/pom.xml | 31 +++++ .../paimon/catalog/PaimonCatalog.java | 13 ++- .../paimon/catalog/PaimonCatalogEnum.java | 33 ++++++ .../paimon/catalog/PaimonCatalogFactory.java | 19 ++-- .../paimon/catalog/PaimonCatalogLoader.java | 26 +++-- .../seatunnel/paimon/config/PaimonConfig.java | 45 ++++++-- .../paimon/sink/PaimonSinkFactory.java | 10 +- .../connector/paimon/PaimonSinkHdfsIT.java | 71 ++++++++++++ ...nk_paimon_with_hdfs_with_hive_catalog.conf | 99 ++++++++++++++++ 13 files changed, 536 insertions(+), 36 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogEnum.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_cdc_sink_paimon_with_hdfs_with_hive_catalog.conf diff --git a/docs/en/connector-v2/sink/Paimon.md b/docs/en/connector-v2/sink/Paimon.md index e0ce1350bb9..d79d7c9b004 100644 --- a/docs/en/connector-v2/sink/Paimon.md +++ b/docs/en/connector-v2/sink/Paimon.md @@ -6,6 +6,24 @@ Sink connector for Apache Paimon. It can support cdc mode 、auto create table. +## Supported DataSource Info + +| Datasource | Dependent | Maven | +|------------|-----------|---------------------------------------------------------------------------| +| Paimon | hive-exec | [Download](https://mvnrepository.com/artifact/org.apache.hive/hive-exec) | +| Paimon | libfb303 | [Download](https://mvnrepository.com/artifact/org.apache.thrift/libfb303) | + +## Database Dependency + +> In order to be compatible with different versions of Hadoop and Hive, the scope of hive-exec in the project pom file are provided, so if you use the Flink engine, first you may need to add the following Jar packages to /lib directory, if you are using the Spark engine and integrated with Hadoop, then you do not need to add the following Jar packages. + +``` +hive-exec-xxx.jar +libfb303-xxx.jar +``` + +> Some versions of the hive-exec package do not have libfb303-xxx.jar, so you also need to manually import the Jar package. + ## Key features - [x] [exactly-once](../../concept/connector-v2-features.md) @@ -15,6 +33,8 @@ Sink connector for Apache Paimon. It can support cdc mode 、auto create table. | name | type | required | default value | Description | |-----------------------------|--------|----------|------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------| | warehouse | String | Yes | - | Paimon warehouse path | +| catalog_type | String | No | filesystem | Catalog type of Paimon, support filesystem and hive | +| catalog_uri | String | No | - | Catalog uri of Paimon, only needed when catalog_type is hive | | database | String | Yes | - | The database you want to access | | table | String | Yes | - | The table you want to access | | hdfs_site_path | String | No | - | The path of hdfs-site.xml | @@ -101,6 +121,91 @@ sink { } ``` +### Single table(Hive catalog) + +```hocon +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + pk_id = bigint + name = string + score = int + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = [1, "A", 100] + }, + { + kind = INSERT + fields = [2, "B", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + } + { + kind = UPDATE_BEFORE + fields = [1, "A", 100] + }, + { + kind = UPDATE_AFTER + fields = [1, "A_1", 100] + }, + { + kind = DELETE + fields = [2, "B", 100] + } + ] + } +} + +sink { + Paimon { + schema_save_mode = "RECREATE_SCHEMA" + catalog_name="seatunnel_test" + catalog_type="hive" + catalog_uri="thrift://hadoop04:9083" + warehouse="hdfs:///tmp/seatunnel" + database="seatunnel_test" + table="st_test3" + paimon.hadoop.conf = { + fs.defaultFS = "hdfs://nameservice1" + dfs.nameservices = "nameservice1" + dfs.ha.namenodes.nameservice1 = "nn1,nn2" + dfs.namenode.rpc-address.nameservice1.nn1 = "hadoop03:8020" + dfs.namenode.rpc-address.nameservice1.nn2 = "hadoop04:8020" + dfs.client.failover.proxy.provider.nameservice1 = "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" + dfs.client.use.datanode.hostname = "true" + } + } +} + +``` + ### Single table with write props of paimon ```hocon diff --git a/docs/zh/connector-v2/sink/Paimon.md b/docs/zh/connector-v2/sink/Paimon.md index ef22add620d..50f88731d3e 100644 --- a/docs/zh/connector-v2/sink/Paimon.md +++ b/docs/zh/connector-v2/sink/Paimon.md @@ -6,6 +6,24 @@ Apache Paimon数据连接器。支持cdc写以及自动建表。 +## 支持的数据源信息 + +| 数据源 | 依赖 | Maven | +|--------|-----------|---------------------------------------------------------------------------| +| Paimon | hive-exec | [Download](https://mvnrepository.com/artifact/org.apache.hive/hive-exec) | +| Paimon | libfb303 | [Download](https://mvnrepository.com/artifact/org.apache.thrift/libfb303) | + +## 数据源依赖 + +> 为了兼容不同版本的Hadoop和Hive,在项目pom文件中Hive -exec的作用域为provided,所以如果您使用Flink引擎,首先可能需要将以下Jar包添加到/lib目录下,如果您使用Spark引擎并与Hadoop集成,则不需要添加以下Jar包。 + +``` +hive-exec-xxx.jar +libfb303-xxx.jar +``` + +> 有些版本的hive-exec包没有libfb303-xxx.jar,所以您还需要手动导入Jar包。 + ## 主要特性 - [x] [exactly-once](../../concept/connector-v2-features.md) @@ -13,8 +31,10 @@ Apache Paimon数据连接器。支持cdc写以及自动建表。 ## 连接器选项 | 名称 | 类型 | 是否必须 | 默认值 | 描述 | -|-----------------------------|-----|------|------------------------------|---------------------------------------------------------------------------------------------------| +|-----------------------------|-----|------|------------------------------|---------------------------------------------------------------------------------------------------|---| | warehouse | 字符串 | 是 | - | Paimon warehouse路径 | +| catalog_type | 字符串 | 否 | filesystem | Paimon的catalog类型,目前支持filesystem和hive | +| catalog_uri | 字符串 | 否 | - | Paimon catalog的uri,仅当catalog_type为hive时需要配置 | | | database | 字符串 | 是 | - | 数据库名称 | | table | 字符串 | 是 | - | 表名 | | hdfs_site_path | 字符串 | 否 | - | hdfs-site.xml文件路径 | @@ -101,6 +121,91 @@ sink { } ``` +### 单表(使用Hive catalog) + +```hocon +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + pk_id = bigint + name = string + score = int + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = [1, "A", 100] + }, + { + kind = INSERT + fields = [2, "B", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + } + { + kind = UPDATE_BEFORE + fields = [1, "A", 100] + }, + { + kind = UPDATE_AFTER + fields = [1, "A_1", 100] + }, + { + kind = DELETE + fields = [2, "B", 100] + } + ] + } +} + +sink { + Paimon { + schema_save_mode = "RECREATE_SCHEMA" + catalog_name="seatunnel_test" + catalog_type="hive" + catalog_uri="thrift://hadoop04:9083" + warehouse="hdfs:///tmp/seatunnel" + database="seatunnel_test" + table="st_test3" + paimon.hadoop.conf = { + fs.defaultFS = "hdfs://nameservice1" + dfs.nameservices = "nameservice1" + dfs.ha.namenodes.nameservice1 = "nn1,nn2" + dfs.namenode.rpc-address.nameservice1.nn1 = "hadoop03:8020" + dfs.namenode.rpc-address.nameservice1.nn2 = "hadoop04:8020" + dfs.client.failover.proxy.provider.nameservice1 = "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" + dfs.client.use.datanode.hostname = "true" + } + } +} + +``` + ### 指定paimon的写属性的单表 ```hocon diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonError.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonError.java index 918936b3402..9843774d0c6 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonError.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonError.java @@ -29,6 +29,7 @@ import static org.apache.seatunnel.common.exception.CommonErrorCode.CONVERT_TO_CONNECTOR_TYPE_ERROR; import static org.apache.seatunnel.common.exception.CommonErrorCode.CONVERT_TO_CONNECTOR_TYPE_ERROR_SIMPLE; +import static org.apache.seatunnel.common.exception.CommonErrorCode.CONVERT_TO_SEATUNNEL_PROPS_BLANK_ERROR; import static org.apache.seatunnel.common.exception.CommonErrorCode.CONVERT_TO_SEATUNNEL_TYPE_ERROR; import static org.apache.seatunnel.common.exception.CommonErrorCode.CONVERT_TO_SEATUNNEL_TYPE_ERROR_SIMPLE; import static org.apache.seatunnel.common.exception.CommonErrorCode.FILE_NOT_EXISTED; @@ -139,6 +140,14 @@ public static SeaTunnelRuntimeException convertToConnectorTypeError( return new SeaTunnelRuntimeException(CONVERT_TO_CONNECTOR_TYPE_ERROR, params); } + public static SeaTunnelRuntimeException convertToConnectorPropsBlankError( + String connector, String props) { + Map params = new HashMap<>(); + params.put("connector", connector); + params.put("props", props); + return new SeaTunnelRuntimeException(CONVERT_TO_SEATUNNEL_PROPS_BLANK_ERROR, params); + } + public static SeaTunnelRuntimeException convertToConnectorTypeError( String identifier, String dataType, String field) { Map params = new HashMap<>(); diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java index feacf4bd7cd..830e651f806 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/exception/CommonErrorCode.java @@ -56,7 +56,9 @@ public enum CommonErrorCode implements SeaTunnelErrorCode { VERSION_NOT_SUPPORTED("COMMON-25", " is unsupported."), - OPERATION_NOT_SUPPORTED("COMMON-26", " is unsupported."); + OPERATION_NOT_SUPPORTED("COMMON-26", " is unsupported."), + CONVERT_TO_SEATUNNEL_PROPS_BLANK_ERROR( + "COMMON-27", "The props named '' of '' is blank."); private final String code; private final String description; diff --git a/seatunnel-connectors-v2/connector-paimon/pom.xml b/seatunnel-connectors-v2/connector-paimon/pom.xml index bc0756ecdc4..267e66dc0ad 100644 --- a/seatunnel-connectors-v2/connector-paimon/pom.xml +++ b/seatunnel-connectors-v2/connector-paimon/pom.xml @@ -31,6 +31,7 @@ 0.7.0-incubating + 2.3.9 @@ -60,6 +61,36 @@ provided + + org.apache.hive + hive-exec + ${hive.version} + core + provided + + + org.apache.logging.log4j + * + + + org.pentaho + * + + + org.apache.parquet + * + + + org.apache.orc + * + + + org.apache.avro + * + + + + diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java index 8d3395af3c4..fab64da52e3 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.paimon.catalog; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.Catalog; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; @@ -27,6 +28,7 @@ import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; import org.apache.seatunnel.api.table.catalog.exception.TableAlreadyExistException; import org.apache.seatunnel.api.table.catalog.exception.TableNotExistException; +import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonConfig; import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonSinkConfig; import org.apache.seatunnel.connectors.seatunnel.paimon.utils.SchemaUtil; @@ -47,14 +49,14 @@ public class PaimonCatalog implements Catalog, PaimonTable { private static final String DEFAULT_DATABASE = "default"; private String catalogName; - private PaimonSinkConfig paimonSinkConfig; + private ReadonlyConfig readonlyConfig; private PaimonCatalogLoader paimonCatalogLoader; private org.apache.paimon.catalog.Catalog catalog; - public PaimonCatalog(String catalogName, PaimonSinkConfig paimonSinkConfig) { - this.paimonSinkConfig = paimonSinkConfig; + public PaimonCatalog(String catalogName, ReadonlyConfig readonlyConfig) { + this.readonlyConfig = readonlyConfig; this.catalogName = catalogName; - this.paimonCatalogLoader = new PaimonCatalogLoader(paimonSinkConfig); + this.paimonCatalogLoader = new PaimonCatalogLoader(new PaimonConfig(readonlyConfig)); } @Override @@ -135,7 +137,8 @@ public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreI throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { try { Schema paimonSchema = - SchemaUtil.toPaimonSchema(table.getTableSchema(), this.paimonSinkConfig); + SchemaUtil.toPaimonSchema( + table.getTableSchema(), new PaimonSinkConfig(readonlyConfig)); catalog.createTable(toIdentifier(tablePath), paimonSchema, ignoreIfExists); } catch (org.apache.paimon.catalog.Catalog.TableAlreadyExistException e) { throw new TableAlreadyExistException(this.catalogName, tablePath); diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogEnum.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogEnum.java new file mode 100644 index 00000000000..a90bd729f46 --- /dev/null +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogEnum.java @@ -0,0 +1,33 @@ +/* + * 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.paimon.catalog; + +public enum PaimonCatalogEnum { + FILESYSTEM("filesystem"), + HIVE("hive"); + + final String type; + + PaimonCatalogEnum(String type) { + this.type = type; + } + + public String getType() { + return type; + } +} diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogFactory.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogFactory.java index 9994df1d86f..8858c211938 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogFactory.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogFactory.java @@ -22,6 +22,7 @@ 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.connectors.seatunnel.paimon.config.PaimonConfig; import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonSinkConfig; import com.google.auto.service.AutoService; @@ -30,7 +31,7 @@ public class PaimonCatalogFactory implements CatalogFactory { @Override public Catalog createCatalog(String catalogName, ReadonlyConfig readonlyConfig) { - return new PaimonCatalog(catalogName, new PaimonSinkConfig(readonlyConfig)); + return new PaimonCatalog(catalogName, readonlyConfig); } @Override @@ -41,19 +42,19 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { return OptionRule.builder() - .required( - PaimonSinkConfig.WAREHOUSE, - PaimonSinkConfig.DATABASE, - PaimonSinkConfig.TABLE) + .required(PaimonConfig.WAREHOUSE, PaimonConfig.DATABASE, PaimonConfig.TABLE) .optional( - PaimonSinkConfig.HDFS_SITE_PATH, + PaimonConfig.HDFS_SITE_PATH, + PaimonConfig.HADOOP_CONF, + PaimonConfig.HADOOP_CONF_PATH, + PaimonConfig.CATALOG_TYPE, PaimonSinkConfig.SCHEMA_SAVE_MODE, PaimonSinkConfig.DATA_SAVE_MODE, PaimonSinkConfig.PRIMARY_KEYS, PaimonSinkConfig.PARTITION_KEYS, - PaimonSinkConfig.WRITE_PROPS, - PaimonSinkConfig.HADOOP_CONF, - PaimonSinkConfig.HADOOP_CONF_PATH) + PaimonSinkConfig.WRITE_PROPS) + .conditional( + PaimonConfig.CATALOG_TYPE, PaimonCatalogEnum.HIVE, PaimonConfig.CATALOG_URI) .build(); } } diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogLoader.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogLoader.java index 1c00e6dc3b8..774576c408f 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogLoader.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogLoader.java @@ -17,8 +17,8 @@ package org.apache.seatunnel.connectors.seatunnel.paimon.catalog; +import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonConfig; import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonHadoopConfiguration; -import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonSinkConfig; import org.apache.seatunnel.connectors.seatunnel.paimon.exception.PaimonConnectorErrorCode; import org.apache.seatunnel.connectors.seatunnel.paimon.exception.PaimonConnectorException; import org.apache.seatunnel.connectors.seatunnel.paimon.security.PaimonSecurityContext; @@ -28,6 +28,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; import lombok.extern.slf4j.Slf4j; @@ -37,8 +38,6 @@ import java.util.Iterator; import java.util.Map; -import static org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonConfig.WAREHOUSE; - @Slf4j public class PaimonCatalogLoader implements Serializable { /** hdfs uri is required */ @@ -51,24 +50,35 @@ public class PaimonCatalogLoader implements Serializable { private static final String HDFS_IMPL_KEY = "fs.hdfs.impl"; private String warehouse; + private PaimonCatalogEnum catalogType; + private String catalogUri; private PaimonHadoopConfiguration paimonHadoopConfiguration; - public PaimonCatalogLoader(PaimonSinkConfig paimonSinkConfig) { - this.warehouse = paimonSinkConfig.getWarehouse(); - this.paimonHadoopConfiguration = PaimonSecurityContext.loadHadoopConfig(paimonSinkConfig); + public PaimonCatalogLoader(PaimonConfig paimonConfig) { + this.warehouse = paimonConfig.getWarehouse(); + this.catalogType = paimonConfig.getCatalogType(); + this.catalogUri = paimonConfig.getCatalogUri(); + this.paimonHadoopConfiguration = PaimonSecurityContext.loadHadoopConfig(paimonConfig); } public Catalog loadCatalog() { // When using the seatunel engine, set the current class loader to prevent loading failures Thread.currentThread().setContextClassLoader(PaimonCatalogLoader.class.getClassLoader()); final Map optionsMap = new HashMap<>(1); - optionsMap.put(WAREHOUSE.key(), warehouse); - final Options options = Options.fromMap(optionsMap); + optionsMap.put(CatalogOptions.WAREHOUSE.key(), warehouse); + optionsMap.put(CatalogOptions.METASTORE.key(), catalogType.getType()); if (warehouse.startsWith(HDFS_PREFIX)) { checkConfiguration(paimonHadoopConfiguration, HDFS_DEF_FS_NAME); paimonHadoopConfiguration.set(HDFS_IMPL_KEY, HDFS_IMPL); } + if (PaimonCatalogEnum.HIVE.getType().equals(catalogType.getType())) { + optionsMap.put(CatalogOptions.URI.key(), catalogUri); + paimonHadoopConfiguration + .getPropsWithPrefix(StringUtils.EMPTY) + .forEach((k, v) -> optionsMap.put(k, v)); + } + final Options options = Options.fromMap(optionsMap); PaimonSecurityContext.shouldEnableKerberos(paimonHadoopConfiguration); final CatalogContext catalogContext = CatalogContext.create(options, paimonHadoopConfiguration); diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/config/PaimonConfig.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/config/PaimonConfig.java index 01207570837..0f41402d034 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/config/PaimonConfig.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/config/PaimonConfig.java @@ -25,6 +25,11 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.common.exception.CommonError; +import org.apache.seatunnel.common.utils.SeaTunnelException; +import org.apache.seatunnel.connectors.seatunnel.paimon.catalog.PaimonCatalogEnum; + +import org.apache.commons.lang3.StringUtils; import lombok.Getter; @@ -35,7 +40,6 @@ import java.util.Map; import static java.util.stream.Collectors.toList; -import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull; /** * Utility class to store configuration options, used by {@link SeaTunnelSource} and {@link @@ -50,11 +54,23 @@ public class PaimonConfig implements Serializable { .noDefaultValue() .withDescription("The warehouse path of paimon"); + public static final Option CATALOG_TYPE = + Options.key("catalog_type") + .enumType(PaimonCatalogEnum.class) + .defaultValue(PaimonCatalogEnum.FILESYSTEM) + .withDescription("The type of paimon catalog"); + + public static final Option CATALOG_URI = + Options.key("catalog_uri") + .stringType() + .noDefaultValue() + .withDescription("The uri of paimon with hive catalog"); + public static final Option CATALOG_NAME = Options.key("catalog_name") .stringType() .defaultValue("paimon") - .withDescription(" the iceberg catalog name"); + .withDescription(" the paimon catalog name"); public static final Option DATABASE = Options.key("database") @@ -95,6 +111,8 @@ public class PaimonConfig implements Serializable { "The specified loading path for the 'core-site.xml', 'hdfs-site.xml', 'hive-site.xml' files"); protected String catalogName; + protected PaimonCatalogEnum catalogType; + protected String catalogUri; protected String warehouse; protected String namespace; protected String table; @@ -103,18 +121,27 @@ public class PaimonConfig implements Serializable { protected String hadoopConfPath; public PaimonConfig(ReadonlyConfig readonlyConfig) { - this.catalogName = checkArgumentNotNull(readonlyConfig.get(CATALOG_NAME)); - this.warehouse = checkArgumentNotNull(readonlyConfig.get(WAREHOUSE)); - this.namespace = checkArgumentNotNull(readonlyConfig.get(DATABASE)); - this.table = checkArgumentNotNull(readonlyConfig.get(TABLE)); + this.catalogName = + checkArgumentNotBlank(readonlyConfig.get(CATALOG_NAME), CATALOG_NAME.key()); + this.warehouse = checkArgumentNotBlank(readonlyConfig.get(WAREHOUSE), WAREHOUSE.key()); + this.namespace = checkArgumentNotBlank(readonlyConfig.get(DATABASE), DATABASE.key()); + this.table = checkArgumentNotBlank(readonlyConfig.get(TABLE), TABLE.key()); this.hdfsSitePath = readonlyConfig.get(HDFS_SITE_PATH); this.hadoopConfProps = readonlyConfig.get(HADOOP_CONF); this.hadoopConfPath = readonlyConfig.get(HADOOP_CONF_PATH); + this.catalogType = readonlyConfig.get(CATALOG_TYPE); + if (PaimonCatalogEnum.HIVE.getType().equals(catalogType.getType())) { + this.catalogUri = + checkArgumentNotBlank(readonlyConfig.get(CATALOG_URI), CATALOG_URI.key()); + } } - protected T checkArgumentNotNull(T argument) { - checkNotNull(argument); - return argument; + protected String checkArgumentNotBlank(String propValue, String propKey) { + if (StringUtils.isBlank(propValue)) { + throw new SeaTunnelException( + CommonError.convertToConnectorPropsBlankError("Paimon", propKey)); + } + return propValue; } @VisibleForTesting diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkFactory.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkFactory.java index dd656cd8ceb..46b92afb097 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkFactory.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkFactory.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; +import org.apache.seatunnel.connectors.seatunnel.paimon.catalog.PaimonCatalogEnum; import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonConfig; import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonSinkConfig; @@ -52,13 +53,16 @@ public OptionRule optionRule() { .required(PaimonConfig.WAREHOUSE, PaimonConfig.DATABASE, PaimonConfig.TABLE) .optional( PaimonConfig.HDFS_SITE_PATH, + PaimonConfig.HADOOP_CONF, + PaimonConfig.HADOOP_CONF_PATH, + PaimonConfig.CATALOG_TYPE, PaimonSinkConfig.SCHEMA_SAVE_MODE, PaimonSinkConfig.DATA_SAVE_MODE, PaimonSinkConfig.PRIMARY_KEYS, PaimonSinkConfig.PARTITION_KEYS, - PaimonSinkConfig.WRITE_PROPS, - PaimonConfig.HADOOP_CONF, - PaimonConfig.HADOOP_CONF_PATH) + PaimonSinkConfig.WRITE_PROPS) + .conditional( + PaimonConfig.CATALOG_TYPE, PaimonCatalogEnum.HIVE, PaimonConfig.CATALOG_URI) .build(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkHdfsIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkHdfsIT.java index 72301bcf9d7..5d2ce86c2b5 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkHdfsIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkHdfsIT.java @@ -24,9 +24,11 @@ import org.apache.seatunnel.connectors.seatunnel.paimon.catalog.PaimonCatalogLoader; import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonSinkConfig; 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.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; @@ -57,6 +59,29 @@ @Disabled( "HDFS is not available in CI, if you want to run this test, please set up your own HDFS environment in the test case file and the below setup") public class PaimonSinkHdfsIT extends TestSuiteBase { + + private String hiveExecUrl() { + return "https://repo1.maven.org/maven2/org/apache/hive/hive-exec/3.1.3/hive-exec-3.1.3.jar"; + } + + private String libfb303Url() { + return "https://repo1.maven.org/maven2/org/apache/thrift/libfb303/0.9.0/libfb303-0.9.0.jar"; + } + + @TestContainerExtension + protected final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Paimon/lib && cd /tmp/seatunnel/plugins/Paimon/lib && wget " + + hiveExecUrl() + + " && wget " + + libfb303Url()); + Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); + }; + private Map PAIMON_SINK_PROPERTIES; @BeforeAll @@ -124,4 +149,50 @@ public void testFakeCDCSinkPaimon(TestContainer container) throws Exception { }); }); } + + @TestTemplate + public void testFakeCDCSinkPaimonWithHiveCatalog(TestContainer container) throws Exception { + Container.ExecResult execResult = + container.executeJob("/fake_cdc_sink_paimon_with_hdfs_with_hive_catalog.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + + given().ignoreExceptions() + .await() + .atLeast(200L, TimeUnit.MILLISECONDS) + .atMost(40L, TimeUnit.SECONDS) + .untilAsserted( + () -> { + PaimonSinkConfig paimonSinkConfig = + new PaimonSinkConfig( + ReadonlyConfig.fromMap(PAIMON_SINK_PROPERTIES)); + PaimonCatalogLoader paimonCatalogLoader = + new PaimonCatalogLoader(paimonSinkConfig); + Catalog catalog = paimonCatalogLoader.loadCatalog(); + Table table = + catalog.getTable( + Identifier.create("seatunnel_namespace1", "st_test")); + ReadBuilder readBuilder = table.newReadBuilder(); + TableScan.Plan plan = readBuilder.newScan().plan(); + TableRead tableRead = readBuilder.newRead(); + List paimonRecords = new ArrayList<>(); + try (RecordReader reader = tableRead.createReader(plan)) { + reader.forEachRemaining( + row -> + paimonRecords.add( + new PaimonRecord( + row.getLong(0), + row.getString(1).toString()))); + } + Assertions.assertEquals(2, paimonRecords.size()); + paimonRecords.forEach( + paimonRecord -> { + if (paimonRecord.getPkId() == 1) { + Assertions.assertEquals("A_1", paimonRecord.getName()); + } + if (paimonRecord.getPkId() == 3) { + Assertions.assertEquals("C", paimonRecord.getName()); + } + }); + }); + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_cdc_sink_paimon_with_hdfs_with_hive_catalog.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_cdc_sink_paimon_with_hdfs_with_hive_catalog.conf new file mode 100644 index 00000000000..3afdc59701b --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_cdc_sink_paimon_with_hdfs_with_hive_catalog.conf @@ -0,0 +1,99 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + pk_id = bigint + name = string + score = int + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = [1, "A", 100] + }, + { + kind = INSERT + fields = [2, "B", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + } + { + kind = UPDATE_BEFORE + fields = [1, "A", 100] + }, + { + kind = UPDATE_AFTER + fields = [1, "A_1", 100] + }, + { + kind = DELETE + fields = [2, "B", 100] + } + ] + } +} + +sink { + Paimon { + schema_save_mode = "RECREATE_SCHEMA" + catalog_name="seatunnel_test" + catalog_type="hive" + catalog_uri="thrift://hadoop04:9083" + warehouse="hdfs:///tmp/seatunnel" + database="seatunnel_test" + table="st_test3" + paimon.hadoop.conf = { + fs.defaultFS = "hdfs://nameservice1" + dfs.nameservices = "nameservice1" + dfs.ha.namenodes.nameservice1 = "nn1,nn2" + dfs.namenode.rpc-address.nameservice1.nn1 = "hadoop03:8020" + dfs.namenode.rpc-address.nameservice1.nn2 = "hadoop04:8020" + dfs.client.failover.proxy.provider.nameservice1 = "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" + dfs.client.use.datanode.hostname = "true" + } + } +} From 60104602d128bcc0c1319be38c5f11658690b201 Mon Sep 17 00:00:00 2001 From: ZhilinLi Date: Wed, 22 May 2024 14:17:21 +0800 Subject: [PATCH 14/14] [Feature][Kafka] Support multi-table source read (#5992) --- docs/en/connector-v2/source/kafka.md | 62 ++++++ .../amazonsqs/source/AmazonSqsSource.java | 17 +- .../fake/source/FakeDataGenerator.java | 3 +- .../google/sheets/source/SheetsSource.java | 11 +- .../GoogleSheetsDeserializerTest.java | 7 +- .../seatunnel/http/source/HttpSource.java | 3 +- .../seatunnel/kafka/config/Config.java | 6 + .../kafka/source/ConsumerMetadata.java | 7 +- .../kafka/source/KafkaConsumerThread.java | 14 +- .../seatunnel/kafka/source/KafkaSource.java | 20 +- .../kafka/source/KafkaSourceConfig.java | 104 ++++++---- .../kafka/source/KafkaSourceFactory.java | 3 +- .../kafka/source/KafkaSourceReader.java | 26 ++- .../kafka/source/KafkaSourceSplit.java | 19 +- .../source/KafkaSourceSplitEnumerator.java | 148 +++++++------- .../seatunnel/pulsar/source/PulsarSource.java | 6 +- .../source/PulsarCanalDecoratorTest.java | 7 +- .../rabbitmq/source/RabbitmqSource.java | 6 +- .../seatunnel/redis/source/RedisSource.java | 2 +- .../rocketmq/source/RocketMqSource.java | 8 +- .../e2e/connector/kafka/KafkaFormatIT.java | 122 +++++++++--- .../e2e/connector/kafka/KafkaIT.java | 11 +- .../src/test/resources/canal/canal_data.txt | 10 +- .../kafka_multi_source_to_pg.conf | 73 +++++++ .../src/test/resources/ogg/ogg_data.txt | 2 +- .../avro/AvroDeserializationSchema.java | 17 +- .../avro/AvroSerializationSchemaTest.java | 21 +- ...ibleKafkaConnectDeserializationSchema.java | 27 ++- .../json/JsonDeserializationSchema.java | 38 +++- .../canal/CanalJsonDeserializationSchema.java | 155 +++++++++------ .../DebeziumJsonDeserializationSchema.java | 99 ++++++---- .../ogg/OggJsonDeserializationSchema.java | 185 ++++++++++-------- .../json/JsonRowDataSerDeSchemaTest.java | 43 ++-- .../json/canal/CanalJsonSerDeSchemaTest.java | 64 +++--- .../debezium/DebeziumJsonSerDeSchemaTest.java | 60 +++--- .../json/ogg/OggJsonSerDeSchemaTest.java | 52 ++--- .../text/TextDeserializationSchema.java | 33 +++- 37 files changed, 999 insertions(+), 492 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf diff --git a/docs/en/connector-v2/source/kafka.md b/docs/en/connector-v2/source/kafka.md index 79bfc49b407..c42c84a8a5a 100644 --- a/docs/en/connector-v2/source/kafka.md +++ b/docs/en/connector-v2/source/kafka.md @@ -35,6 +35,7 @@ They can be downloaded via install-plugin.sh or from the Maven central repositor | Name | Type | Required | Default | Description | |-------------------------------------|-----------------------------------------------------------------------------|----------|--------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | topic | String | Yes | - | Topic name(s) to read data from when the table is used as source. It also supports topic list for source by separating topic by comma like 'topic-1,topic-2'. | +| table_list | Map | No | - | Topic list config You can configure only one `table_list` and one `topic` at the same time | | bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | | pattern | Boolean | No | false | If `pattern` is set to `true`,the regular expression for a pattern of topic names to read from. All topics in clients with names that match the specified regular expression will be subscribed by the consumer. | | consumer.group | String | No | SeaTunnel-Consumer-Group | `Kafka consumer group id`, used to distinguish different consumer groups. | @@ -180,3 +181,64 @@ source { } ``` +### Multiple Kafka Source + +> This is written to the same pg table according to different formats and topics of parsing kafka Perform upsert operations based on the id + +```hocon + +env { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + Kafka { + bootstrap.servers = "kafka_e2e:9092" + table_list = [ + { + topic = "^test-ogg-sou.*" + pattern = "true" + consumer.group = "ogg_multi_group" + start_mode = earliest + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "string" + } + }, + format = ogg_json + }, + { + topic = "test-cdc_mds" + start_mode = earliest + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "string" + } + }, + format = canal_json + } + ] + } +} + +sink { + Jdbc { + driver = org.postgresql.Driver + url = "jdbc:postgresql://postgresql:5432/test?loggerLevel=OFF" + user = test + password = test + generate_sink_sql = true + database = test + table = public.sink + primary_keys = ["id"] + } +} +``` + diff --git a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java index 9022abe1f10..c4c3ad372d2 100644 --- a/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java +++ b/seatunnel-connectors-v2/connector-amazonsqs/src/main/java/org/apache/seatunnel/connectors/seatunnel/amazonsqs/source/AmazonSqsSource.java @@ -26,6 +26,7 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SupportColumnProjection; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -66,6 +67,7 @@ public class AmazonSqsSource extends AbstractSingleSplitSource private AmazonSqsSourceOptions amazonSqsSourceOptions; private DeserializationSchema deserializationSchema; private SeaTunnelRowType typeInfo; + private CatalogTable catalogTable; @Override public String getPluginName() { @@ -84,9 +86,9 @@ public void prepare(Config pluginConfig) throws PrepareFailException { "PluginName: %s, PluginType: %s, Message: %s", getPluginName(), PluginType.SOURCE, result.getMsg())); } - amazonSqsSourceOptions = new AmazonSqsSourceOptions(pluginConfig); - typeInfo = CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); - + this.amazonSqsSourceOptions = new AmazonSqsSourceOptions(pluginConfig); + this.catalogTable = CatalogTableUtil.buildWithConfig(pluginConfig); + this.typeInfo = catalogTable.getSeaTunnelRowType(); setDeserialization(pluginConfig); } @@ -109,11 +111,11 @@ public AbstractSingleSplitReader createReader( private void setDeserialization(Config config) { if (config.hasPath(TableSchemaOptions.SCHEMA.key())) { - typeInfo = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); MessageFormat format = ReadonlyConfig.fromConfig(config).get(FORMAT); switch (format) { case JSON: - deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); + deserializationSchema = + new JsonDeserializationSchema(catalogTable, false, false); break; case TEXT: String delimiter = DEFAULT_FIELD_DELIMITER; @@ -128,7 +130,7 @@ private void setDeserialization(Config config) { break; case CANAL_JSON: deserializationSchema = - CanalJsonDeserializationSchema.builder(typeInfo) + CanalJsonDeserializationSchema.builder(catalogTable) .setIgnoreParseErrors(true) .build(); break; @@ -138,7 +140,8 @@ private void setDeserialization(Config config) { includeSchema = config.getBoolean(DEBEZIUM_RECORD_INCLUDE_SCHEMA.key()); } deserializationSchema = - new DebeziumJsonDeserializationSchema(typeInfo, true, includeSchema); + new DebeziumJsonDeserializationSchema( + catalogTable, true, includeSchema); break; default: throw new SeaTunnelJsonFormatException( diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java index fa5de48e0e3..b5b6c69adb9 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java @@ -52,8 +52,7 @@ public FakeDataGenerator(FakeConfig fakeConfig) { this.jsonDeserializationSchema = fakeConfig.getFakeRows() == null ? null - : new JsonDeserializationSchema( - false, false, catalogTable.getSeaTunnelRowType()); + : new JsonDeserializationSchema(catalogTable, false, false); this.fakeDataRandomUtils = new FakeDataRandomUtils(fakeConfig); } diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java index 0661f6db366..279a39256ec 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/main/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/source/SheetsSource.java @@ -24,6 +24,7 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -46,6 +47,7 @@ public class SheetsSource extends AbstractSingleSplitSource { private SeaTunnelRowType seaTunnelRowType; + private CatalogTable catalogTable; private SheetsParameters sheetsParameters; @@ -75,12 +77,13 @@ public void prepare(Config pluginConfig) throws PrepareFailException { } this.sheetsParameters = new SheetsParameters().buildWithConfig(pluginConfig); if (pluginConfig.hasPath(TableSchemaOptions.SCHEMA.key())) { - this.seaTunnelRowType = - CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); + this.catalogTable = CatalogTableUtil.buildWithConfig(pluginConfig); } else { - this.seaTunnelRowType = CatalogTableUtil.buildSimpleTextSchema(); + this.catalogTable = CatalogTableUtil.buildSimpleTextTable(); } - this.deserializationSchema = new JsonDeserializationSchema(false, false, seaTunnelRowType); + + this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); + this.deserializationSchema = new JsonDeserializationSchema(catalogTable, false, false); } @Override diff --git a/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java b/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java index 8c81a51c581..c55228471c2 100644 --- a/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java +++ b/seatunnel-connectors-v2/connector-google-sheets/src/test/java/org/apache/seatunnel/connectors/seatunnel/google/sheets/deserialize/GoogleSheetsDeserializerTest.java @@ -18,6 +18,8 @@ package org.apache.seatunnel.connectors.seatunnel.google.sheets.deserialize; import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -39,8 +41,11 @@ public class GoogleSheetsDeserializerTest { public void testJsonParseError() { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); + + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + final DeserializationSchema deser = - new JsonDeserializationSchema(false, false, schema); + new JsonDeserializationSchema(catalogTables, false, false); final GoogleSheetsDeserializer googleSheetsDeser = new GoogleSheetsDeserializer(schema.getFieldNames(), deser); List row = new ArrayList<>(); diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java index 4e3f3ab1a59..754a7b93664 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java @@ -127,8 +127,7 @@ protected void buildSchemaWithConfig(Config pluginConfig) { switch (format) { case JSON: this.deserializationSchema = - new JsonDeserializationSchema( - false, false, catalogTable.getSeaTunnelRowType()); + new JsonDeserializationSchema(catalogTable, false, false); if (pluginConfig.hasPath(HttpConfig.JSON_FIELD.key())) { jsonField = getJsonField(pluginConfig.getConfig(HttpConfig.JSON_FIELD.key())); diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java index 1bf0c19fc22..6d6f43dc4c2 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java @@ -177,4 +177,10 @@ public class Config { .defaultValue(KafkaSemantics.NON) .withDescription( "Semantics that can be chosen EXACTLY_ONCE/AT_LEAST_ONCE/NON, default NON."); + public static final Option>> TABLE_LIST = + Options.key("table_list") + .type(new TypeReference>>() {}) + .noDefaultValue() + .withDescription( + "Topic list config. You can configure only one `table_list` or one `topic` at the same time"); } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java index e19fbc79244..8ce9dbb870a 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java @@ -17,6 +17,9 @@ package org.apache.seatunnel.connectors.seatunnel.kafka.source; +import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.connectors.seatunnel.kafka.config.StartMode; import org.apache.kafka.common.TopicPartition; @@ -33,11 +36,11 @@ public class ConsumerMetadata implements Serializable { private String topic; private boolean isPattern = false; - private String bootstrapServers; private Properties properties; private String consumerGroup; - private boolean commitOnCheckpoint = false; private StartMode startMode = StartMode.GROUP_OFFSETS; private Map specificStartOffsets; private Long startOffsetsTimestamp; + private DeserializationSchema deserializationSchema; + private CatalogTable catalogTable; } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java index 62e9d7b9fdf..90d46bc502e 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java @@ -37,15 +37,15 @@ public class KafkaConsumerThread implements Runnable { private final LinkedBlockingQueue>> tasks; - public KafkaConsumerThread(ConsumerMetadata metadata) { + public KafkaConsumerThread(KafkaSourceConfig kafkaSourceConfig, ConsumerMetadata metadata) { this.metadata = metadata; this.tasks = new LinkedBlockingQueue<>(); this.consumer = initConsumer( - this.metadata.getBootstrapServers(), - this.metadata.getConsumerGroup(), - this.metadata.getProperties(), - !this.metadata.isCommitOnCheckpoint()); + kafkaSourceConfig.getBootstrap(), + metadata.getConsumerGroup(), + kafkaSourceConfig.getProperties(), + kafkaSourceConfig.isCommitOnCheckpoint()); } @Override @@ -64,7 +64,9 @@ public void run() { } } finally { try { - consumer.close(); + if (consumer != null) { + consumer.close(); + } } catch (Throwable t) { throw new KafkaConnectorException(KafkaConnectorErrorCode.CONSUMER_CLOSE_FAILED, t); } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java index 6a737bb1ae9..0c8cb4d496a 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java @@ -29,9 +29,8 @@ import org.apache.seatunnel.common.constants.JobMode; import org.apache.seatunnel.connectors.seatunnel.kafka.state.KafkaSourceState; -import com.google.common.collect.Lists; - import java.util.List; +import java.util.stream.Collectors; public class KafkaSource implements SeaTunnelSource, @@ -59,15 +58,16 @@ public String getPluginName() { @Override public List getProducedCatalogTables() { - return Lists.newArrayList(kafkaSourceConfig.getCatalogTable()); + return kafkaSourceConfig.getMapMetadata().values().stream() + .map(ConsumerMetadata::getCatalogTable) + .collect(Collectors.toList()); } @Override public SourceReader createReader( SourceReader.Context readerContext) { return new KafkaSourceReader( - kafkaSourceConfig.getMetadata(), - kafkaSourceConfig.getDeserializationSchema(), + kafkaSourceConfig, readerContext, kafkaSourceConfig.getMessageFormatErrorHandleWay()); } @@ -75,10 +75,7 @@ public SourceReader createReader( @Override public SourceSplitEnumerator createEnumerator( SourceSplitEnumerator.Context enumeratorContext) { - return new KafkaSourceSplitEnumerator( - kafkaSourceConfig.getMetadata(), - enumeratorContext, - kafkaSourceConfig.getDiscoveryIntervalMillis()); + return new KafkaSourceSplitEnumerator(kafkaSourceConfig, enumeratorContext, null); } @Override @@ -86,10 +83,7 @@ public SourceSplitEnumerator restoreEnumerat SourceSplitEnumerator.Context enumeratorContext, KafkaSourceState checkpointState) { return new KafkaSourceSplitEnumerator( - kafkaSourceConfig.getMetadata(), - enumeratorContext, - checkpointState, - kafkaSourceConfig.getDiscoveryIntervalMillis()); + kafkaSourceConfig, enumeratorContext, checkpointState); } @Override diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java index 8c449c68430..232ee2f0e0d 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java @@ -20,16 +20,18 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.schema.ReadonlyConfigParser; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.connectors.seatunnel.kafka.config.Config; import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormat; import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormatErrorHandleWay; import org.apache.seatunnel.connectors.seatunnel.kafka.config.StartMode; @@ -52,13 +54,14 @@ import java.io.Serializable; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.stream.Collectors; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.BOOTSTRAP_SERVERS; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.COMMIT_ON_CHECKPOINT; -import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.CONNECTOR_IDENTITY; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.CONSUMER_GROUP; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.DEBEZIUM_RECORD_INCLUDE_SCHEMA; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.FIELD_DELIMITER; @@ -76,33 +79,62 @@ public class KafkaSourceConfig implements Serializable { private static final long serialVersionUID = 1L; - @Getter private final ConsumerMetadata metadata; - - @Getter private final DeserializationSchema deserializationSchema; - - @Getter private final CatalogTable catalogTable; - - @Getter private final MessageFormatErrorHandleWay messageFormatErrorHandleWay; - + @Getter private final String bootstrap; + @Getter private final Map mapMetadata; + @Getter private final boolean commitOnCheckpoint; + @Getter private final Properties properties; @Getter private final long discoveryIntervalMillis; + @Getter private final MessageFormatErrorHandleWay messageFormatErrorHandleWay; public KafkaSourceConfig(ReadonlyConfig readonlyConfig) { - this.metadata = createConsumerMetadata(readonlyConfig); + this.bootstrap = readonlyConfig.get(BOOTSTRAP_SERVERS); + this.mapMetadata = createMapConsumerMetadata(readonlyConfig); + this.commitOnCheckpoint = readonlyConfig.get(COMMIT_ON_CHECKPOINT); + this.properties = createKafkaProperties(readonlyConfig); this.discoveryIntervalMillis = readonlyConfig.get(KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS); this.messageFormatErrorHandleWay = readonlyConfig.get(MESSAGE_FORMAT_ERROR_HANDLE_WAY_OPTION); - this.catalogTable = createCatalogTable(readonlyConfig); - this.deserializationSchema = createDeserializationSchema(catalogTable, readonlyConfig); + } + + private Properties createKafkaProperties(ReadonlyConfig readonlyConfig) { + Properties resultProperties = new Properties(); + readonlyConfig.getOptional(KAFKA_CONFIG).ifPresent(resultProperties::putAll); + return resultProperties; + } + + private Map createMapConsumerMetadata( + ReadonlyConfig readonlyConfig) { + List consumerMetadataList; + if (readonlyConfig.getOptional(Config.TABLE_LIST).isPresent()) { + consumerMetadataList = + readonlyConfig.get(Config.TABLE_LIST).stream() + .map(ReadonlyConfig::fromMap) + .map(config -> createConsumerMetadata(config)) + .collect(Collectors.toList()); + } else { + consumerMetadataList = + Collections.singletonList(createConsumerMetadata(readonlyConfig)); + } + + return consumerMetadataList.stream() + .collect( + Collectors.toMap( + consumerMetadata -> TablePath.of(consumerMetadata.getTopic()), + consumerMetadata -> consumerMetadata)); } private ConsumerMetadata createConsumerMetadata(ReadonlyConfig readonlyConfig) { ConsumerMetadata consumerMetadata = new ConsumerMetadata(); consumerMetadata.setTopic(readonlyConfig.get(TOPIC)); - consumerMetadata.setBootstrapServers(readonlyConfig.get(BOOTSTRAP_SERVERS)); consumerMetadata.setPattern(readonlyConfig.get(PATTERN)); - consumerMetadata.setProperties(new Properties()); consumerMetadata.setConsumerGroup(readonlyConfig.get(CONSUMER_GROUP)); - consumerMetadata.setCommitOnCheckpoint(readonlyConfig.get(COMMIT_ON_CHECKPOINT)); + consumerMetadata.setProperties(new Properties()); + // Create a catalog + CatalogTable catalogTable = createCatalogTable(readonlyConfig); + consumerMetadata.setCatalogTable(catalogTable); + consumerMetadata.setDeserializationSchema( + createDeserializationSchema(catalogTable, readonlyConfig)); + // parse start mode readonlyConfig .getOptional(START_MODE) @@ -153,25 +185,18 @@ private ConsumerMetadata createConsumerMetadata(ReadonlyConfig readonlyConfig) { } }); - readonlyConfig - .getOptional(KAFKA_CONFIG) - .ifPresent( - kafkaConfig -> - kafkaConfig.forEach( - (key, value) -> - consumerMetadata.getProperties().put(key, value))); - return consumerMetadata; } private CatalogTable createCatalogTable(ReadonlyConfig readonlyConfig) { Optional> schemaOptions = readonlyConfig.getOptional(TableSchemaOptions.SCHEMA); + TablePath tablePath = TablePath.of(readonlyConfig.get(TOPIC)); + TableSchema tableSchema; if (schemaOptions.isPresent()) { - return CatalogTableUtil.buildWithConfig(readonlyConfig); + tableSchema = new ReadonlyConfigParser().parse(readonlyConfig); } else { - TableIdentifier tableIdentifier = TableIdentifier.of(CONNECTOR_IDENTITY, null, null); - TableSchema tableSchema = + tableSchema = TableSchema.builder() .column( PhysicalColumn.of( @@ -186,13 +211,13 @@ private CatalogTable createCatalogTable(ReadonlyConfig readonlyConfig) { null, null)) .build(); - return CatalogTable.of( - tableIdentifier, - tableSchema, - Collections.emptyMap(), - Collections.emptyList(), - null); } + return CatalogTable.of( + TableIdentifier.of("", tablePath), + tableSchema, + Collections.emptyMap(), + Collections.emptyList(), + null); } private DeserializationSchema createDeserializationSchema( @@ -203,13 +228,14 @@ private DeserializationSchema createDeserializationSchema( return TextDeserializationSchema.builder() .seaTunnelRowType(seaTunnelRowType) .delimiter(TextFormatConstant.PLACEHOLDER) + .setCatalogTable(catalogTable) .build(); } MessageFormat format = readonlyConfig.get(FORMAT); switch (format) { case JSON: - return new JsonDeserializationSchema(false, false, seaTunnelRowType); + return new JsonDeserializationSchema(catalogTable, false, false); case TEXT: String delimiter = readonlyConfig.get(FIELD_DELIMITER); return TextDeserializationSchema.builder() @@ -217,11 +243,11 @@ private DeserializationSchema createDeserializationSchema( .delimiter(delimiter) .build(); case CANAL_JSON: - return CanalJsonDeserializationSchema.builder(seaTunnelRowType) + return CanalJsonDeserializationSchema.builder(catalogTable) .setIgnoreParseErrors(true) .build(); case OGG_JSON: - return OggJsonDeserializationSchema.builder(seaTunnelRowType) + return OggJsonDeserializationSchema.builder(catalogTable) .setIgnoreParseErrors(true) .build(); case COMPATIBLE_KAFKA_CONNECT_JSON: @@ -232,12 +258,12 @@ private DeserializationSchema createDeserializationSchema( readonlyConfig.get( KafkaConnectJsonFormatOptions.VALUE_CONVERTER_SCHEMA_ENABLED); return new CompatibleKafkaConnectDeserializationSchema( - seaTunnelRowType, keySchemaEnable, valueSchemaEnable, false, false); + catalogTable, keySchemaEnable, valueSchemaEnable, false, false); case DEBEZIUM_JSON: boolean includeSchema = readonlyConfig.get(DEBEZIUM_RECORD_INCLUDE_SCHEMA); - return new DebeziumJsonDeserializationSchema(seaTunnelRowType, true, includeSchema); + return new DebeziumJsonDeserializationSchema(catalogTable, true, includeSchema); case AVRO: - return new AvroDeserializationSchema(seaTunnelRowType); + return new AvroDeserializationSchema(catalogTable); default: throw new SeaTunnelJsonFormatException( CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java index 4a41d38d6d6..431e9a8c195 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java @@ -42,7 +42,8 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { return OptionRule.builder() - .required(Config.TOPIC, Config.BOOTSTRAP_SERVERS) + .required(Config.BOOTSTRAP_SERVERS) + .exclusive(Config.TOPIC, Config.TABLE_LIST) .optional( Config.START_MODE, Config.PATTERN, diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java index 252bd0746e0..d136fabc403 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormatErrorHandleWay; import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorErrorCode; @@ -56,12 +57,13 @@ public class KafkaSourceReader implements SourceReader tablePathMetadataMap; private final Set sourceSplits; private final Map> checkpointOffsetMap; private final Map consumerThreadMap; private final ExecutorService executorService; - private final DeserializationSchema deserializationSchema; private final MessageFormatErrorHandleWay messageFormatErrorHandleWay; private final LinkedBlockingQueue pendingPartitionsQueue; @@ -69,15 +71,14 @@ public class KafkaSourceReader implements SourceReader deserializationSchema, + KafkaSourceConfig kafkaSourceConfig, Context context, MessageFormatErrorHandleWay messageFormatErrorHandleWay) { - this.metadata = metadata; + this.kafkaSourceConfig = kafkaSourceConfig; + this.tablePathMetadataMap = kafkaSourceConfig.getMapMetadata(); this.context = context; this.messageFormatErrorHandleWay = messageFormatErrorHandleWay; this.sourceSplits = new HashSet<>(); - this.deserializationSchema = deserializationSchema; this.consumerThreadMap = new ConcurrentHashMap<>(); this.checkpointOffsetMap = new ConcurrentHashMap<>(); this.executorService = @@ -110,13 +111,21 @@ public void pollNext(Collector output) throws Exception { consumerThreadMap.computeIfAbsent( sourceSplit.getTopicPartition(), s -> { - KafkaConsumerThread thread = new KafkaConsumerThread(metadata); + ConsumerMetadata currentSplitConsumerMetaData = + tablePathMetadataMap.get(sourceSplit.getTablePath()); + KafkaConsumerThread thread = + new KafkaConsumerThread( + kafkaSourceConfig, + currentSplitConsumerMetaData); executorService.submit(thread); return thread; })); sourceSplits.forEach( sourceSplit -> { CompletableFuture completableFuture = new CompletableFuture<>(); + TablePath tablePath = sourceSplit.getTablePath(); + DeserializationSchema deserializationSchema = + tablePathMetadataMap.get(tablePath).getDeserializationSchema(); try { consumerThreadMap .get(sourceSplit.getTopicPartition()) @@ -251,7 +260,8 @@ public void notifyCheckpointComplete(long checkpointId) { .getTasks() .put( consumer -> { - if (this.metadata.isCommitOnCheckpoint()) { + if (kafkaSourceConfig + .isCommitOnCheckpoint()) { Map offsets = new HashMap<>(); if (offset >= 0) { diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java index f894a917dce..1c7cb17678f 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplit.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.kafka.source; import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.kafka.common.TopicPartition; @@ -25,15 +26,19 @@ public class KafkaSourceSplit implements SourceSplit { + private TablePath tablePath; private TopicPartition topicPartition; private long startOffset = -1L; private long endOffset = -1L; - public KafkaSourceSplit(TopicPartition topicPartition) { + public KafkaSourceSplit(TablePath tablePath, TopicPartition topicPartition) { + this.tablePath = tablePath; this.topicPartition = topicPartition; } - public KafkaSourceSplit(TopicPartition topicPartition, long startOffset, long endOffset) { + public KafkaSourceSplit( + TablePath tablePath, TopicPartition topicPartition, long startOffset, long endOffset) { + this.tablePath = tablePath; this.topicPartition = topicPartition; this.startOffset = startOffset; this.endOffset = endOffset; @@ -63,6 +68,14 @@ public void setTopicPartition(TopicPartition topicPartition) { this.topicPartition = topicPartition; } + public TablePath getTablePath() { + return tablePath; + } + + public void setTablePath(TablePath tablePath) { + this.tablePath = tablePath; + } + @Override public String splitId() { return topicPartition.topic() + "-" + topicPartition.partition(); @@ -87,6 +100,6 @@ public int hashCode() { public KafkaSourceSplit copy() { return new KafkaSourceSplit( - this.topicPartition, this.getStartOffset(), this.getEndOffset()); + this.tablePath, this.topicPartition, this.getStartOffset(), this.getEndOffset()); } } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java index 9ab9f92841d..a7471ae0869 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.kafka.source; import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.common.config.Common; import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorErrorCode; import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorException; @@ -56,46 +57,29 @@ public class KafkaSourceSplitEnumerator private static final String CLIENT_ID_PREFIX = "seatunnel"; - private final ConsumerMetadata metadata; + private final Map tablePathMetadataMap; private final Context context; - private long discoveryIntervalMillis; + private final long discoveryIntervalMillis; private final AdminClient adminClient; - + private final KafkaSourceConfig kafkaSourceConfig; private final Map pendingSplit; private final Map assignedSplit; private ScheduledExecutorService executor; private ScheduledFuture scheduledFuture; - KafkaSourceSplitEnumerator(ConsumerMetadata metadata, Context context) { - this.metadata = metadata; - this.context = context; - this.assignedSplit = new HashMap<>(); - this.pendingSplit = new HashMap<>(); - this.adminClient = initAdminClient(this.metadata.getProperties()); - } + private final Map topicMappingTablePathMap = new HashMap<>(); KafkaSourceSplitEnumerator( - ConsumerMetadata metadata, + KafkaSourceConfig kafkaSourceConfig, Context context, KafkaSourceState sourceState) { - this(metadata, context); - } - - KafkaSourceSplitEnumerator( - ConsumerMetadata metadata, - Context context, - long discoveryIntervalMillis) { - this(metadata, context); - this.discoveryIntervalMillis = discoveryIntervalMillis; - } - - KafkaSourceSplitEnumerator( - ConsumerMetadata metadata, - Context context, - KafkaSourceState sourceState, - long discoveryIntervalMillis) { - this(metadata, context, sourceState); - this.discoveryIntervalMillis = discoveryIntervalMillis; + this.kafkaSourceConfig = kafkaSourceConfig; + this.tablePathMetadataMap = kafkaSourceConfig.getMapMetadata(); + this.context = context; + this.assignedSplit = new HashMap<>(); + this.pendingSplit = new HashMap<>(); + this.adminClient = initAdminClient(this.kafkaSourceConfig.getProperties()); + this.discoveryIntervalMillis = kafkaSourceConfig.getDiscoveryIntervalMillis(); } @Override @@ -133,30 +117,45 @@ public void run() throws ExecutionException, InterruptedException { } private void setPartitionStartOffset() throws ExecutionException, InterruptedException { - Collection topicPartitions = pendingSplit.keySet(); - Map topicPartitionOffsets = null; - switch (metadata.getStartMode()) { - case EARLIEST: - topicPartitionOffsets = listOffsets(topicPartitions, OffsetSpec.earliest()); - break; - case GROUP_OFFSETS: - topicPartitionOffsets = listConsumerGroupOffsets(topicPartitions); - break; - case LATEST: - topicPartitionOffsets = listOffsets(topicPartitions, OffsetSpec.latest()); - break; - case TIMESTAMP: - topicPartitionOffsets = - listOffsets( - topicPartitions, - OffsetSpec.forTimestamp(metadata.getStartOffsetsTimestamp())); - break; - case SPECIFIC_OFFSETS: - topicPartitionOffsets = metadata.getSpecificStartOffsets(); - break; - default: - break; + Set pendingTopicPartitions = pendingSplit.keySet(); + Map topicPartitionOffsets = new HashMap<>(); + // Set kafka TopicPartition based on the topicPath granularity + Map> tablePathPartitionMap = + pendingTopicPartitions.stream() + .collect( + Collectors.groupingBy( + tp -> topicMappingTablePathMap.get(tp.topic()), + Collectors.toSet())); + for (TablePath tablePath : tablePathPartitionMap.keySet()) { + // Supports topic list fine-grained Settings for kafka consumer configurations + ConsumerMetadata metadata = tablePathMetadataMap.get(tablePath); + Set topicPartitions = tablePathPartitionMap.get(tablePath); + switch (metadata.getStartMode()) { + case EARLIEST: + topicPartitionOffsets.putAll( + listOffsets(topicPartitions, OffsetSpec.earliest())); + break; + case GROUP_OFFSETS: + topicPartitionOffsets.putAll( + listConsumerGroupOffsets(topicPartitions, metadata)); + break; + case LATEST: + topicPartitionOffsets.putAll(listOffsets(topicPartitions, OffsetSpec.latest())); + break; + case TIMESTAMP: + topicPartitionOffsets.putAll( + listOffsets( + topicPartitions, + OffsetSpec.forTimestamp(metadata.getStartOffsetsTimestamp()))); + break; + case SPECIFIC_OFFSETS: + topicPartitionOffsets.putAll(metadata.getSpecificStartOffsets()); + break; + default: + break; + } } + topicPartitionOffsets.forEach( (key, value) -> { if (pendingSplit.containsKey(key)) { @@ -237,35 +236,41 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { private AdminClient initAdminClient(Properties properties) { Properties props = new Properties(); - props.putAll(properties); + if (properties != null) { + props.putAll(properties); + } props.setProperty( - ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, this.metadata.getBootstrapServers()); - if (this.metadata.getProperties().get("client.id") == null) { + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaSourceConfig.getBootstrap()); + if (properties.get("client.id") != null) { props.setProperty( - ConsumerConfig.CLIENT_ID_CONFIG, - CLIENT_ID_PREFIX + "-enumerator-admin-client-" + this.hashCode()); + ConsumerConfig.CLIENT_ID_CONFIG, properties.get("client.id").toString()); } else { props.setProperty( ConsumerConfig.CLIENT_ID_CONFIG, - this.metadata.getProperties().get("client.id").toString()); + CLIENT_ID_PREFIX + "-enumerator-admin-client-" + this.hashCode()); } return AdminClient.create(props); } private Set getTopicInfo() throws ExecutionException, InterruptedException { - Collection topics; - if (this.metadata.isPattern()) { - Pattern pattern = Pattern.compile(this.metadata.getTopic()); - topics = - this.adminClient.listTopics().names().get().stream() - .filter(t -> pattern.matcher(t).matches()) - .collect(Collectors.toSet()); - } else { - topics = Arrays.asList(this.metadata.getTopic().split(",")); + Collection topics = new HashSet<>(); + for (TablePath tablePath : tablePathMetadataMap.keySet()) { + ConsumerMetadata metadata = tablePathMetadataMap.get(tablePath); + Set currentPathTopics = new HashSet<>(); + if (metadata.isPattern()) { + Pattern pattern = Pattern.compile(metadata.getTopic()); + currentPathTopics.addAll( + this.adminClient.listTopics().names().get().stream() + .filter(t -> pattern.matcher(t).matches()) + .collect(Collectors.toSet())); + } else { + currentPathTopics.addAll(Arrays.asList(metadata.getTopic().split(","))); + } + currentPathTopics.forEach(topic -> topicMappingTablePathMap.put(topic, tablePath)); + topics.addAll(currentPathTopics); } log.info("Discovered topics: {}", topics); - Collection partitions = adminClient.describeTopics(topics).all().get().values().stream() .flatMap( @@ -280,7 +285,9 @@ private Set getTopicInfo() throws ExecutionException, Interrup return partitions.stream() .map( partition -> { - KafkaSourceSplit split = new KafkaSourceSplit(partition); + // Obtain the corresponding topic TablePath from kafka topic + TablePath tablePath = topicMappingTablePathMap.get(partition.topic()); + KafkaSourceSplit split = new KafkaSourceSplit(tablePath, partition); split.setEndOffset(latestOffsets.get(split.getTopicPartition())); return split; }) @@ -341,7 +348,8 @@ private Map listOffsets( .get(); } - public Map listConsumerGroupOffsets(Collection partitions) + public Map listConsumerGroupOffsets( + Collection partitions, ConsumerMetadata metadata) throws ExecutionException, InterruptedException { ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions().topicPartitions(new ArrayList<>(partitions)); diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java index db0bb24ef01..b998d85762a 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java @@ -27,6 +27,7 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -300,7 +301,8 @@ private void setPartitionDiscoverer(Config config) { private void setDeserialization(Config config) { if (config.hasPath(SCHEMA.key())) { - typeInfo = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); + CatalogTable catalogTable = CatalogTableUtil.buildWithConfig(config); + typeInfo = catalogTable.getSeaTunnelRowType(); String format = FORMAT.defaultValue(); if (config.hasPath(FORMAT.key())) { format = config.getString(FORMAT.key()); @@ -312,7 +314,7 @@ private void setDeserialization(Config config) { case "CANAL_JSON": deserializationSchema = new PulsarCanalDecorator( - CanalJsonDeserializationSchema.builder(typeInfo) + CanalJsonDeserializationSchema.builder(catalogTable) .setIgnoreParseErrors(true) .build()); break; diff --git a/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java b/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java index 0076bbbdbea..7b1ee39fd48 100644 --- a/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java +++ b/seatunnel-connectors-v2/connector-pulsar/src/test/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarCanalDecoratorTest.java @@ -18,6 +18,8 @@ package org.apache.seatunnel.connectors.seatunnel.pulsar.source; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -55,9 +57,10 @@ void decoder() throws IOException { }; SeaTunnelRowType seaTunnelRowType = new SeaTunnelRowType(fieldNames, dataTypes); - + CatalogTable catalogTables = + CatalogTableUtil.getCatalogTable("", "", "", "", seaTunnelRowType); CanalJsonDeserializationSchema canalJsonDeserializationSchema = - CanalJsonDeserializationSchema.builder(seaTunnelRowType).build(); + CanalJsonDeserializationSchema.builder(catalogTables).build(); PulsarCanalDecorator pulsarCanalDecorator = new PulsarCanalDecorator(canalJsonDeserializationSchema); diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java index 049c90c3ecd..5684b21d4f3 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/source/RabbitmqSource.java @@ -28,11 +28,11 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.common.config.CheckResult; import org.apache.seatunnel.common.constants.JobMode; @@ -134,7 +134,7 @@ public void setJobContext(JobContext jobContext) { private void setDeserialization(Config config) { // TODO: format SPI // only support json deserializationSchema - SeaTunnelRowType rowType = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); - this.deserializationSchema = new JsonDeserializationSchema(false, false, rowType); + CatalogTable catalogTable = CatalogTableUtil.buildWithConfig(config); + this.deserializationSchema = new JsonDeserializationSchema(catalogTable, false, false); } } diff --git a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java index 3818848fc15..be1ba32acb9 100644 --- a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java +++ b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java @@ -72,7 +72,7 @@ public RedisSource(ReadonlyConfig readonlyConfig) { this.catalogTable = CatalogTableUtil.buildWithConfig(readonlyConfig); this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); this.deserializationSchema = - new JsonDeserializationSchema(false, false, seaTunnelRowType); + new JsonDeserializationSchema(catalogTable, false, false); } } else { this.catalogTable = CatalogTableUtil.buildSimpleTextTable(); diff --git a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java index ceebd71404a..39052e51721 100644 --- a/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java +++ b/seatunnel-connectors-v2/connector-rocketmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rocketmq/source/RocketMqSource.java @@ -30,6 +30,7 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -76,6 +77,7 @@ public class RocketMqSource private final ConsumerMetadata metadata = new ConsumerMetadata(); private JobContext jobContext; private SeaTunnelRowType typeInfo; + private CatalogTable catalogTable; private DeserializationSchema deserializationSchema; private long discoveryIntervalMillis = ConsumerConfig.KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS.defaultValue(); @@ -215,6 +217,8 @@ public void prepare(Config config) throws PrepareFailException { this.discoveryIntervalMillis = config.getLong(ConsumerConfig.KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS.key()); } + this.catalogTable = CatalogTableUtil.buildWithConfig(config); + this.typeInfo = catalogTable.getSeaTunnelRowType(); // set deserialization setDeserialization(config); @@ -252,14 +256,14 @@ public SourceSplitEnumerator restoreEn private void setDeserialization(Config config) { if (config.hasPath(ConsumerConfig.SCHEMA.key())) { - typeInfo = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); SchemaFormat format = SchemaFormat.JSON; if (config.hasPath(FORMAT.key())) { format = SchemaFormat.find(config.getString(FORMAT.key())); } switch (format) { case JSON: - deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo); + deserializationSchema = + new JsonDeserializationSchema(catalogTable, false, false); break; case TEXT: String delimiter = DEFAULT_FIELD_DELIMITER; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java index bf09aef14b7..25fa9b4aab4 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java @@ -103,7 +103,6 @@ public class KafkaFormatIT extends TestSuiteBase implements TestResource { // ---------------------------Canal Format Parameter--------------------------------------- private static final String CANAL_KAFKA_SINK_TOPIC = "test-canal-sink"; - private static final String CANAL_MYSQL_DATABASE = "canal"; private static final String CANAL_DATA_PATH = "/canal/canal_data.txt"; private static final String CANAL_KAFKA_SOURCE_TOPIC = "test-cdc_mds"; @@ -332,6 +331,24 @@ public void startUp() throws ClassNotFoundException, InterruptedException, IOExc Thread.sleep(20 * 1000); } + @DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "The multi-catalog does not currently support the Spark Flink engine") + @TestTemplate + public void testMultiFormatCheck(TestContainer container) + throws IOException, InterruptedException { + LOG.info( + "====================== Multi Source Format Canal and Ogg Check ======================"); + Container.ExecResult execCanalAndOggResultKafka = + container.executeJob("/multiFormatIT/kafka_multi_source_to_pg.conf"); + Assertions.assertEquals( + 0, + execCanalAndOggResultKafka.getExitCode(), + execCanalAndOggResultKafka.getStderr()); + checkFormatCanalAndOgg(); + } + @TestTemplate public void testFormatCanalCheck(TestContainer container) throws IOException, InterruptedException { @@ -401,23 +418,79 @@ public void testFormatCompatibleCheck(TestContainer container) checkCompatibleFormat(); } + private void checkFormatCanalAndOgg() { + List> postgreSinkTableList = getPostgreSinkTableList(PG_SINK_TABLE1); + List> checkArraysResult = + Stream.>of( + Arrays.asList( + 101, + "scooter", + "Small 2-wheel scooter", + "3.140000104904175"), + Arrays.asList( + 102, "car battery", "12V car battery", "8.100000381469727"), + Arrays.asList( + 103, + "12-pack drill bits", + "12-pack of drill bits with sizes ranging from #40 to #3", + "0.800000011920929"), + Arrays.asList(104, "hammer", "12oz carpenter's hammer", "0.75"), + Arrays.asList(105, "hammer", "14oz carpenter's hammer", "0.875"), + Arrays.asList(106, "hammer", "18oz carpenter hammer", "1"), + Arrays.asList( + 107, "rocks", "box of assorted rocks", "5.099999904632568"), + Arrays.asList( + 108, + "jacket", + "water resistent black wind breaker", + "0.10000000149011612"), + Arrays.asList( + 109, + "spare tire", + "24 inch spare tire", + "22.200000762939453"), + Arrays.asList( + 110, + "jacket", + "new water resistent white wind breaker", + "0.5"), + Arrays.asList(1101, "scooter", "Small 2-wheel scooter", "4.56"), + Arrays.asList(1102, "car battery", "12V car battery", "8.1"), + Arrays.asList( + 1103, + "12-pack drill bits", + "12-pack of drill bits with sizes ranging from #40 to #3", + "0.8"), + Arrays.asList(1104, "hammer", "12oz carpenter's hammer", "0.75"), + Arrays.asList(1105, "hammer", "14oz carpenter's hammer", "0.875"), + Arrays.asList(1106, "hammer", "16oz carpenter's hammer", "1.0"), + Arrays.asList(1107, "rocks", "box of assorted rocks", "7.88"), + Arrays.asList( + 1108, + "jacket", + "water resistent black wind breaker", + "0.1")) + .collect(Collectors.toList()); + Assertions.assertIterableEquals(postgreSinkTableList, checkArraysResult); + } + private void checkCanalFormat() { List expectedResult = Arrays.asList( - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"},\"type\":\"INSERT\"}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":\"8.1\"},\"type\":\"INSERT\"}", - "{\"data\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":\"0.8\"},\"type\":\"INSERT\"}", - "{\"data\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":\"0.75\"},\"type\":\"INSERT\"}", - "{\"data\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":\"0.875\"},\"type\":\"INSERT\"}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":\"1.0\"},\"type\":\"INSERT\"}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"},\"type\":\"INSERT\"}", - "{\"data\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":\"0.1\"},\"type\":\"INSERT\"}", - "{\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"},\"type\":\"INSERT\"}", - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"},\"type\":\"DELETE\"}", - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"4.56\"},\"type\":\"INSERT\"}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"},\"type\":\"DELETE\"}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"7.88\"},\"type\":\"INSERT\"}", - "{\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"},\"type\":\"DELETE\"}"); + "{\"data\":{\"id\":1101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":\"8.1\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":\"0.8\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":\"0.75\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":\"0.875\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":\"1.0\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":\"0.1\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"},\"type\":\"DELETE\"}", + "{\"data\":{\"id\":1101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"4.56\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"},\"type\":\"DELETE\"}", + "{\"data\":{\"id\":1107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"7.88\"},\"type\":\"INSERT\"}", + "{\"data\":{\"id\":1109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"},\"type\":\"DELETE\"}"); ArrayList result = new ArrayList<>(); ArrayList topics = new ArrayList<>(); @@ -440,19 +513,22 @@ private void checkCanalFormat() { List> expected = Stream.>of( - Arrays.asList(101, "scooter", "Small 2-wheel scooter", "4.56"), - Arrays.asList(102, "car battery", "12V car battery", "8.1"), + Arrays.asList(1101, "scooter", "Small 2-wheel scooter", "4.56"), + Arrays.asList(1102, "car battery", "12V car battery", "8.1"), Arrays.asList( - 103, + 1103, "12-pack drill bits", "12-pack of drill bits with sizes ranging from #40 to #3", "0.8"), - Arrays.asList(104, "hammer", "12oz carpenter's hammer", "0.75"), - Arrays.asList(105, "hammer", "14oz carpenter's hammer", "0.875"), - Arrays.asList(106, "hammer", "16oz carpenter's hammer", "1.0"), - Arrays.asList(107, "rocks", "box of assorted rocks", "7.88"), + Arrays.asList(1104, "hammer", "12oz carpenter's hammer", "0.75"), + Arrays.asList(1105, "hammer", "14oz carpenter's hammer", "0.875"), + Arrays.asList(1106, "hammer", "16oz carpenter's hammer", "1.0"), + Arrays.asList(1107, "rocks", "box of assorted rocks", "7.88"), Arrays.asList( - 108, "jacket", "water resistent black wind breaker", "0.1")) + 1108, + "jacket", + "water resistent black wind breaker", + "0.1")) .collect(Collectors.toList()); Assertions.assertIterableEquals(expected, postgreSinkTableList); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java index d86aa891865..2f1c92048e0 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java @@ -20,6 +20,8 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; @@ -370,8 +372,10 @@ public void testFakeSourceToKafkaAvroFormat(TestContainer container) subRow }; SeaTunnelRowType fake_source_row_type = new SeaTunnelRowType(fieldNames, fieldTypes); + CatalogTable catalogTable = + CatalogTableUtil.getCatalogTable("", "", "", "", fake_source_row_type); AvroDeserializationSchema avroDeserializationSchema = - new AvroDeserializationSchema(fake_source_row_type); + new AvroDeserializationSchema(catalogTable); List kafkaSTRow = getKafkaSTRow( "test_avro_topic_fake_source", @@ -420,8 +424,11 @@ public void testKafkaAvroToAssert(TestContainer container) Container.ExecResult execResult = container.executeJob("/avro/kafka_avro_to_assert.conf"); Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + CatalogTable catalogTable = + CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); + AvroDeserializationSchema avroDeserializationSchema = - new AvroDeserializationSchema(SEATUNNEL_ROW_TYPE); + new AvroDeserializationSchema(catalogTable); List kafkaSTRow = getKafkaSTRow( "test_avro_topic", diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/canal/canal_data.txt b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/canal/canal_data.txt index 6b4d4dfe544..d959351555a 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/canal/canal_data.txt +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/canal/canal_data.txt @@ -1,7 +1,7 @@ {"data":null,"database":"canal_hvygfc","es":1697788899000,"id":1,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"CREATE DATABASE IF NOT EXISTS canal_hvygfc","sqlType":null,"table":"","ts":1697788899992,"type":"QUERY"} {"data":null,"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"CREATE TABLE products (\nid INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,\nname VARCHAR(255) NOT NULL DEFAULT 'SeaTunnel',\ndescription VARCHAR(512),\nweight VARCHAR(512)\n)","sqlType":null,"table":"products","ts":1697788900618,"type":"CREATE"} -{"data":null,"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"ALTER TABLE products AUTO_INCREMENT = 101","sqlType":null,"table":"products","ts":1697788900618,"type":"ALTER"} -{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"106","name":"hammer","description":"16oz carpenter's hammer","weight":"1.0"},{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900618,"type":"INSERT"} -{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"4.56"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":[{"weight":"3.14"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900619,"type":"UPDATE"} -{"data":[{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"7.88"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":[{"weight":"5.3"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900619,"type":"UPDATE"} -{"data":[{"id":"109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900619,"type":"DELETE"} \ No newline at end of file +{"data":null,"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"ALTER TABLE products AUTO_INCREMENT = 1101","sqlType":null,"table":"products","ts":1697788900618,"type":"ALTER"} +{"data":[{"id":"1101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"1102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"1103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"1104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"1105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"1106","name":"hammer","description":"16oz carpenter's hammer","weight":"1.0"},{"id":"1107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"1108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"1109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900618,"type":"INSERT"} +{"data":[{"id":"1101","name":"scooter","description":"Small 2-wheel scooter","weight":"4.56"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":[{"weight":"3.14"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900619,"type":"UPDATE"} +{"data":[{"id":"1107","name":"rocks","description":"box of assorted rocks","weight":"7.88"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":[{"weight":"5.3"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900619,"type":"UPDATE"} +{"data":[{"id":"1109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"canal_hvygfc","es":1697788899000,"id":2,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"VARCHAR(512)"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":12},"table":"products","ts":1697788900619,"type":"DELETE"} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf new file mode 100644 index 00000000000..8bc6d41cd3f --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf @@ -0,0 +1,73 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + Kafka { + bootstrap.servers = "kafka_e2e:9092" + table_list = [ + { + topic = "^test-ogg-sou.*" + pattern = "true" + consumer.group = "ogg_multi_group" + start_mode = earliest + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "string" + } + }, + format = ogg_json + }, + { + topic = "test-cdc_mds" + start_mode = earliest + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "string" + } + }, + format = canal_json + } + ] + } +} + +sink { + Jdbc { + driver = org.postgresql.Driver + url = "jdbc:postgresql://postgresql:5432/test?loggerLevel=OFF" + user = test + password = test + generate_sink_sql = true + database = test + table = public.sink + primary_keys = ["id"] + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/ogg/ogg_data.txt b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/ogg/ogg_data.txt index 2576267a5bf..0c470fd7c45 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/ogg/ogg_data.txt +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/ogg/ogg_data.txt @@ -13,4 +13,4 @@ {"table":"OGG.OGG_TEST","pos":"00000000000000000000155","primary_keys":["id"],"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"op_type":"I","op_ts":"2020-05-13 17:30:43.428000"} {"table":"OGG.OGG_TEST","pos":"00000000000000000000156","primary_keys":["id"],"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"op_type":"U","op_ts":"2020-05-13 17:32:20.327000"} {"table":"OGG.OGG_TEST","pos":"00000000000000000000157","primary_keys":["id"],"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"op_type":"U","op_ts":"2020-05-13 17:32:10.904000"} -{"table":"OGG.OGG_TEST","pos":"00000000000000000000158","primary_keys":["id"],"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"after":null,"op_type":"D","op_ts":"2020-05-13 17:32:24.455000"} +{"table":"OGG.OGG_TEST","pos":"00000000000000000000158","primary_keys":["id"],"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"after":null,"op_type":"D","op_ts":"2020-05-13 17:32:24.455000"} \ No newline at end of file diff --git a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java index b682a8e6431..32396c245ea 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroDeserializationSchema.java @@ -18,6 +18,8 @@ package org.apache.seatunnel.format.avro; import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -27,6 +29,7 @@ import org.apache.avro.io.DecoderFactory; import java.io.IOException; +import java.util.Optional; public class AvroDeserializationSchema implements DeserializationSchema { @@ -34,9 +37,11 @@ public class AvroDeserializationSchema implements DeserializationSchema tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath); + if (tablePath.isPresent()) { + seaTunnelRow.setTableId(tablePath.toString()); + } + return seaTunnelRow; } @Override diff --git a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java index 8b2eadaf1e1..1e1554be712 100644 --- a/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.format.avro; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; @@ -38,13 +40,14 @@ class AvroSerializationSchemaTest { - private LocalDate localDate = LocalDate.of(2023, 1, 1); - private BigDecimal bigDecimal = new BigDecimal("61592600349703735722.724745739637773662"); - private LocalDateTime localDateTime = LocalDateTime.of(2023, 1, 1, 6, 30, 40); + private static final LocalDate localDate = LocalDate.of(2023, 1, 1); + private static final BigDecimal bigDecimal = + new BigDecimal("61592600349703735722.724745739637773662"); + private static final LocalDateTime localDateTime = LocalDateTime.of(2023, 1, 1, 6, 30, 40); private SeaTunnelRow buildSeaTunnelRow() { SeaTunnelRow subSeaTunnelRow = new SeaTunnelRow(14); - Map map = new HashMap(); + Map map = new HashMap<>(); map.put("k1", "v1"); map.put("k2", "v2"); String[] strArray = new String[] {"l1", "l2"}; @@ -116,7 +119,6 @@ private SeaTunnelRowType buildSeaTunnelRowType() { new DecimalType(38, 18), LocalTimeType.LOCAL_DATE_TIME_TYPE }; - SeaTunnelRowType subRow = new SeaTunnelRowType(subField, subFieldTypes); String[] fieldNames = { "c_map", @@ -150,19 +152,20 @@ private SeaTunnelRowType buildSeaTunnelRowType() { LocalTimeType.LOCAL_DATE_TYPE, new DecimalType(38, 18), LocalTimeType.LOCAL_DATE_TIME_TYPE, - subRow + new SeaTunnelRowType(subField, subFieldTypes) }; - SeaTunnelRowType rowType = new SeaTunnelRowType(fieldNames, fieldTypes); - return rowType; + return new SeaTunnelRowType(fieldNames, fieldTypes); } @Test public void testSerialization() throws IOException { SeaTunnelRowType rowType = buildSeaTunnelRowType(); + CatalogTable catalogTable = CatalogTableUtil.getCatalogTable("", "", "", "", rowType); SeaTunnelRow seaTunnelRow = buildSeaTunnelRow(); AvroSerializationSchema serializationSchema = new AvroSerializationSchema(rowType); byte[] bytes = serializationSchema.serialize(seaTunnelRow); - AvroDeserializationSchema deserializationSchema = new AvroDeserializationSchema(rowType); + AvroDeserializationSchema deserializationSchema = + new AvroDeserializationSchema(catalogTable); SeaTunnelRow deserialize = deserializationSchema.deserialize(bytes); String[] strArray1 = (String[]) seaTunnelRow.getField(1); String[] strArray2 = (String[]) deserialize.getField(1); diff --git a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java index 4d00725dd50..256d41b3574 100644 --- a/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-compatible-connect-json/src/main/java/org/apache/seatunnel/format/compatible/kafka/connect/json/CompatibleKafkaConnectDeserializationSchema.java @@ -21,6 +21,8 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -42,10 +44,10 @@ import lombok.RequiredArgsConstructor; import java.io.IOException; -import java.io.UnsupportedEncodingException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.Collections; +import java.util.Optional; import static com.google.common.base.Preconditions.checkNotNull; @@ -69,17 +71,18 @@ public class CompatibleKafkaConnectDeserializationSchema /** Object mapper for parsing the JSON. */ private final ObjectMapper objectMapper = new ObjectMapper(); + private final CatalogTable catalogTable; + public CompatibleKafkaConnectDeserializationSchema( - @NonNull SeaTunnelRowType seaTunnelRowType, + @NonNull CatalogTable catalogTable, boolean keySchemaEnable, boolean valueSchemaEnable, boolean failOnMissingField, boolean ignoreParseErrors) { - - this.seaTunnelRowType = seaTunnelRowType; + this.catalogTable = catalogTable; + this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); this.keySchemaEnable = keySchemaEnable; this.valueSchemaEnable = valueSchemaEnable; - // Runtime converter this.runtimeConverter = new JsonToRowConverters(failOnMissingField, ignoreParseErrors) @@ -88,7 +91,8 @@ public CompatibleKafkaConnectDeserializationSchema( @Override public SeaTunnelRow deserialize(byte[] message) throws IOException { - throw new UnsupportedEncodingException(); + throw new UnsupportedOperationException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); } /** @@ -101,6 +105,9 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { public void deserialize(ConsumerRecord msg, Collector out) throws InvocationTargetException, IllegalAccessException { tryInitConverter(); + if (msg == null) { + return; + } SinkRecord record = convertToSinkRecord(msg); RowKind rowKind = RowKind.INSERT; JsonNode jsonNode = @@ -108,16 +115,24 @@ public void deserialize(ConsumerRecord msg, Collector tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath); if (payload.isArray()) { ArrayNode arrayNode = (ArrayNode) payload; for (int i = 0; i < arrayNode.size(); i++) { SeaTunnelRow row = convertJsonNode(arrayNode.get(i)); row.setRowKind(rowKind); + if (tablePath.isPresent()) { + row.setTableId(tablePath.toString()); + } out.collect(row); } } else { SeaTunnelRow row = convertJsonNode(payload); row.setRowKind(rowKind); + if (tablePath.isPresent()) { + row.setTableId(tablePath.toString()); + } out.collect(row); } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java index afc2f070c60..6580f3d49e4 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonDeserializationSchema.java @@ -28,6 +28,8 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.CompositeType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -38,6 +40,7 @@ import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; import java.io.IOException; +import java.util.Optional; import static com.google.common.base.Preconditions.checkNotNull; @@ -58,11 +61,13 @@ public class JsonDeserializationSchema implements DeserializationSchema dataType) { if (dataType.getSqlType() == SqlType.DECIMAL) { return true; @@ -119,14 +145,24 @@ public void collect(byte[] message, Collector out) throws IOExcept ArrayNode arrayNode = (ArrayNode) jsonNode; for (int i = 0; i < arrayNode.size(); i++) { SeaTunnelRow deserialize = convertJsonNode(arrayNode.get(i)); + setCollectorTablePath(deserialize, catalogTable); out.collect(deserialize); } } else { SeaTunnelRow deserialize = convertJsonNode(jsonNode); + setCollectorTablePath(deserialize, catalogTable); out.collect(deserialize); } } + public void setCollectorTablePath(SeaTunnelRow deserialize, CatalogTable catalogTable) { + Optional tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath); + if (tablePath.isPresent()) { + deserialize.setTableId(tablePath.toString()); + } + } + private SeaTunnelRow convertJsonNode(JsonNode jsonNode) { if (jsonNode.isNull()) { return null; diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java index c33e586c7d5..77055a2f8ea 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonDeserializationSchema.java @@ -24,6 +24,8 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -32,7 +34,10 @@ import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.format.json.JsonDeserializationSchema; +import lombok.NonNull; + import java.io.IOException; +import java.util.Optional; import java.util.regex.Pattern; import static java.lang.String.format; @@ -64,9 +69,9 @@ public class CanalJsonDeserializationSchema implements DeserializationSchema getProducedType() { - return this.physicalRowType; + return this.seaTunnelRowType; } public void deserialize(ObjectNode jsonNode, Collector out) throws IOException { + TablePath tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath).orElse(null); + try { if (database != null && !databasePattern.matcher(jsonNode.get(FIELD_DATABASE).asText()).matches()) { @@ -127,47 +136,65 @@ public void deserialize(ObjectNode jsonNode, Collector out) throws } JsonNode dataNode = jsonNode.get(FIELD_DATA); - String type = jsonNode.get(FIELD_TYPE).asText(); + String op = jsonNode.get(FIELD_TYPE).asText(); // When a null value is encountered, an exception needs to be thrown for easy sensing if (dataNode == null || dataNode.isNull()) { // We'll skip the query or create or alter event data - if (OP_QUERY.equals(type) || OP_CREATE.equals(type) || OP_ALTER.equals(type)) { + if (OP_QUERY.equals(op) || OP_CREATE.equals(op) || OP_ALTER.equals(op)) { return; } throw new IllegalStateException( format("Null data value '%s' Cannot send downstream", jsonNode)); } - if (OP_INSERT.equals(type)) { - for (int i = 0; i < dataNode.size(); i++) { - SeaTunnelRow row = convertJsonNode(dataNode.get(i)); - out.collect(row); - } - } else if (OP_UPDATE.equals(type)) { - final ArrayNode oldNode = (ArrayNode) jsonNode.get(FIELD_OLD); - for (int i = 0; i < dataNode.size(); i++) { - SeaTunnelRow after = convertJsonNode(dataNode.get(i)); - SeaTunnelRow before = convertJsonNode(oldNode.get(i)); - for (int f = 0; f < fieldCount; f++) { - if (before.isNullAt(f) && oldNode.findValue(fieldNames[f]) == null) { - // fields in "old" (before) means the fields are changed - // fields not in "old" (before) means the fields are not changed - // so we just copy the not changed fields into before - before.setField(f, after.getField(f)); + + switch (op) { + case OP_INSERT: + for (int i = 0; i < dataNode.size(); i++) { + SeaTunnelRow row = convertJsonNode(dataNode.get(i)); + if (tablePath != null && !tablePath.toString().isEmpty()) { + row.setTableId(tablePath.toString()); } + out.collect(row); } - before.setRowKind(RowKind.UPDATE_BEFORE); - after.setRowKind(RowKind.UPDATE_AFTER); - out.collect(before); - out.collect(after); - } - } else if (OP_DELETE.equals(type)) { - for (int i = 0; i < dataNode.size(); i++) { - SeaTunnelRow row = convertJsonNode(dataNode.get(i)); - row.setRowKind(RowKind.DELETE); - out.collect(row); - } - } else { - throw new IllegalStateException(format("Unknown operation type '%s'.", type)); + break; + case OP_UPDATE: + final ArrayNode oldNode = (ArrayNode) jsonNode.get(FIELD_OLD); + for (int i = 0; i < dataNode.size(); i++) { + SeaTunnelRow after = convertJsonNode(dataNode.get(i)); + SeaTunnelRow before = convertJsonNode(oldNode.get(i)); + for (int f = 0; f < fieldCount; f++) { + if (before.isNullAt(f) && oldNode.findValue(fieldNames[f]) == null) { + // fields in "old" (before) means the fields are changed + // fields not in "old" (before) means the fields are not changed + // so we just copy the not changed fields into before + before.setField(f, after.getField(f)); + } + } + before.setRowKind(RowKind.UPDATE_BEFORE); + if (tablePath != null && !tablePath.toString().isEmpty()) { + before.setTableId(tablePath.toString()); + } + after.setRowKind(RowKind.UPDATE_AFTER); + if (tablePath != null && !tablePath.toString().isEmpty()) { + after.setTableId(tablePath.toString()); + } + out.collect(before); + out.collect(after); + } + break; + case OP_DELETE: + for (int i = 0; i < dataNode.size(); i++) { + SeaTunnelRow row = convertJsonNode(dataNode.get(i)); + row.setRowKind(RowKind.DELETE); + if (tablePath != null && !tablePath.toString().isEmpty()) { + row.setTableId(tablePath.toString()); + } + out.collect(row); + } + break; + default: + throw new IllegalStateException( + String.format("Unknown operation type '%s'.", op)); } } catch (RuntimeException e) { if (!ignoreParseErrors) { @@ -177,31 +204,26 @@ public void deserialize(ObjectNode jsonNode, Collector out) throws } private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException { + if (message == null || message.length == 0) { + return null; + } + try { return (ObjectNode) jsonDeserializer.deserializeToJsonNode(message); } catch (Throwable t) { - throw CommonError.jsonOperationError(FORMAT, new String(message), t); + if (!ignoreParseErrors) { + throw CommonError.jsonOperationError(FORMAT, new String(message), t); + } + return null; } } @Override public void deserialize(byte[] message, Collector out) throws IOException { - if (message == null) { - return; + ObjectNode jsonNodes = convertBytes(message); + if (jsonNodes != null) { + deserialize(convertBytes(message), out); } - - ObjectNode jsonNode; - try { - jsonNode = convertBytes(message); - } catch (SeaTunnelRuntimeException cause) { - if (!ignoreParseErrors) { - throw cause; - } else { - return; - } - } - - deserialize(jsonNode, out); } private SeaTunnelRow convertJsonNode(JsonNode root) { @@ -218,8 +240,8 @@ private static SeaTunnelRowType createJsonRowType(SeaTunnelRowType physicalDataT // ------------------------------------------------------------------------------------------ /** Creates A builder for building a {@link CanalJsonDeserializationSchema}. */ - public static Builder builder(SeaTunnelRowType physicalDataType) { - return new Builder(physicalDataType); + public static Builder builder(CatalogTable catalogTable) { + return new Builder(catalogTable); } public static class Builder { @@ -230,10 +252,10 @@ public static class Builder { private String table = null; - private final SeaTunnelRowType physicalDataType; + private CatalogTable catalogTable; - public Builder(SeaTunnelRowType physicalDataType) { - this.physicalDataType = physicalDataType; + public Builder(CatalogTable catalogTable) { + this.catalogTable = catalogTable; } public Builder setDatabase(String database) { @@ -251,9 +273,14 @@ public Builder setIgnoreParseErrors(boolean ignoreParseErrors) { return this; } + public Builder setCatalogTable(CatalogTable catalogTable) { + this.catalogTable = catalogTable; + return this; + } + public CanalJsonDeserializationSchema build() { return new CanalJsonDeserializationSchema( - physicalDataType, database, table, ignoreParseErrors); + catalogTable, database, table, ignoreParseErrors); } } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java index f51a5171fd6..0329c68e643 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java @@ -21,6 +21,8 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -29,6 +31,7 @@ import org.apache.seatunnel.format.json.JsonDeserializationSchema; import java.io.IOException; +import java.util.Optional; import static java.lang.String.format; @@ -57,23 +60,28 @@ public class DebeziumJsonDeserializationSchema implements DeserializationSchema< private final boolean debeziumEnabledSchema; - public DebeziumJsonDeserializationSchema(SeaTunnelRowType rowType, boolean ignoreParseErrors) { - this.rowType = rowType; + private CatalogTable catalogTable; + + public DebeziumJsonDeserializationSchema(CatalogTable catalogTable, boolean ignoreParseErrors) { + this.catalogTable = catalogTable; + this.rowType = catalogTable.getSeaTunnelRowType(); this.ignoreParseErrors = ignoreParseErrors; this.jsonDeserializer = - new JsonDeserializationSchema(false, ignoreParseErrors, createJsonRowType(rowType)); + new JsonDeserializationSchema(catalogTable, false, ignoreParseErrors); this.debeziumRowConverter = new DebeziumRowConverter(rowType); this.debeziumEnabledSchema = false; } public DebeziumJsonDeserializationSchema( - SeaTunnelRowType rowType, boolean ignoreParseErrors, boolean debeziumEnabledSchema) { - this.rowType = rowType; + CatalogTable catalogTable, boolean ignoreParseErrors, boolean debeziumEnabledSchema) { + this.catalogTable = catalogTable; + this.rowType = catalogTable.getSeaTunnelRowType(); this.ignoreParseErrors = ignoreParseErrors; this.jsonDeserializer = - new JsonDeserializationSchema(false, ignoreParseErrors, createJsonRowType(rowType)); + new JsonDeserializationSchema(catalogTable, false, ignoreParseErrors); this.debeziumRowConverter = new DebeziumRowConverter(rowType); this.debeziumEnabledSchema = debeziumEnabledSchema; + this.catalogTable = catalogTable; } @Override @@ -83,7 +91,14 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { } @Override - public void deserialize(byte[] message, Collector out) throws IOException { + public void deserialize(byte[] message, Collector out) { + TablePath tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath).orElse(null); + deserializeMessage(message, out, tablePath); + } + + private void deserializeMessage( + byte[] message, Collector out, TablePath tablePath) { if (message == null || message.length == 0) { // skip tombstone messages return; @@ -93,32 +108,50 @@ public void deserialize(byte[] message, Collector out) throws IOEx JsonNode payload = getPayload(convertBytes(message)); String op = payload.get("op").asText(); - if (OP_CREATE.equals(op) || OP_READ.equals(op)) { - SeaTunnelRow insert = convertJsonNode(payload.get("after")); - insert.setRowKind(RowKind.INSERT); - out.collect(insert); - } else if (OP_UPDATE.equals(op)) { - SeaTunnelRow before = convertJsonNode(payload.get("before")); - if (before == null) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); - } - before.setRowKind(RowKind.UPDATE_BEFORE); - out.collect(before); - - SeaTunnelRow after = convertJsonNode(payload.get("after")); - after.setRowKind(RowKind.UPDATE_AFTER); - out.collect(after); - } else if (OP_DELETE.equals(op)) { - SeaTunnelRow delete = convertJsonNode(payload.get("before")); - if (delete == null) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); - } - delete.setRowKind(RowKind.DELETE); - out.collect(delete); - } else { - throw new IllegalStateException(format("Unknown operation type '%s'.", op)); + switch (op) { + case OP_CREATE: + case OP_READ: + SeaTunnelRow insert = convertJsonNode(payload.get("after")); + insert.setRowKind(RowKind.INSERT); + if (tablePath != null) { + insert.setTableId(tablePath.toString()); + } + out.collect(insert); + break; + case OP_UPDATE: + SeaTunnelRow before = convertJsonNode(payload.get("before")); + if (before == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } + before.setRowKind(RowKind.UPDATE_BEFORE); + if (tablePath != null) { + before.setTableId(tablePath.toString()); + } + out.collect(before); + + SeaTunnelRow after = convertJsonNode(payload.get("after")); + after.setRowKind(RowKind.UPDATE_AFTER); + + if (tablePath != null) { + after.setTableId(tablePath.toString()); + } + out.collect(after); + break; + case OP_DELETE: + SeaTunnelRow delete = convertJsonNode(payload.get("before")); + if (delete == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } + delete.setRowKind(RowKind.DELETE); + if (tablePath != null) { + delete.setTableId(tablePath.toString()); + } + out.collect(delete); + break; + default: + throw new IllegalStateException(format("Unknown operation type '%s'.", op)); } } catch (RuntimeException e) { // a big try catch to protect the processing. diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java index c618a120df4..14f3b5944ed 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonDeserializationSchema.java @@ -23,6 +23,8 @@ import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -31,11 +33,12 @@ import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.format.json.JsonDeserializationSchema; +import lombok.NonNull; + import java.io.IOException; +import java.util.Optional; import java.util.regex.Pattern; -import static java.lang.String.format; - public class OggJsonDeserializationSchema implements DeserializationSchema { private static final long serialVersionUID = 1L; @@ -61,9 +64,9 @@ public class OggJsonDeserializationSchema implements DeserializationSchema getProducedType() { - return this.physicalRowType; + return this.seaTunnelRowType; } - private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException { + public void deserializeMessage( + byte[] message, Collector out, TablePath tablePath) { + + if (message == null || message.length == 0) { + // skip tombstone messages + return; + } + + ObjectNode jsonNode; try { - return (ObjectNode) jsonDeserializer.deserializeToJsonNode(message); - } catch (Throwable t) { - throw CommonError.jsonOperationError(FORMAT, new String(message), t); + jsonNode = convertBytes(message); + } catch (RuntimeException e) { + if (!ignoreParseErrors) { + throw e; + } else { + return; + } } - } - public void deserialize(ObjectNode jsonNode, Collector out) throws IOException { try { if (database != null && !databasePattern @@ -136,46 +151,64 @@ public void deserialize(ObjectNode jsonNode, Collector out) throws } String op = jsonNode.get(FIELD_TYPE).asText().trim(); - if (OP_INSERT.equals(op)) { - // Gets the data for the INSERT operation - JsonNode dataAfter = jsonNode.get(DATA_AFTER); - SeaTunnelRow row = convertJsonNode(dataAfter); - out.collect(row); - } else if (OP_UPDATE.equals(op)) { - JsonNode dataBefore = jsonNode.get(DATA_BEFORE); - // Modify Operation Data cannot be empty before modification - if (dataBefore == null || dataBefore.isNull()) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); - } - JsonNode dataAfter = jsonNode.get(DATA_AFTER); - // Gets the data for the UPDATE BEFORE operation - SeaTunnelRow before = convertJsonNode(dataBefore); - // Gets the data for the UPDATE AFTER operation - SeaTunnelRow after = convertJsonNode(dataAfter); - - before.setRowKind(RowKind.UPDATE_BEFORE); - out.collect(before); - - after.setRowKind(RowKind.UPDATE_AFTER); - out.collect(after); - } else if (OP_DELETE.equals(op)) { - JsonNode dataBefore = jsonNode.get(DATA_BEFORE); - if (dataBefore == null || dataBefore.isNull()) { - throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); - } - // Gets the data for the DELETE BEFORE operation - SeaTunnelRow before = convertJsonNode(dataBefore); - if (before == null) { + + switch (op) { + case OP_INSERT: + // Gets the data for the INSERT operation + JsonNode dataInsert = jsonNode.get(DATA_AFTER); + SeaTunnelRow row = convertJsonNode(dataInsert); + if (tablePath != null) { + row.setTableId(tablePath.toString()); + } + out.collect(row); + break; + case OP_UPDATE: + JsonNode dataBefore = jsonNode.get(DATA_BEFORE); + // Modify Operation Data cannot be empty before modification + if (dataBefore == null || dataBefore.isNull()) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } + JsonNode dataAfter = jsonNode.get(DATA_AFTER); + // Gets the data for the UPDATE BEFORE operation + SeaTunnelRow before = convertJsonNode(dataBefore); + // Gets the data for the UPDATE AFTER operation + SeaTunnelRow after = convertJsonNode(dataAfter); + before.setRowKind(RowKind.UPDATE_BEFORE); + if (tablePath != null) { + before.setTableId(tablePath.toString()); + } + out.collect(before); + + after.setRowKind(RowKind.UPDATE_AFTER); + if (tablePath != null) { + after.setTableId(tablePath.toString()); + } + out.collect(after); + break; + case OP_DELETE: + JsonNode dataBeforeDel = jsonNode.get(DATA_BEFORE); + if (dataBeforeDel == null || dataBeforeDel.isNull()) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); + } + // Gets the data for the DELETE BEFORE operation + SeaTunnelRow beforeDelete = convertJsonNode(dataBeforeDel); + if (beforeDelete == null) { + throw new IllegalStateException( + String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); + } + beforeDelete.setRowKind(RowKind.DELETE); + if (tablePath != null) { + beforeDelete.setTableId(tablePath.toString()); + } + out.collect(beforeDelete); + break; + default: throw new IllegalStateException( - String.format(REPLICA_IDENTITY_EXCEPTION, "DELETE")); - } - before.setRowKind(RowKind.DELETE); - out.collect(before); - } else { - throw new IllegalStateException(format("Unknown operation type '%s'.", op)); + String.format("Unknown operation type '%s'.", op)); } + } catch (RuntimeException e) { if (!ignoreParseErrors) { throw CommonError.jsonOperationError(FORMAT, jsonNode.toString(), e); @@ -183,23 +216,19 @@ public void deserialize(ObjectNode jsonNode, Collector out) throws } } - public void deserialize(byte[] message, Collector out) throws IOException { - if (message == null || message.length == 0) { - // skip tombstone messages - return; - } - - ObjectNode jsonNode; + private ObjectNode convertBytes(byte[] message) throws SeaTunnelRuntimeException { try { - jsonNode = convertBytes(message); - } catch (RuntimeException e) { - if (!ignoreParseErrors) { - throw e; - } else { - return; - } + return (ObjectNode) jsonDeserializer.deserializeToJsonNode(message); + } catch (Throwable t) { + throw CommonError.jsonOperationError(FORMAT, new String(message), t); } - deserialize(jsonNode, out); + } + + @Override + public void deserialize(byte[] message, Collector out) { + TablePath tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath).orElse(null); + deserializeMessage(message, out, tablePath); } private SeaTunnelRow convertJsonNode(JsonNode root) { @@ -216,8 +245,8 @@ private static SeaTunnelRowType createJsonRowType(SeaTunnelRowType physicalDataT // ------------------------------------------------------------------------------------------ /** Creates A builder for building a {@link OggJsonDeserializationSchema}. */ - public static Builder builder(SeaTunnelRowType physicalDataType) { - return new Builder(physicalDataType); + public static Builder builder(CatalogTable catalogTable) { + return new Builder(catalogTable); } public static class Builder { @@ -228,10 +257,10 @@ public static class Builder { private String table = null; - private final SeaTunnelRowType physicalDataType; + private CatalogTable catalogTable; - public Builder(SeaTunnelRowType physicalDataType) { - this.physicalDataType = physicalDataType; + public Builder(CatalogTable catalogTable) { + this.catalogTable = catalogTable; } public Builder setDatabase(String database) { @@ -251,7 +280,7 @@ public Builder setIgnoreParseErrors(boolean ignoreParseErrors) { public OggJsonDeserializationSchema build() { return new OggJsonDeserializationSchema( - physicalDataType, database, table, ignoreParseErrors); + catalogTable, database, table, ignoreParseErrors); } } } diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java index f552d7519ac..1ec184845d4 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/JsonRowDataSerDeSchemaTest.java @@ -24,6 +24,8 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ArrayNode; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.DecimalType; import org.apache.seatunnel.api.table.type.LocalTimeType; import org.apache.seatunnel.api.table.type.MapType; @@ -166,9 +168,9 @@ public void testSerDe() throws Exception { new MapType(STRING_TYPE, new MapType(STRING_TYPE, INT_TYPE)) }) }); - + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); JsonDeserializationSchema deserializationSchema = - new JsonDeserializationSchema(false, false, schema); + new JsonDeserializationSchema(catalogTables, false, false); SeaTunnelRow expected = new SeaTunnelRow(13); expected.setField(0, true); @@ -226,8 +228,10 @@ public void testSerDeMultiRows() throws Exception { new SeaTunnelDataType[] {STRING_TYPE, INT_TYPE}) }); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + JsonDeserializationSchema deserializationSchema = - new JsonDeserializationSchema(false, false, schema); + new JsonDeserializationSchema(catalogTables, false, false); JsonSerializationSchema serializationSchema = new JsonSerializationSchema(schema); ObjectMapper objectMapper = new ObjectMapper(); @@ -302,8 +306,11 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { new MapType(STRING_TYPE, DOUBLE_TYPE) }); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", rowType); + JsonDeserializationSchema deserializationSchema = - new JsonDeserializationSchema(false, true, rowType); + new JsonDeserializationSchema(catalogTables, false, true); + JsonSerializationSchema serializationSchema = new JsonSerializationSchema(rowType); for (int i = 0; i < jsons.length; i++) { @@ -318,8 +325,10 @@ public void testSerDeMultiRowsWithNullValues() throws Exception { public void testDeserializationNullRow() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + JsonDeserializationSchema deserializationSchema = - new JsonDeserializationSchema(true, false, schema); + new JsonDeserializationSchema(catalogTables, true, false); String s = null; assertNull(deserializationSchema.deserialize(s)); } @@ -328,9 +337,10 @@ public void testDeserializationNullRow() throws Exception { public void testDeserializationMissingNode() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); JsonDeserializationSchema deserializationSchema = - new JsonDeserializationSchema(true, false, schema); + new JsonDeserializationSchema(catalogTables, true, false); SeaTunnelRow rowData = deserializationSchema.deserialize("".getBytes()); assertEquals(null, rowData); } @@ -347,8 +357,11 @@ public void testDeserializationPassMissingField() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + // pass on missing field - final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, false, schema); + final JsonDeserializationSchema deser = + new JsonDeserializationSchema(catalogTables, false, false); SeaTunnelRow expected = new SeaTunnelRow(1); SeaTunnelRow actual = deser.deserialize(serializedJson); @@ -367,8 +380,11 @@ public void testDeserializationMissingField() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + // fail on missing field - final JsonDeserializationSchema deser = new JsonDeserializationSchema(true, false, schema); + final JsonDeserializationSchema deser = + new JsonDeserializationSchema(catalogTables, true, false); SeaTunnelRuntimeException expected = CommonError.jsonOperationError("Common", root.toString()); @@ -400,9 +416,11 @@ public void testDeserializationIgnoreParseError() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); SeaTunnelRow expected = new SeaTunnelRow(1); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); // ignore on parse error - final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, true, schema); + final JsonDeserializationSchema deser = + new JsonDeserializationSchema(catalogTables, false, true); assertEquals(expected, deser.deserialize(serializedJson)); } @@ -415,7 +433,7 @@ public void testDeserializationFailOnMissingFieldIgnoreParseError() throws Excep assertThrows( SeaTunnelJsonFormatException.class, () -> { - new JsonDeserializationSchema(true, true, null); + new JsonDeserializationSchema(null, true, true); }, "expecting exception message: " + errorMessage); assertEquals(actual.getMessage(), errorMessage); @@ -426,8 +444,11 @@ public void testDeserializationNoJson() throws Exception { SeaTunnelRowType schema = new SeaTunnelRowType(new String[] {"name"}, new SeaTunnelDataType[] {STRING_TYPE}); + CatalogTable catalogTables = CatalogTableUtil.getCatalogTable("", "", "", "", schema); + String noJson = "{]"; - final JsonDeserializationSchema deser = new JsonDeserializationSchema(false, false, schema); + final JsonDeserializationSchema deser = + new JsonDeserializationSchema(catalogTables, false, false); SeaTunnelRuntimeException expected = CommonError.jsonOperationError("Common", noJson); SeaTunnelRuntimeException actual = diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java index 601e9c738c6..d35849e8bd4 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/canal/CanalJsonSerDeSchemaTest.java @@ -19,6 +19,8 @@ package org.apache.seatunnel.format.json.canal; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -47,16 +49,18 @@ public class CanalJsonSerDeSchemaTest { private static final String FORMAT = "Canal"; - private static final SeaTunnelRowType PHYSICAL_DATA_TYPE = + private static final SeaTunnelRowType SEATUNNEL_ROW_TYPE = new SeaTunnelRowType( new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); + private static final CatalogTable catalogTables = + CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); @Test public void testFilteringTables() throws Exception { List lines = readLines("canal-data-filter-table.txt"); CanalJsonDeserializationSchema deserializationSchema = - new CanalJsonDeserializationSchema.Builder(PHYSICAL_DATA_TYPE) + new CanalJsonDeserializationSchema.Builder(catalogTables) .setDatabase("^my.*") .setTable("^prod.*") .build(); @@ -163,39 +167,39 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali List expected = Arrays.asList( - "SeaTunnelRow{tableId=, kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[106, hammer, null, 1.0]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[106, hammer, null, 1.0]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=, kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[101, scooter, Small 2-wheel scooter, 5.17]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[102, car battery, 12V car battery, 5.17]}", - "SeaTunnelRow{tableId=, kind=-D, fields=[102, car battery, 12V car battery, 5.17]}", - "SeaTunnelRow{tableId=, kind=-D, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}"); + "SeaTunnelRow{tableId=.., kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[106, hammer, null, 1.0]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[106, hammer, null, 1.0]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=.., kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[101, scooter, Small 2-wheel scooter, 5.17]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[102, car battery, 12V car battery, 5.17]}", + "SeaTunnelRow{tableId=.., kind=-D, fields=[102, car battery, 12V car battery, 5.17]}", + "SeaTunnelRow{tableId=.., kind=-D, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); assertEquals(expected, actual); // test Serialization CanalJsonSerializationSchema serializationSchema = - new CanalJsonSerializationSchema(PHYSICAL_DATA_TYPE); + new CanalJsonSerializationSchema(SEATUNNEL_ROW_TYPE); List result = new ArrayList<>(); for (SeaTunnelRow rowData : collector.list) { result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); @@ -238,7 +242,7 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali private CanalJsonDeserializationSchema createCanalJsonDeserializationSchema( String database, String table) { - return CanalJsonDeserializationSchema.builder(PHYSICAL_DATA_TYPE) + return CanalJsonDeserializationSchema.builder(catalogTables) .setDatabase(database) .setTable(table) .setIgnoreParseErrors(false) diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java index e74e58e19e0..67d499efd91 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -19,6 +19,8 @@ package org.apache.seatunnel.format.json.debezium; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -48,15 +50,17 @@ public class DebeziumJsonSerDeSchemaTest { private static final String FORMAT = "Debezium"; - private static final SeaTunnelRowType PHYSICAL_DATA_TYPE = + private static final SeaTunnelRowType SEATUNNEL_ROW_TYPE = new SeaTunnelRowType( new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); + private static final CatalogTable catalogTables = + CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); @Test void testNullRowMessages() throws Exception { DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); + new DebeziumJsonDeserializationSchema(catalogTables, false); SimpleCollector collector = new SimpleCollector(); deserializationSchema.deserialize(null, collector); @@ -72,7 +76,7 @@ public void testSerializationAndSchemaExcludeDeserialization() throws Exception @Test public void testDeserializeNoJson() throws Exception { final DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); + new DebeziumJsonDeserializationSchema(catalogTables, false); final SimpleCollector collector = new SimpleCollector(); String noJsonMsg = "{]"; @@ -90,7 +94,7 @@ public void testDeserializeNoJson() throws Exception { @Test public void testDeserializeEmptyJson() throws Exception { final DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); + new DebeziumJsonDeserializationSchema(catalogTables, false); final SimpleCollector collector = new SimpleCollector(); String emptyMsg = "{}"; SeaTunnelRuntimeException expected = CommonError.jsonOperationError(FORMAT, emptyMsg); @@ -106,7 +110,7 @@ public void testDeserializeEmptyJson() throws Exception { @Test public void testDeserializeNoDataJson() throws Exception { final DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); + new DebeziumJsonDeserializationSchema(catalogTables, false); final SimpleCollector collector = new SimpleCollector(); String noDataMsg = "{\"op\":\"u\"}"; SeaTunnelRuntimeException expected = CommonError.jsonOperationError(FORMAT, noDataMsg); @@ -132,7 +136,7 @@ public void testDeserializeNoDataJson() throws Exception { @Test public void testDeserializeUnknownOperationTypeJson() throws Exception { final DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); + new DebeziumJsonDeserializationSchema(catalogTables, false); final SimpleCollector collector = new SimpleCollector(); String unknownType = "XX"; String unknownOperationMsg = @@ -161,7 +165,7 @@ private void testSerializationDeserialization(String resourceFile, boolean schem throws Exception { List lines = readLines(resourceFile); DebeziumJsonDeserializationSchema deserializationSchema = - new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, true, schemaInclude); + new DebeziumJsonDeserializationSchema(catalogTables, true, schemaInclude); SimpleCollector collector = new SimpleCollector(); @@ -171,32 +175,32 @@ private void testSerializationDeserialization(String resourceFile, boolean schem List expected = Arrays.asList( - "SeaTunnelRow{tableId=, kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=, kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); + "SeaTunnelRow{tableId=.., kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=.., kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); assertEquals(expected, actual); DebeziumJsonSerializationSchema serializationSchema = - new DebeziumJsonSerializationSchema(PHYSICAL_DATA_TYPE); + new DebeziumJsonSerializationSchema(SEATUNNEL_ROW_TYPE); actual = new ArrayList<>(); for (SeaTunnelRow rowData : collector.list) { diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java index 1f44c7cfbcb..04fea16ecad 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java @@ -19,6 +19,8 @@ package org.apache.seatunnel.format.json.ogg; import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -48,16 +50,18 @@ public class OggJsonSerDeSchemaTest { private static final String FORMAT = "Ogg"; - private static final SeaTunnelRowType PHYSICAL_DATA_TYPE = + private static final SeaTunnelRowType SEATUNNEL_ROW_TYPE = new SeaTunnelRowType( new String[] {"id", "name", "description", "weight"}, new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); + private static final CatalogTable catalogTables = + CatalogTableUtil.getCatalogTable("", "", "", "", SEATUNNEL_ROW_TYPE); @Test public void testFilteringTables() throws Exception { List lines = readLines("ogg-data-filter-table.txt"); OggJsonDeserializationSchema deserializationSchema = - new OggJsonDeserializationSchema.Builder(PHYSICAL_DATA_TYPE) + new OggJsonDeserializationSchema.Builder(catalogTables) .setDatabase("^OG.*") .setTable("^TBL.*") .build(); @@ -168,33 +172,33 @@ public void runTest(List lines, OggJsonDeserializationSchema deserializa List expected = Arrays.asList( - "SeaTunnelRow{tableId=, kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=, kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", - "SeaTunnelRow{tableId=, kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", - "SeaTunnelRow{tableId=, kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", - "SeaTunnelRow{tableId=, kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); + "SeaTunnelRow{tableId=.., kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=.., kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", + "SeaTunnelRow{tableId=.., kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=.., kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=.., kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); List actual = collector.list.stream().map(Object::toString).collect(Collectors.toList()); assertEquals(expected, actual); // test Serialization OggJsonSerializationSchema serializationSchema = - new OggJsonSerializationSchema(PHYSICAL_DATA_TYPE); + new OggJsonSerializationSchema(SEATUNNEL_ROW_TYPE); List result = new ArrayList<>(); for (SeaTunnelRow rowData : collector.list) { result.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); @@ -232,7 +236,7 @@ public void runTest(List lines, OggJsonDeserializationSchema deserializa private OggJsonDeserializationSchema createOggJsonDeserializationSchema( String database, String table) { - return OggJsonDeserializationSchema.builder(PHYSICAL_DATA_TYPE) + return OggJsonDeserializationSchema.builder(catalogTables) .setDatabase(database) .setTable(table) .setIgnoreParseErrors(false) diff --git a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java index fd0c18f31e4..dfde5683d68 100644 --- a/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java +++ b/seatunnel-formats/seatunnel-format-text/src/main/java/org/apache/seatunnel/format/text/TextDeserializationSchema.java @@ -18,12 +18,14 @@ package org.apache.seatunnel.format.text; import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.MapType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.common.exception.CommonErrorCode; import org.apache.seatunnel.common.utils.DateTimeUtils; import org.apache.seatunnel.common.utils.DateUtils; import org.apache.seatunnel.common.utils.EncodingUtils; @@ -52,12 +54,14 @@ import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; +import java.util.Optional; public class TextDeserializationSchema implements DeserializationSchema { private final SeaTunnelRowType seaTunnelRowType; private final String[] separators; private final String encoding; private final TextLineSplitor splitor; + private final CatalogTable catalogTable; @SuppressWarnings("MagicNumber") public static final DateTimeFormatter TIME_FORMAT = @@ -72,11 +76,13 @@ private TextDeserializationSchema( @NonNull SeaTunnelRowType seaTunnelRowType, String[] separators, String encoding, - TextLineSplitor splitor) { + TextLineSplitor splitor, + CatalogTable catalogTable) { this.seaTunnelRowType = seaTunnelRowType; this.separators = separators; this.encoding = encoding; this.splitor = splitor; + this.catalogTable = catalogTable; } public static Builder builder() { @@ -85,6 +91,7 @@ public static Builder builder() { public static class Builder { private SeaTunnelRowType seaTunnelRowType; + private CatalogTable catalogTable; private String[] separators = TextFormatConstant.SEPARATOR.clone(); private DateUtils.Formatter dateFormatter = DateUtils.Formatter.YYYY_MM_DD; private DateTimeUtils.Formatter dateTimeFormatter = @@ -95,6 +102,11 @@ public static class Builder { private Builder() {} + public Builder setCatalogTable(CatalogTable catalogTable) { + this.catalogTable = catalogTable; + return this; + } + public Builder seaTunnelRowType(SeaTunnelRowType seaTunnelRowType) { this.seaTunnelRowType = seaTunnelRowType; return this; @@ -137,12 +149,15 @@ public Builder textLineSplitor(TextLineSplitor splitor) { public TextDeserializationSchema build() { return new TextDeserializationSchema( - seaTunnelRowType, separators, encoding, textLineSplitor); + seaTunnelRowType, separators, encoding, textLineSplitor, catalogTable); } } @Override public SeaTunnelRow deserialize(byte[] message) throws IOException { + if (message == null || message.length == 0) { + return null; + } String content = new String(message, EncodingUtils.tryParseCharset(encoding)); Map splitsMap = splitLineBySeaTunnelRowType(content, seaTunnelRowType, 0); Object[] objects = new Object[seaTunnelRowType.getTotalFields()]; @@ -154,7 +169,13 @@ public SeaTunnelRow deserialize(byte[] message) throws IOException { 0, seaTunnelRowType.getFieldNames()[i]); } - return new SeaTunnelRow(objects); + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(objects); + Optional tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath); + if (tablePath.isPresent()) { + seaTunnelRow.setTableId(tablePath.toString()); + } + return seaTunnelRow; } @Override @@ -219,7 +240,7 @@ private Object convert( return objectArrayList.toArray(new LocalDateTime[0]); default: throw new SeaTunnelTextFormatException( - CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + CommonErrorCode.UNSUPPORTED_DATA_TYPE, String.format( "SeaTunnel array not support this data type [%s]", elementType.getSqlType())); @@ -300,7 +321,7 @@ private Object convert( return new SeaTunnelRow(objects); default: throw new SeaTunnelTextFormatException( - CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + CommonErrorCode.UNSUPPORTED_DATA_TYPE, String.format( "SeaTunnel not support this data type [%s]", fieldType.getSqlType()));