From 6cb4abf0af1fe5f065539919ce437026b30854e6 Mon Sep 17 00:00:00 2001 From: 00193254 Date: Tue, 19 Aug 2025 17:00:38 +0800 Subject: [PATCH 01/37] init --- docs/en/connector-v2/source/MySQL-CDC.md | 3 +- docs/zh/connector-v2/source/MySQL-CDC.md | 3 +- .../cdc/base/config/StartupConfig.java | 2 +- .../source/MySqlIncrementalSourceFactory.java | 4 + .../cdc/mysql/source/MySqlSourceOptions.java | 5 +- .../cdc/mysql/source/offset/BinlogOffset.java | 6 + .../source/offset/BinlogOffsetFactory.java | 3 +- .../fetch/MySqlSourceFetchTaskContext.java | 28 ++- .../cdc/mysql/utils/MySqlConnectionUtils.java | 90 ++++++++ .../MysqlCDCSpecificStartingOffsetIT.java | 206 ++++++++++++++++++ 10 files changed, 339 insertions(+), 11 deletions(-) diff --git a/docs/en/connector-v2/source/MySQL-CDC.md b/docs/en/connector-v2/source/MySQL-CDC.md index c5337cc8109..fef49b83ad4 100644 --- a/docs/en/connector-v2/source/MySQL-CDC.md +++ b/docs/en/connector-v2/source/MySQL-CDC.md @@ -193,9 +193,10 @@ When an initial consistent snapshot is made for large databases, your establishe | table-names | List | Yes | - | Table name of the database to monitor. The table name needs to include the database name, for example: `database_name.table_name` | | table-pattern | String | Yes | - | The table names RegEx of the database to capture. The table name needs to include the database name, for example: `database.*\\.table_.*` | | table-names-config | List | No | - | Table config list. for example: [{"table": "db1.schema1.table1","primaryKeys": ["key1"],"snapshotSplitColumn": "key2"}] | -| startup.mode | Enum | No | INITIAL | Optional startup mode for MySQL CDC consumer, valid enumerations are `initial`, `earliest`, `latest` and `specific`.
`initial`: Synchronize historical data at startup, and then synchronize incremental data.
`earliest`: Startup from the earliest offset possible.
`latest`: Startup from the latest offset.
`specific`: Startup from user-supplied specific offsets. | +| startup.mode | Enum | No | INITIAL | Optional startup mode for MySQL CDC consumer, valid enumerations are `initial`, `earliest`, `latest` , `specific` and `timestamp`.
`initial`: Synchronize historical data at startup, and then synchronize incremental data.
`earliest`: Startup from the earliest offset possible.
`latest`: Startup from the latest offset.
`specific`: Startup from user-supplied specific offsets.
`timestamp`: Startup from user-supplied timestamp. | | startup.specific-offset.file | String | No | - | Start from the specified binlog file name. **Note, This option is required when the `startup.mode` option used `specific`.** | | startup.specific-offset.pos | Long | No | - | Start from the specified binlog file position. **Note, This option is required when the `startup.mode` option used `specific`.** | +| startup.timestamp | Long | No | - | Start from the specified timestamp. **Note, This option is required when the `startup.mode` option used `timestamp`.** | | stop.mode | Enum | No | NEVER | Optional stop mode for MySQL CDC consumer, valid enumerations are `never`, `latest` or `specific`.
`never`: Real-time job don't stop the source.
`latest`: Stop from the latest offset.
`specific`: Stop from user-supplied specific offset. | | stop.specific-offset.file | String | No | - | Stop from the specified binlog file name. **Note, This option is required when the `stop.mode` option used `specific`.** | | stop.specific-offset.pos | Long | No | - | Stop from the specified binlog file position. **Note, This option is required when the `stop.mode` option used `specific`.** | diff --git a/docs/zh/connector-v2/source/MySQL-CDC.md b/docs/zh/connector-v2/source/MySQL-CDC.md index 7d60a95dae9..d1911f84d3c 100644 --- a/docs/zh/connector-v2/source/MySQL-CDC.md +++ b/docs/zh/connector-v2/source/MySQL-CDC.md @@ -192,9 +192,10 @@ show variables where variable_name in ('log_bin', 'binlog_format', 'binlog_row_i | table-names | List | 是 | - | 要监控的表名. 表名需要包括库名, 例如: `database_name.table_name` | | table-pattern | String | 是 | - | 要捕获的表名称的正则表达式. 表名需要包括库名, 例如: `database.*\\.table_.*` | | table-names-config | List | 否 | - | 表配置的列表集合. 例如: [{"table": "db1.schema1.table1","primaryKeys": ["key1"],"snapshotSplitColumn": "key2"}] | -| startup.mode | Enum | 否 | INITIAL | MySQL CDC 消费者的可选启动模式, 有效枚举值为 `initial`, `earliest`, `latest` 和 `specific`.
`initial`: 启动时同步历史数据, 然后同步增量数据.
`earliest`: 从尽可能最早的偏移量开始启动.
`latest`: 从最近的偏移量启动.
`specific`: 从用户提供的特定偏移量开始启动. | +| startup.mode | Enum | 否 | INITIAL | MySQL CDC 消费者的可选启动模式, 有效枚举值为 `initial`, `earliest`, `latest` , `specific` 和 `timestamp`.
`initial`: 启动时同步历史数据, 然后同步增量数据.
`earliest`: 从尽可能最早的偏移量开始启动.
`latest`: 从最近的偏移量启动.
`timestamp`: 从用户提供的特定时间戳开始启动. | | startup.specific-offset.file | String | 否 | - | 从指定的binlog日志文件名开始. **注意, 当使用 `startup.mode` 选项为 `specific` 时,此选项为必填项.** | | startup.specific-offset.pos | Long | 否 | - | 从指定的binlog日志文件位置开始. **注意, 当使用 `startup.mode` 选项为 `specific` 时,此选项为必填项.** | +| startup.timestamp | Long | No | - | 从指定的binlog时间戳文件位置开始. **注意, 当使用 `startup.mode` 选项为 `timestamp` 时,此选项为必填项.** | | stop.mode | Enum | 否 | NEVER | MySQL CDC 消费者的可选停止模式, 有效枚举值为 `never`, `latest` 和 `specific`.
`never`: 实时任务一直运行不停止.
`latest`: 从最新的偏移量处停止.
`specific`: 从用户提供的特定偏移量处停止. | | stop.specific-offset.file | String | 否 | - | 从指定的binlog日志文件名停止. **注意, 当使用 `stop.mode` 选项为 `specific` 时,此选项为必填项.** | | stop.specific-offset.pos | Long | 否 | - | 从指定的binlog日志文件位置停止. **注意, 当使用 `stop.mode` 选项为 `specific` 时,此选项为必填项.** | diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/config/StartupConfig.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/config/StartupConfig.java index 52e7ec23eae..40dc755540d 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/config/StartupConfig.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/config/StartupConfig.java @@ -34,7 +34,7 @@ public final class StartupConfig implements Serializable { @Getter private final StartupMode startupMode; private final String specificOffsetFile; private final Long specificOffsetPos; - private final Long timestamp; + @Getter private final Long timestamp; public Offset getStartupOffset(OffsetFactory offsetFactory) { switch (startupMode) { diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlIncrementalSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlIncrementalSourceFactory.java index a37b2bdc0c1..c3eb321c8fc 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlIncrementalSourceFactory.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlIncrementalSourceFactory.java @@ -90,6 +90,10 @@ public OptionRule optionRule() { StopMode.SPECIFIC, SourceOptions.STOP_SPECIFIC_OFFSET_FILE, SourceOptions.STOP_SPECIFIC_OFFSET_POS) + .conditional( + MySqlSourceOptions.STARTUP_MODE, + StartupMode.TIMESTAMP, + SourceOptions.STARTUP_TIMESTAMP) .build(); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlSourceOptions.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlSourceOptions.java index bc59fd0f5c1..34b73e8fb47 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlSourceOptions.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlSourceOptions.java @@ -35,11 +35,12 @@ public class MySqlSourceOptions { StartupMode.INITIAL, StartupMode.EARLIEST, StartupMode.LATEST, - StartupMode.SPECIFIC)) + StartupMode.SPECIFIC, + StartupMode.TIMESTAMP)) .defaultValue(StartupMode.INITIAL) .withDescription( "Optional startup mode for CDC source, valid enumerations are " - + "\"initial\", \"earliest\", \"latest\" or \"specific\""); + + "\"initial\", \"earliest\", \"latest\" , \"specific\" or \"timestamp\""); public static final SingleChoiceOption STOP_MODE = (SingleChoiceOption) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/offset/BinlogOffset.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/offset/BinlogOffset.java index f94401ba345..6698bae05ff 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/offset/BinlogOffset.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/offset/BinlogOffset.java @@ -81,6 +81,12 @@ public BinlogOffset( this.offset = offsetMap; } + public BinlogOffset(long timestamp) { + Map offsetMap = new HashMap<>(); + offsetMap.put(TIMESTAMP_KEY, String.valueOf(timestamp)); + this.offset = offsetMap; + } + public String getFilename() { return offset.get(BINLOG_FILENAME_OFFSET_KEY); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/offset/BinlogOffsetFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/offset/BinlogOffsetFactory.java index 814c4d7c240..e1d6916ac58 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/offset/BinlogOffsetFactory.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/offset/BinlogOffsetFactory.java @@ -76,6 +76,7 @@ public Offset specific(String filename, Long position) { @Override public Offset timestamp(long timestamp) { - throw new UnsupportedOperationException("not supported create new Offset by timestamp."); + // mysql binlog timestamp is second, so we need to divide 1000 + return new BinlogOffset(timestamp / 1000); } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/reader/fetch/MySqlSourceFetchTaskContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/reader/fetch/MySqlSourceFetchTaskContext.java index fcf25d50ea4..04b9ed65ebb 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/reader/fetch/MySqlSourceFetchTaskContext.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/reader/fetch/MySqlSourceFetchTaskContext.java @@ -19,8 +19,10 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.utils.ReflectionUtils; +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.option.StartupMode; import org.apache.seatunnel.connectors.cdc.base.relational.JdbcSourceEventDispatcher; import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; import org.apache.seatunnel.connectors.cdc.base.source.reader.external.JdbcSourceFetchTaskContext; @@ -52,6 +54,7 @@ import io.debezium.connector.mysql.MySqlTaskContext; import io.debezium.connector.mysql.MySqlTopicSelector; import io.debezium.data.Envelope; +import io.debezium.jdbc.JdbcConnection; import io.debezium.pipeline.DataChangeEvent; import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.metrics.SnapshotChangeEventSourceMetrics; @@ -77,6 +80,7 @@ import static org.apache.seatunnel.connectors.seatunnel.cdc.mysql.source.offset.BinlogOffset.BINLOG_FILENAME_OFFSET_KEY; import static org.apache.seatunnel.connectors.seatunnel.cdc.mysql.utils.MySqlConnectionUtils.createBinaryClient; import static org.apache.seatunnel.connectors.seatunnel.cdc.mysql.utils.MySqlConnectionUtils.createMySqlConnection; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mysql.utils.MySqlConnectionUtils.findBinlogOffsetBytimestamp; /** The context for fetch task that fetching data of snapshot split from MySQL data source. */ @Slf4j @@ -265,11 +269,8 @@ public Offset getStreamOffset(SourceRecord sourceRecord) { /** Loads the connector's persistent offset (if present) via the given loader. */ private MySqlOffsetContext loadStartingOffsetState( MySqlOffsetContext.Loader loader, SourceSplitBase mySqlSplit) { - Offset offset = - mySqlSplit.isSnapshotSplit() - ? BinlogOffset.INITIAL_OFFSET - : mySqlSplit.asIncrementalSplit().getStartupOffset(); - + Offset offset = getInitOffset(mySqlSplit); + LOG.info("mysql cdc start at {}", offset); MySqlOffsetContext mySqlOffsetContext = loader.load(offset.getOffset()); if (!isBinlogAvailable(mySqlOffsetContext)) { @@ -282,6 +283,23 @@ private MySqlOffsetContext loadStartingOffsetState( return mySqlOffsetContext; } + private Offset getInitOffset(SourceSplitBase mySqlSplit) { + StartupMode startupMode = getSourceConfig().getStartupConfig().getStartupMode(); + if (startupMode.equals(StartupMode.TIMESTAMP)) { + long timestamp = getSourceConfig().getStartupConfig().getTimestamp(); + try (JdbcConnection jdbcConnection = + getDataSourceDialect().openJdbcConnection(getSourceConfig())) { + return findBinlogOffsetBytimestamp(jdbcConnection, binaryLogClient, timestamp); + } catch (Exception e) { + throw new SeaTunnelException(e); + } + } else { + return mySqlSplit.isSnapshotSplit() + ? BinlogOffset.INITIAL_OFFSET + : mySqlSplit.asIncrementalSplit().getStartupOffset(); + } + } + private boolean isBinlogAvailable(MySqlOffsetContext offset) { String gtidStr = offset.gtidSet(); if (gtidStr != null) { diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java index d38553677c7..0d44279b259 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java @@ -22,6 +22,9 @@ import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.source.offset.BinlogOffset; import com.github.shyiko.mysql.binlog.BinaryLogClient; +import com.github.shyiko.mysql.binlog.event.EventData; +import com.github.shyiko.mysql.binlog.event.EventHeaderV4; +import com.github.shyiko.mysql.binlog.event.RotateEventData; import io.debezium.config.Configuration; import io.debezium.connector.mysql.MySqlConnection; import io.debezium.connector.mysql.MySqlConnectorConfig; @@ -36,9 +39,13 @@ import io.debezium.schema.TopicSelector; import io.debezium.util.SchemaNameAdjuster; +import java.io.IOException; import java.sql.SQLException; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.concurrent.ArrayBlockingQueue; /** MySQL connection Utilities. */ public class MySqlConnectionUtils { @@ -185,4 +192,87 @@ private static Map querySystemVariables( return variables; } + + public static BinlogOffset findBinlogOffsetBytimestamp( + JdbcConnection jdbc, BinaryLogClient client, long timestamp) { + final String showBinaryLogStmt = "SHOW BINARY LOGS"; + List binlogFiles = new ArrayList<>(); + JdbcConnection.ResultSetConsumer rsc = + rs -> { + while (rs.next()) { + String fileName = rs.getString(1); + long fileSize = rs.getLong(2); + if (fileSize > 0) { + binlogFiles.add(fileName); + } + } + }; + try { + jdbc.query(showBinaryLogStmt, rsc); + if (binlogFiles.isEmpty()) { + return BinlogOffset.INITIAL_OFFSET; + } + String binlogName = searchBinlogName(client, timestamp, binlogFiles); + return new BinlogOffset(binlogName, 0); + } catch (Exception e) { + throw new SeaTunnelException(e); + } + } + + private static String searchBinlogName( + BinaryLogClient client, long targetMs, List binlogFiles) + throws IOException, InterruptedException { + int startIdx = 0; + int endIdx = binlogFiles.size() - 1; + + while (startIdx <= endIdx) { + int mid = startIdx + (endIdx - startIdx) / 2; + long midTs = getBinlogTimestamp(client, binlogFiles.get(mid)); + if (midTs < targetMs) { + startIdx = mid + 1; + } else if (targetMs < midTs) { + endIdx = mid - 1; + } else { + return binlogFiles.get(mid); + } + } + + return endIdx < 0 ? binlogFiles.get(0) : binlogFiles.get(endIdx); + } + + public static long getBinlogTimestamp(BinaryLogClient client, String binlogFile) + throws IOException, InterruptedException { + + ArrayBlockingQueue binlogTimestamps = new ArrayBlockingQueue<>(1); + BinaryLogClient.EventListener eventListener = + event -> { + EventData data = event.getData(); + if (data instanceof RotateEventData) { + // We skip RotateEventData because it does not contain the timestamp we are + // interested in. + return; + } + + EventHeaderV4 header = event.getHeader(); + long timestamp = header.getTimestamp(); + if (timestamp > 0) { + binlogTimestamps.offer(timestamp); + try { + client.disconnect(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }; + + try { + client.registerEventListener(eventListener); + client.setBinlogFilename(binlogFile); + client.setBinlogPosition(0); + client.connect(); + } finally { + client.unregisterEventListener(eventListener); + } + return binlogTimestamps.take(); + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java index 5321e499b26..36f858a73c7 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.cdc.mysql; +import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfigFactory; import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.config.MySqlSourceConfigFactory; import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.source.MySqlDialect; @@ -43,6 +44,7 @@ import org.testcontainers.lifecycle.Startables; import org.testcontainers.utility.DockerLoggerFactory; +import com.github.shyiko.mysql.binlog.BinaryLogClient; import io.debezium.jdbc.JdbcConnection; import lombok.extern.slf4j.Slf4j; @@ -401,6 +403,171 @@ public void testMysqlCdcSpecificOffset(TestContainer container) throws Exception }); } + @TestTemplate + public void testMysqlCdcTimestampOffset(TestContainer container) throws Exception { + log.info("begin testMysqlCdcTimestampOffset"); + // clean data + flushLogs(); + clearTable(MYSQL_DATABASE, SOURCE_TABLE_1); + clearTable(MYSQL_DATABASE, SINK_TABLE); + + // write error data + executeSql( + String.format( + "INSERT INTO %s.%s ( id, f_binary, f_blob, f_long_varbinary, f_longblob, f_tinyblob, f_varbinary, f_smallint,\n" + + " f_smallint_unsigned, f_mediumint, f_mediumint_unsigned, f_int, f_int_unsigned, f_integer,\n" + + " f_integer_unsigned, f_bigint, f_bigint_unsigned, f_numeric, f_decimal, f_float, f_double,\n" + + " f_double_precision, f_longtext, f_mediumtext, f_text, f_tinytext, f_varchar, f_date, f_datetime,\n" + + " f_timestamp, f_bit1, f_bit64, f_char, f_enum, f_mediumblob, f_long_varchar, f_real, f_time,\n" + + " f_tinyint, f_tinyint_unsigned, f_json, f_year )\n" + + "VALUES ( 10, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000,\n" + + " 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL,\n" + + " 0x74696E79626C6F62, 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321,\n" + + " 123456789, 987654321, 123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field',\n" + + " 'This is a text field', 'This is a tiny text field', '测试字段4', '2022-04-27', '2022-04-27 14:30:00',\n" + + " '2023-04-27 11:08:40', 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2',\n" + + " 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field',\n" + + " 12.345, '14:30:00', -128, 255, '{ \"key\": \"value4\" }', 2022 )", + MYSQL_DATABASE, SOURCE_TABLE_1)); + executeSql( + String.format( + "INSERT INTO %s.%s ( id, f_binary, f_blob, f_long_varbinary, f_longblob, f_tinyblob, f_varbinary, f_smallint,\n" + + " f_smallint_unsigned, f_mediumint, f_mediumint_unsigned, f_int, f_int_unsigned, f_integer,\n" + + " f_integer_unsigned, f_bigint, f_bigint_unsigned, f_numeric, f_decimal, f_float, f_double,\n" + + " f_double_precision, f_longtext, f_mediumtext, f_text, f_tinytext, f_varchar, f_date, f_datetime,\n" + + " f_timestamp, f_bit1, f_bit64, f_char, f_enum, f_mediumblob, f_long_varchar, f_real, f_time,\n" + + " f_tinyint, f_tinyint_unsigned, f_json, f_year )\n" + + "VALUES ( 11, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000,\n" + + " 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL, 0x74696E79626C6F62,\n" + + " 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321, 123456789, 987654321,\n" + + " 123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field', 'This is a text field',\n" + + " 'This is a tiny text field', '测试字段5', '2022-04-27', '2022-04-27 14:30:00', '2023-04-27 11:08:40',\n" + + " 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2',\n" + + " 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field',\n" + + " 112.345, '14:30:00', -128, 22, '{ \"key\": \"value5\" }', 2013 )", + MYSQL_DATABASE, SOURCE_TABLE_1)); + // mysql binlog timestamp is second, wait for 3 seconds to make sure the timestamp is + // different + Thread.sleep(3000); + flushLogs(); + + // get latest binlog timestamp + String[] variables = { + "timestamp=" + getCurrentBinlogTimestamp(), + }; + log.info("offset start with timestamp :{}", variables[0]); + + // Insert data + executeSql( + String.format( + "INSERT INTO %s.%s ( id, f_binary, f_blob, f_long_varbinary, f_longblob, f_tinyblob, f_varbinary, f_smallint,\n" + + " f_smallint_unsigned, f_mediumint, f_mediumint_unsigned, f_int, f_int_unsigned, f_integer,\n" + + " f_integer_unsigned, f_bigint, f_bigint_unsigned, f_numeric, f_decimal, f_float, f_double,\n" + + " f_double_precision, f_longtext, f_mediumtext, f_text, f_tinytext, f_varchar, f_date, f_datetime,\n" + + " f_timestamp, f_bit1, f_bit64, f_char, f_enum, f_mediumblob, f_long_varchar, f_real, f_time,\n" + + " f_tinyint, f_tinyint_unsigned, f_json, f_year )\n" + + "VALUES ( 14, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000,\n" + + " 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL,\n" + + " 0x74696E79626C6F62, 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321,\n" + + " 123456789, 987654321, 123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field',\n" + + " 'This is a text field', 'This is a tiny text field', '测试字段4', '2022-04-27', '2022-04-27 14:30:00',\n" + + " '2023-04-27 11:08:40', 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2',\n" + + " 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field',\n" + + " 12.345, '14:30:00', -128, 255, '{ \"key\": \"value4\" }', 2022 )", + MYSQL_DATABASE, SOURCE_TABLE_1)); + executeSql( + String.format( + "INSERT INTO %s.%s ( id, f_binary, f_blob, f_long_varbinary, f_longblob, f_tinyblob, f_varbinary, f_smallint,\n" + + " f_smallint_unsigned, f_mediumint, f_mediumint_unsigned, f_int, f_int_unsigned, f_integer,\n" + + " f_integer_unsigned, f_bigint, f_bigint_unsigned, f_numeric, f_decimal, f_float, f_double,\n" + + " f_double_precision, f_longtext, f_mediumtext, f_text, f_tinytext, f_varchar, f_date, f_datetime,\n" + + " f_timestamp, f_bit1, f_bit64, f_char, f_enum, f_mediumblob, f_long_varchar, f_real, f_time,\n" + + " f_tinyint, f_tinyint_unsigned, f_json, f_year )\n" + + "VALUES ( 15, 0x61626374000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000,\n" + + " 0x68656C6C6F, 0x18000000789C0BC9C82C5600A244859CFCBC7485B2C4A2A4CCBCC4A24A00697308D4, NULL, 0x74696E79626C6F62,\n" + + " 0x48656C6C6F20776F726C64, 12345, 54321, 123456, 654321, 1234567, 7654321, 1234567, 7654321, 123456789, 987654321,\n" + + " 123, 789, 12.34, 56.78, 90.12, 'This is a long text field', 'This is a medium text field', 'This is a text field',\n" + + " 'This is a tiny text field', '测试字段5', '2022-04-27', '2022-04-27 14:30:00', '2023-04-27 11:08:40',\n" + + " 1, b'0101010101010101010101010101010101010101010101010101010101010101', 'C', 'enum2',\n" + + " 0x1B000000789C0BC9C82C5600A24485DCD494CCD25C85A49CFC2485B4CCD49C140083FF099A, 'This is a long varchar field',\n" + + " 112.345, '14:30:00', -128, 22, '{ \"key\": \"value5\" }', 2013 )", + MYSQL_DATABASE, SOURCE_TABLE_1)); + + String jobId = String.valueOf(JobIdGenerator.newJobId()); + String jobConfigFile = "/mysqlcdc_timestamp_offset.conf"; + String source_sql_where_id_template = + "select id, cast(f_binary as char) as f_binary, cast(f_blob as char) as f_blob, cast(f_long_varbinary as char) as f_long_varbinary," + + " cast(f_longblob as char) as f_longblob, cast(f_tinyblob as char) as f_tinyblob, cast(f_varbinary as char) as f_varbinary," + + " f_smallint, f_smallint_unsigned, f_mediumint, f_mediumint_unsigned, f_int, f_int_unsigned, f_integer, f_integer_unsigned," + + " f_bigint, f_bigint_unsigned, f_numeric, f_decimal, f_float, f_double, f_double_precision, f_longtext, f_mediumtext," + + " f_text, f_tinytext, f_varchar, f_date, f_datetime, f_timestamp, f_bit1, cast(f_bit64 as char) as f_bit64, f_char," + + " f_enum, cast(f_mediumblob as char) as f_mediumblob, f_long_varchar, f_real, f_time, f_tinyint, f_tinyint_unsigned," + + " f_json, f_year from %s.%s where id in (%s)"; + + CompletableFuture.supplyAsync( + () -> { + try { + container.executeJob(jobConfigFile, jobId, variables); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); + + // validate results + await().atMost(90000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertIterableEquals( + query( + String.format( + source_sql_where_id_template, + MYSQL_DATABASE, + SOURCE_TABLE_1, + "14,15")), + query(getSinkQuerySQL(MYSQL_DATABASE, SINK_TABLE))); + }); + + // Take a savepoint + Assertions.assertEquals(0, container.savepointJob(jobId).getExitCode()); + // Make some changes after the savepoint + executeSql( + String.format( + "UPDATE %s.%s SET f_year = '2025' WHERE id = 15", + MYSQL_DATABASE, SOURCE_TABLE_1)); + + CompletableFuture.supplyAsync( + () -> { + try { + container.restoreJob(jobConfigFile, jobId, variables); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); + + // Make some changes after the restore + executeSql( + String.format( + "UPDATE %s.%s SET f_tinyint_unsigned = '77' WHERE id = 15", + MYSQL_DATABASE, SOURCE_TABLE_1)); + + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertIterableEquals( + query( + String.format( + source_sql_where_id_template, + MYSQL_DATABASE, + SOURCE_TABLE_1, + "14,15")), + query(getSinkQuerySQL(MYSQL_DATABASE, SINK_TABLE))); + }); + } + @Override @AfterAll public void tearDown() { @@ -444,4 +611,43 @@ private void purgeBinaryLogs() { executeSql( String.format("PURGE BINARY LOGS TO '%s'", getCurrentBinlogOffset().getFilename())); } + + private long getCurrentBinlogTimestamp() { + JdbcSourceConfigFactory configFactory = + new MySqlSourceConfigFactory() + .hostname(MYSQL_CONTAINER.getHost()) + .port(MYSQL_CONTAINER.getDatabasePort()) + .username(MYSQL_CONTAINER.getUsername()) + .password(MYSQL_CONTAINER.getPassword()) + .databaseList(MYSQL_CONTAINER.getDatabaseName()); + JdbcSourceConfig jdbcSourceConfig = configFactory.create(0); + MySqlDialect mySqlDialect = + new MySqlDialect((MySqlSourceConfigFactory) configFactory, Collections.emptyList()); + + final String showBinaryLogStmt = "SHOW BINARY LOGS"; + List binlogFiles = new ArrayList<>(); + JdbcConnection.ResultSetConsumer rsc = + rs -> { + while (rs.next()) { + String fileName = rs.getString(1); + long fileSize = rs.getLong(2); + if (fileSize > 0) { + binlogFiles.add(fileName); + } + } + }; + try (JdbcConnection jdbc = mySqlDialect.openJdbcConnection(jdbcSourceConfig)) { + jdbc.query(showBinaryLogStmt, rsc); + if (binlogFiles.isEmpty()) { + return System.currentTimeMillis(); + } + log.info("SHOW BINARY LOGS result :{}", binlogFiles); + BinaryLogClient client = + MySqlConnectionUtils.createBinaryClient(jdbcSourceConfig.getDbzConfiguration()); + return MySqlConnectionUtils.getBinlogTimestamp( + client, binlogFiles.get(binlogFiles.size() - 1)); + } catch (Exception e) { + throw new RuntimeException(e); + } + } } From e7f063274f8d43a6ce0ed0dbc75366fb27c8a865 Mon Sep 17 00:00:00 2001 From: 00193254 Date: Wed, 20 Aug 2025 10:54:31 +0800 Subject: [PATCH 02/37] init --- .../fetch/MySqlSourceFetchTaskContext.java | 9 +- .../fetch/binlog/MySqlBinlogFetchTask.java | 118 ++++++++++++++++-- .../MysqlCDCSpecificStartingOffsetIT.java | 68 +++++----- 3 files changed, 153 insertions(+), 42 deletions(-) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/reader/fetch/MySqlSourceFetchTaskContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/reader/fetch/MySqlSourceFetchTaskContext.java index 04b9ed65ebb..0316dfd954f 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/reader/fetch/MySqlSourceFetchTaskContext.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/reader/fetch/MySqlSourceFetchTaskContext.java @@ -269,7 +269,10 @@ public Offset getStreamOffset(SourceRecord sourceRecord) { /** Loads the connector's persistent offset (if present) via the given loader. */ private MySqlOffsetContext loadStartingOffsetState( MySqlOffsetContext.Loader loader, SourceSplitBase mySqlSplit) { - Offset offset = getInitOffset(mySqlSplit); + Offset offset = + mySqlSplit.isSnapshotSplit() + ? BinlogOffset.INITIAL_OFFSET + : getInitOffset(mySqlSplit); LOG.info("mysql cdc start at {}", offset); MySqlOffsetContext mySqlOffsetContext = loader.load(offset.getOffset()); @@ -294,9 +297,7 @@ private Offset getInitOffset(SourceSplitBase mySqlSplit) { throw new SeaTunnelException(e); } } else { - return mySqlSplit.isSnapshotSplit() - ? BinlogOffset.INITIAL_OFFSET - : mySqlSplit.asIncrementalSplit().getStartupOffset(); + return mySqlSplit.asIncrementalSplit().getStartupOffset(); } } 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 d31c4833b8c..e3b8ea7ac0e 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 @@ -17,6 +17,8 @@ package org.apache.seatunnel.connectors.seatunnel.cdc.mysql.source.reader.fetch.binlog; +import org.apache.seatunnel.connectors.cdc.base.config.StartupConfig; +import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; import org.apache.seatunnel.connectors.cdc.base.relational.JdbcSourceEventDispatcher; import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; import org.apache.seatunnel.connectors.cdc.base.source.split.IncrementalSplit; @@ -31,6 +33,8 @@ import com.github.shyiko.mysql.binlog.BinaryLogClient; import com.github.shyiko.mysql.binlog.event.Event; +import com.github.shyiko.mysql.binlog.event.EventHeader; +import com.github.shyiko.mysql.binlog.event.EventHeaderV4; import io.debezium.DebeziumException; import io.debezium.connector.mysql.MySqlConnection; import io.debezium.connector.mysql.MySqlConnectorConfig; @@ -63,16 +67,37 @@ public MySqlBinlogFetchTask(IncrementalSplit split) { public void execute(FetchTask.Context context) throws Exception { MySqlSourceFetchTaskContext sourceFetchContext = (MySqlSourceFetchTaskContext) context; taskRunning = true; + MySqlStreamingChangeEventSource mySqlStreamingChangeEventSource; - MySqlStreamingChangeEventSource mySqlStreamingChangeEventSource = - new MySqlStreamingChangeEventSource( - sourceFetchContext.getDbzConnectorConfig(), - sourceFetchContext.getConnection(), - sourceFetchContext.getDispatcher(), - sourceFetchContext.getErrorHandler(), - Clock.SYSTEM, - sourceFetchContext.getTaskContext(), - sourceFetchContext.getStreamingChangeEventSourceMetrics()); + StartupConfig startupConfig = sourceFetchContext.getSourceConfig().getStartupConfig(); + + StartupMode startupMode = startupConfig.getStartupMode(); + if (startupMode.equals(StartupMode.TIMESTAMP)) { + log.info( + "Starting MySQL binlog reader,with timestamp filter {}", + startupConfig.getTimestamp()); + + mySqlStreamingChangeEventSource = + new TimestampFilterMySqlStreamingChangeEventSource( + sourceFetchContext.getDbzConnectorConfig(), + sourceFetchContext.getConnection(), + sourceFetchContext.getDispatcher(), + sourceFetchContext.getErrorHandler(), + Clock.SYSTEM, + sourceFetchContext.getTaskContext(), + sourceFetchContext.getStreamingChangeEventSourceMetrics(), + startupConfig.getTimestamp()); + } else { + mySqlStreamingChangeEventSource = + new MySqlStreamingChangeEventSource( + sourceFetchContext.getDbzConnectorConfig(), + sourceFetchContext.getConnection(), + sourceFetchContext.getDispatcher(), + sourceFetchContext.getErrorHandler(), + Clock.SYSTEM, + sourceFetchContext.getTaskContext(), + sourceFetchContext.getStreamingChangeEventSourceMetrics()); + } BinlogSplitChangeEventSourceContext changeEventSourceContext = new BinlogSplitChangeEventSourceContext(); @@ -204,6 +229,81 @@ public static BinlogOffset getBinlogPosition(Map offset) { } } + private class TimestampFilterMySqlStreamingChangeEventSource + extends MySqlStreamingChangeEventSource { + + private final Long targetTimestamp; + private long logTimestamp; + private boolean loggedWaitingMessage; + private final long LOG_INTERVAL_MS = 10000; + + public TimestampFilterMySqlStreamingChangeEventSource( + MySqlConnectorConfig connectorConfig, + MySqlConnection connection, + JdbcSourceEventDispatcher dispatcher, + ErrorHandler errorHandler, + Clock clock, + MySqlTaskContext taskContext, + MySqlStreamingChangeEventSourceMetrics metrics, + Long targetTimestamp) { + super( + connectorConfig, + connection, + dispatcher, + errorHandler, + clock, + taskContext, + metrics); + this.targetTimestamp = targetTimestamp; + } + + @Override + protected void handleEvent( + MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) { + if (event == null) { + super.handleEvent(partition, offsetContext, event); + return; + } + + long eventTs = event.getHeader().getTimestamp(); + if (eventTs == 0 || targetTimestamp == null || targetTimestamp == 0) { + super.handleEvent(partition, offsetContext, event); + return; + } + boolean shouldSkip = eventTs < targetTimestamp; + if (shouldSkip) { + if (!loggedWaitingMessage) { + log.info( + "skip binlog, currentTime:{}, filterTime:{}", eventTs, targetTimestamp); + loggedWaitingMessage = true; + logTimestamp = eventTs; + } + if (eventTs - logTimestamp >= LOG_INTERVAL_MS) { + loggedWaitingMessage = false; + } + updateOffsetPosition(offsetContext, event.getHeader()); + return; + } + + super.handleEvent(partition, offsetContext, event); + } + + private void updateOffsetPosition( + MySqlOffsetContext offsetContext, EventHeader eventHeader) { + try { + if (eventHeader instanceof EventHeaderV4) { + EventHeaderV4 headerV4 = (EventHeaderV4) eventHeader; + offsetContext.setEventPosition( + headerV4.getPosition(), headerV4.getEventLength()); + } + offsetContext.setBinlogServerId(eventHeader.getServerId()); + offsetContext.completeEvent(); + } catch (Exception e) { + log.warn("Failed to update offset for skipped event: {}", e.getMessage()); + } + } + } + private class BinlogSplitChangeEventSourceContext implements ChangeEventSource.ChangeEventSourceContext { @Override diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java index 36f858a73c7..2d560979a28 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java @@ -45,6 +45,9 @@ import org.testcontainers.utility.DockerLoggerFactory; import com.github.shyiko.mysql.binlog.BinaryLogClient; +import com.github.shyiko.mysql.binlog.event.EventData; +import com.github.shyiko.mysql.binlog.event.EventHeaderV4; +import com.github.shyiko.mysql.binlog.event.RotateEventData; import io.debezium.jdbc.JdbcConnection; import lombok.extern.slf4j.Slf4j; @@ -57,6 +60,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.stream.Stream; @@ -66,7 +70,7 @@ @Slf4j @DisabledOnContainer( value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, + type = {EngineType.SPARK, EngineType.SEATUNNEL}, disabledReason = "Currently SPARK and FLINK do not support restore") public class MysqlCDCSpecificStartingOffsetIT extends TestSuiteBase implements TestResource { @@ -406,8 +410,6 @@ public void testMysqlCdcSpecificOffset(TestContainer container) throws Exception @TestTemplate public void testMysqlCdcTimestampOffset(TestContainer container) throws Exception { log.info("begin testMysqlCdcTimestampOffset"); - // clean data - flushLogs(); clearTable(MYSQL_DATABASE, SOURCE_TABLE_1); clearTable(MYSQL_DATABASE, SINK_TABLE); @@ -449,11 +451,10 @@ public void testMysqlCdcTimestampOffset(TestContainer container) throws Exceptio // mysql binlog timestamp is second, wait for 3 seconds to make sure the timestamp is // different Thread.sleep(3000); - flushLogs(); // get latest binlog timestamp String[] variables = { - "timestamp=" + getCurrentBinlogTimestamp(), + "timestamp=" + getCurrentBinlogTimestamp() + 1000L, }; log.info("offset start with timestamp :{}", variables[0]); @@ -612,7 +613,10 @@ private void purgeBinaryLogs() { String.format("PURGE BINARY LOGS TO '%s'", getCurrentBinlogOffset().getFilename())); } - private long getCurrentBinlogTimestamp() { + private long getCurrentBinlogTimestamp() throws IOException, InterruptedException { + + BinlogOffset binlogOffset = getCurrentBinlogOffset(); + JdbcSourceConfigFactory configFactory = new MySqlSourceConfigFactory() .hostname(MYSQL_CONTAINER.getHost()) @@ -621,33 +625,39 @@ private long getCurrentBinlogTimestamp() { .password(MYSQL_CONTAINER.getPassword()) .databaseList(MYSQL_CONTAINER.getDatabaseName()); JdbcSourceConfig jdbcSourceConfig = configFactory.create(0); - MySqlDialect mySqlDialect = - new MySqlDialect((MySqlSourceConfigFactory) configFactory, Collections.emptyList()); + BinaryLogClient client = + MySqlConnectionUtils.createBinaryClient(jdbcSourceConfig.getDbzConfiguration()); + + ArrayBlockingQueue binlogTimestamps = new ArrayBlockingQueue<>(1); + BinaryLogClient.EventListener eventListener = + event -> { + EventData data = event.getData(); + if (data instanceof RotateEventData) { + // We skip RotateEventData because it does not contain the timestamp we are + // interested in. + return; + } - final String showBinaryLogStmt = "SHOW BINARY LOGS"; - List binlogFiles = new ArrayList<>(); - JdbcConnection.ResultSetConsumer rsc = - rs -> { - while (rs.next()) { - String fileName = rs.getString(1); - long fileSize = rs.getLong(2); - if (fileSize > 0) { - binlogFiles.add(fileName); + EventHeaderV4 header = event.getHeader(); + long timestamp = header.getTimestamp(); + if (timestamp > 0) { + binlogTimestamps.offer(timestamp); + try { + client.disconnect(); + } catch (IOException e) { + throw new RuntimeException(e); } } }; - try (JdbcConnection jdbc = mySqlDialect.openJdbcConnection(jdbcSourceConfig)) { - jdbc.query(showBinaryLogStmt, rsc); - if (binlogFiles.isEmpty()) { - return System.currentTimeMillis(); - } - log.info("SHOW BINARY LOGS result :{}", binlogFiles); - BinaryLogClient client = - MySqlConnectionUtils.createBinaryClient(jdbcSourceConfig.getDbzConfiguration()); - return MySqlConnectionUtils.getBinlogTimestamp( - client, binlogFiles.get(binlogFiles.size() - 1)); - } catch (Exception e) { - throw new RuntimeException(e); + + try { + client.registerEventListener(eventListener); + client.setBinlogFilename(binlogOffset.getFilename()); + client.setBinlogPosition(binlogOffset.getPosition()); + client.connect(); + } finally { + client.unregisterEventListener(eventListener); } + return binlogTimestamps.take(); } } From 1e48dc4c1e4ffce9abae04878857be4f9981f181 Mon Sep 17 00:00:00 2001 From: 00193254 Date: Wed, 20 Aug 2025 11:25:35 +0800 Subject: [PATCH 03/37] init --- .../seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java index 2d560979a28..aa82d761531 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java @@ -454,7 +454,7 @@ public void testMysqlCdcTimestampOffset(TestContainer container) throws Exceptio // get latest binlog timestamp String[] variables = { - "timestamp=" + getCurrentBinlogTimestamp() + 1000L, + "timestamp=" + (getCurrentBinlogTimestamp() + 1000L), }; log.info("offset start with timestamp :{}", variables[0]); From 086892110b2dc8b836bc6a2093ecee81cea9ab3e Mon Sep 17 00:00:00 2001 From: 00193254 Date: Wed, 20 Aug 2025 11:33:56 +0800 Subject: [PATCH 04/37] init --- .../seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java index aa82d761531..63df1f38653 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java @@ -70,7 +70,7 @@ @Slf4j @DisabledOnContainer( value = {}, - type = {EngineType.SPARK, EngineType.SEATUNNEL}, + type = {EngineType.SPARK, EngineType.FLINK}, disabledReason = "Currently SPARK and FLINK do not support restore") public class MysqlCDCSpecificStartingOffsetIT extends TestSuiteBase implements TestResource { @@ -454,7 +454,7 @@ public void testMysqlCdcTimestampOffset(TestContainer container) throws Exceptio // get latest binlog timestamp String[] variables = { - "timestamp=" + (getCurrentBinlogTimestamp() + 1000L), + "timestamp=" + (getCurrentBinlogTimestamp() + 2000L), }; log.info("offset start with timestamp :{}", variables[0]); From 25cbac9e08e1e84d6faaded85f8a19e0f04934d5 Mon Sep 17 00:00:00 2001 From: 00193254 Date: Wed, 20 Aug 2025 12:29:02 +0800 Subject: [PATCH 05/37] init --- .../resources/mysqlcdc_timestamp_offset.conf | 50 +++++++++++++++++++ 1 file changed, 50 insertions(+) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_timestamp_offset.conf diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_timestamp_offset.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_timestamp_offset.conf new file mode 100644 index 00000000000..bec1f87d3ef --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_timestamp_offset.conf @@ -0,0 +1,50 @@ +# +# 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 = "STREAMING" + checkpoint.interval = 5000 +} + +source { + MySQL-CDC { + plugin_output = "customers_mysql_cdc" + server-id = 5654 + 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" + startup.mode = "timestamp" + startup.timestamp = ${timestamp} + } +} + +sink { + jdbc { + plugin_input = "customers_mysql_cdc" + url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" + driver = "com.mysql.cj.jdbc.Driver" + user = "st_user_sink" + password = "mysqlpw" + + generate_sink_sql = true + database = mysql_cdc + table = mysql_cdc_e2e_sink_table + primary_keys = ["id"] + } +} \ No newline at end of file From 41c06cff271b6086939df7ca546a083b294aeb69 Mon Sep 17 00:00:00 2001 From: 00193254 Date: Wed, 20 Aug 2025 16:23:27 +0800 Subject: [PATCH 06/37] init --- .../MysqlCDCSpecificStartingOffsetIT.java | 83 ++++++++++++------- 1 file changed, 54 insertions(+), 29 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java index 63df1f38653..b27ff9685d9 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java @@ -47,6 +47,7 @@ import com.github.shyiko.mysql.binlog.BinaryLogClient; import com.github.shyiko.mysql.binlog.event.EventData; import com.github.shyiko.mysql.binlog.event.EventHeaderV4; +import com.github.shyiko.mysql.binlog.event.FormatDescriptionEventData; import com.github.shyiko.mysql.binlog.event.RotateEventData; import io.debezium.jdbc.JdbcConnection; import lombok.extern.slf4j.Slf4j; @@ -63,6 +64,7 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import java.util.stream.Stream; import static org.awaitility.Awaitility.await; @@ -613,8 +615,7 @@ private void purgeBinaryLogs() { String.format("PURGE BINARY LOGS TO '%s'", getCurrentBinlogOffset().getFilename())); } - private long getCurrentBinlogTimestamp() throws IOException, InterruptedException { - + private long getCurrentBinlogTimestamp() { BinlogOffset binlogOffset = getCurrentBinlogOffset(); JdbcSourceConfigFactory configFactory = @@ -625,39 +626,63 @@ private long getCurrentBinlogTimestamp() throws IOException, InterruptedExceptio .password(MYSQL_CONTAINER.getPassword()) .databaseList(MYSQL_CONTAINER.getDatabaseName()); JdbcSourceConfig jdbcSourceConfig = configFactory.create(0); + MySqlDialect mySqlDialect = + new MySqlDialect((MySqlSourceConfigFactory) configFactory, Collections.emptyList()); BinaryLogClient client = MySqlConnectionUtils.createBinaryClient(jdbcSourceConfig.getDbzConfiguration()); - ArrayBlockingQueue binlogTimestamps = new ArrayBlockingQueue<>(1); - BinaryLogClient.EventListener eventListener = - event -> { - EventData data = event.getData(); - if (data instanceof RotateEventData) { - // We skip RotateEventData because it does not contain the timestamp we are - // interested in. - return; - } - - EventHeaderV4 header = event.getHeader(); - long timestamp = header.getTimestamp(); - if (timestamp > 0) { - binlogTimestamps.offer(timestamp); - try { - client.disconnect(); - } catch (IOException e) { - throw new RuntimeException(e); - } + final String showBinaryLogStmt = + "SHOW BINLOG EVENTS IN '" + binlogOffset.getFilename() + "'"; + List logPosList = new ArrayList<>(); + JdbcConnection.ResultSetConsumer rsc = + rs -> { + while (rs.next()) { + logPosList.add(rs.getLong(5)); } }; + try (JdbcConnection jdbc = mySqlDialect.openJdbcConnection(jdbcSourceConfig)) { + jdbc.query(showBinaryLogStmt, rsc); + if (logPosList.isEmpty()) { + return System.currentTimeMillis(); + } + log.info("SHOW BINLOG EVENTS result :{}", logPosList); + Long pos = logPosList.stream().distinct().sorted(Collections.reverseOrder()).collect(Collectors.toList()).get(1); + + ArrayBlockingQueue binlogTimestamps = new ArrayBlockingQueue<>(1); + BinaryLogClient.EventListener eventListener = + event -> { + EventData data = event.getData(); + if (data instanceof RotateEventData + || data instanceof FormatDescriptionEventData) { + // We skip RotateEventData because it does not contain the timestamp we + // are + // interested in. + return; + } - try { - client.registerEventListener(eventListener); - client.setBinlogFilename(binlogOffset.getFilename()); - client.setBinlogPosition(binlogOffset.getPosition()); - client.connect(); - } finally { - client.unregisterEventListener(eventListener); + EventHeaderV4 header = event.getHeader(); + long timestamp = header.getTimestamp(); + if (timestamp > 0) { + binlogTimestamps.offer(timestamp); + try { + client.disconnect(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }; + + try { + client.registerEventListener(eventListener); + client.setBinlogFilename(binlogOffset.getFilename()); + client.setBinlogPosition(pos); + client.connect(); + } finally { + client.unregisterEventListener(eventListener); + } + return binlogTimestamps.take(); + } catch (Exception e) { + throw new RuntimeException(e); } - return binlogTimestamps.take(); } } From f780aa68a6e524e1401734506fbb5d867176c293 Mon Sep 17 00:00:00 2001 From: 00193254 Date: Wed, 20 Aug 2025 16:24:53 +0800 Subject: [PATCH 07/37] init --- .../cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java index b27ff9685d9..dfdf7a04130 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCSpecificStartingOffsetIT.java @@ -646,7 +646,12 @@ private long getCurrentBinlogTimestamp() { return System.currentTimeMillis(); } log.info("SHOW BINLOG EVENTS result :{}", logPosList); - Long pos = logPosList.stream().distinct().sorted(Collections.reverseOrder()).collect(Collectors.toList()).get(1); + Long pos = + logPosList.stream() + .distinct() + .sorted(Collections.reverseOrder()) + .collect(Collectors.toList()) + .get(1); ArrayBlockingQueue binlogTimestamps = new ArrayBlockingQueue<>(1); BinaryLogClient.EventListener eventListener = From 02ec69b7b59969b7d68ff91c67947af59bcf8ef4 Mon Sep 17 00:00:00 2001 From: dyp12 <2513643704@qq.com> Date: Thu, 21 Aug 2025 12:26:24 +0800 Subject: [PATCH 08/37] init --- .../cdc/mysql/utils/MySqlConnectionUtils.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java index 0d44279b259..82cae86cec3 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java @@ -45,7 +45,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.atomic.AtomicLong; /** MySQL connection Utilities. */ public class MySqlConnectionUtils { @@ -241,9 +241,9 @@ private static String searchBinlogName( } public static long getBinlogTimestamp(BinaryLogClient client, String binlogFile) - throws IOException, InterruptedException { + throws IOException { - ArrayBlockingQueue binlogTimestamps = new ArrayBlockingQueue<>(1); + AtomicLong binlogTimestamps = new AtomicLong(); BinaryLogClient.EventListener eventListener = event -> { EventData data = event.getData(); @@ -255,8 +255,8 @@ public static long getBinlogTimestamp(BinaryLogClient client, String binlogFile) EventHeaderV4 header = event.getHeader(); long timestamp = header.getTimestamp(); - if (timestamp > 0) { - binlogTimestamps.offer(timestamp); + if (timestamp > 0 && binlogTimestamps.get() == 0) { + binlogTimestamps.set(timestamp); try { client.disconnect(); } catch (IOException e) { @@ -273,6 +273,6 @@ public static long getBinlogTimestamp(BinaryLogClient client, String binlogFile) } finally { client.unregisterEventListener(eventListener); } - return binlogTimestamps.take(); + return binlogTimestamps.get(); } } From 24d6f1a0d759e000dca17cfdc3a67b4d91873829 Mon Sep 17 00:00:00 2001 From: dyp12 <2513643704@qq.com> Date: Mon, 25 Aug 2025 15:42:41 +0800 Subject: [PATCH 09/37] init --- .../seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java index cee98df21e1..7b430daa2f3 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java @@ -199,7 +199,9 @@ private static Map querySystemVariables( public static BinlogOffset findBinlogOffsetBytimestamp( JdbcConnection jdbc, BinaryLogClient client, long timestamp) { - final String showBinaryLogStmt = "SHOW BINARY LOGS"; + final String showBinaryLogStmt = ((MySqlConnection) jdbc).binaryLogStatusStatement().startsWith("SHOW BINARY") + ? "SHOW BINARY LOGS" + : "SHOW MASTER LOGS"; List binlogFiles = new ArrayList<>(); JdbcConnection.ResultSetConsumer rsc = rs -> { From 51ad4e9e2fa053778276746c9dd3a2d297f39a04 Mon Sep 17 00:00:00 2001 From: dyp12 <2513643704@qq.com> Date: Mon, 25 Aug 2025 16:25:03 +0800 Subject: [PATCH 10/37] code format --- .../seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java index 7b430daa2f3..ecfbcbb7343 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlConnectionUtils.java @@ -199,9 +199,10 @@ private static Map querySystemVariables( public static BinlogOffset findBinlogOffsetBytimestamp( JdbcConnection jdbc, BinaryLogClient client, long timestamp) { - final String showBinaryLogStmt = ((MySqlConnection) jdbc).binaryLogStatusStatement().startsWith("SHOW BINARY") - ? "SHOW BINARY LOGS" - : "SHOW MASTER LOGS"; + final String showBinaryLogStmt = + ((MySqlConnection) jdbc).binaryLogStatusStatement().startsWith("SHOW BINARY") + ? "SHOW BINARY LOGS" + : "SHOW MASTER LOGS"; List binlogFiles = new ArrayList<>(); JdbcConnection.ResultSetConsumer rsc = rs -> { From 979fb5eee2a47a13cb6c4450857e36a642defd6f Mon Sep 17 00:00:00 2001 From: dyp12 Date: Fri, 29 Aug 2025 17:53:18 +0800 Subject: [PATCH 11/37] init --- docs/en/connector-v2/sink/CosFile.md | 12 ++- docs/en/connector-v2/sink/HdfsFile.md | 10 +++ docs/en/connector-v2/sink/LocalFile.md | 12 ++- docs/en/connector-v2/sink/ObsFile.md | 14 ++- docs/en/connector-v2/sink/OssFile.md | 54 +++++++----- docs/en/connector-v2/sink/OssJindoFile.md | 13 ++- docs/en/connector-v2/sink/S3File.md | 12 ++- docs/en/connector-v2/sink/SftpFile.md | 12 ++- docs/zh/connector-v2/sink/CosFile.md | 7 ++ docs/zh/connector-v2/sink/HdfsFile.md | 11 ++- docs/zh/connector-v2/sink/LocalFile.md | 9 +- docs/zh/connector-v2/sink/ObsFile.md | 59 +++++++------ docs/zh/connector-v2/sink/OssFile.md | 81 +++++++++-------- docs/zh/connector-v2/sink/OssJindoFile.md | 76 ++++++++-------- docs/zh/connector-v2/sink/S3File.md | 86 ++++++++++--------- docs/zh/connector-v2/sink/SftpFile.md | 82 ++++++++++-------- .../file/config/FileBaseSinkOptions.java | 7 ++ .../file/sink/config/FileSinkConfig.java | 10 +++ .../sink/writer/CanalJsonWriteStrategy.java | 12 ++- .../writer/DebeziumJsonWriteStrategy.java | 12 ++- .../sink/writer/MaxWellJsonWriteStrategy.java | 12 ++- .../json/canal/CanalJsonFormatOptions.java | 14 +++ .../canal/CanalJsonSerializationSchema.java | 45 +++++++--- .../debezium/DebeziumJsonFormatOptions.java | 14 +++ .../DebeziumJsonSerializationSchema.java | 23 ++++- .../maxwell/MaxWellJsonFormatOptions.java | 14 +++ .../MaxWellJsonSerializationSchema.java | 47 ++++++---- .../format/json/ogg/OggJsonFormatOptions.java | 14 +++ .../json/ogg/OggJsonSerializationSchema.java | 49 ++++++++--- 29 files changed, 571 insertions(+), 252 deletions(-) diff --git a/docs/en/connector-v2/sink/CosFile.md b/docs/en/connector-v2/sink/CosFile.md index 457bee37708..efa3b60d270 100644 --- a/docs/en/connector-v2/sink/CosFile.md +++ b/docs/en/connector-v2/sink/CosFile.md @@ -37,6 +37,9 @@ To use this connector you need put hadoop-cos-{hadoop.version}-{version}.jar and - [x] excel - [x] xml - [x] binary + - [x] canal_json + - [x] debezium_json + - [x] maxwell_json ## Options @@ -76,6 +79,7 @@ To use this connector you need put hadoop-cos-{hadoop.version}-{version}.jar and | parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | | parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | | encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | ### path [string] @@ -129,7 +133,7 @@ When the format in the `file_name_expression` parameter is `xxxx-${now}` , `file We supported as the following file types: -`text` `csv` `parquet` `orc` `json` `excel` `xml` `binary` +`text` `csv` `parquet` `orc` `json` `excel` `xml` `binary` `canal_json` `debezium_json` `maxwell_json` Please note that, The final file name will end with the file_format's suffix, the suffix of the text file is `txt`. @@ -245,6 +249,12 @@ Support writing Parquet INT96 from a 12-byte field, only valid for parquet files Only used when file_format_type is json,text,csv,xml. The encoding of the file to write. This param will be parsed by `Charset.forName(encoding)`. +### merge_update_event [boolean] + +Only used when file_format_type is canal_json,debezium_json,maxwell_json. +set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; +set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event + ## Example For text file format with `have_partition` and `custom_filename` and `sink_columns` diff --git a/docs/en/connector-v2/sink/HdfsFile.md b/docs/en/connector-v2/sink/HdfsFile.md index 6514599cb34..0292f5a6b48 100644 --- a/docs/en/connector-v2/sink/HdfsFile.md +++ b/docs/en/connector-v2/sink/HdfsFile.md @@ -32,6 +32,9 @@ By default, we use 2PC commit to ensure `exactly-once` - [x] binary - [x] compress codec - [x] lzo + - [x] canal_json + - [x] debezium_json + - [x] maxwell_json ## Description @@ -85,11 +88,18 @@ Output data to hdfs file | enable_header_write | boolean | no | false | Only used when file_format_type is text,csv.
false:don't write header,true:write header. | | encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | | remote_user | string | no | - | The remote user name of hdfs. | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | ### Tips > If you use spark/flink, In order to use this connector, You must ensure your spark/flink cluster already integrated hadoop. The tested hadoop version is 2.x. If you use SeaTunnel Engine, It automatically integrated the hadoop jar when you download and install SeaTunnel Engine. You can check the jar package under ${SEATUNNEL_HOME}/lib to confirm this. +### merge_update_event [boolean] + +Only used when file_format_type is canal_json,debezium_json,maxwell_json. +set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; +set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event + ## Task Example ### Simple diff --git a/docs/en/connector-v2/sink/LocalFile.md b/docs/en/connector-v2/sink/LocalFile.md index 5ea663514e9..30cd929577e 100644 --- a/docs/en/connector-v2/sink/LocalFile.md +++ b/docs/en/connector-v2/sink/LocalFile.md @@ -36,6 +36,9 @@ By default, we use 2PC commit to ensure `exactly-once` - [x] excel - [x] xml - [x] binary + - [x] canal_json + - [x] debezium_json + - [x] maxwell_json ## Options @@ -74,6 +77,7 @@ By default, we use 2PC commit to ensure `exactly-once` | encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | | schema_save_mode | string | no | CREATE_SCHEMA_WHEN_NOT_EXIST | Existing dir processing method | | data_save_mode | string | no | APPEND_DATA | Existing data processing method | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | ### path [string] @@ -111,7 +115,7 @@ When the format in the `file_name_expression` parameter is `xxxx-${now}` , `file We supported as the following file types: -`text` `csv` `parquet` `orc` `json` `excel` `xml` `binary` +`text` `csv` `parquet` `orc` `json` `excel` `xml` `binary` `canal_json` `debezium_json` `maxwell_json` Please note that, The final file name will end with the file_format_type's suffix, the suffix of the text file is `txt`. @@ -246,6 +250,12 @@ Existing data processing method. - APPEND_DATA: preserve dir, preserve data files - ERROR_WHEN_DATA_EXISTS: when there is data files, an error is reported +### merge_update_event [boolean] + +Only used when file_format_type is canal_json,debezium_json,maxwell_json. +set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; +set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event + ## Example For orc file format simple config diff --git a/docs/en/connector-v2/sink/ObsFile.md b/docs/en/connector-v2/sink/ObsFile.md index c3a171f4fb7..88be0590866 100644 --- a/docs/en/connector-v2/sink/ObsFile.md +++ b/docs/en/connector-v2/sink/ObsFile.md @@ -29,6 +29,9 @@ By default, we use 2PC commit to ensure `exactly-once` - [x] orc - [x] json - [x] excel + - [x] canal_json + - [x] debezium_json + - [x] maxwell_json ## Description @@ -68,7 +71,7 @@ It only supports hadoop version **2.9.X+**. | filename_time_format | string | no | "yyyy.MM.dd" | Specify the time format of the `path`. Only used when custom_filename is true. [Tips](#filename_time_format) | | file_format_type | string | no | "csv" | Supported file types. [Tips](#file_format_type) | | field_delimiter | string | no | '\001' | The separator between columns in a row of data.Only used when file_format is text. | -| row_delimiter | string | no | "\n" | The separator between rows in a file. Only needed by `text`, `csv` and `json` file format. | +| row_delimiter | string | no | "\n" | The separator between rows in a file. Only needed by `text`, `csv` and `json` file format. | | have_partition | boolean | no | false | Whether you need processing partitions. | | partition_by | array | no | - | Partition data based on selected fields. Only used then have_partition is true. | | partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true.[Tips](#partition_dir_expression) | @@ -83,6 +86,7 @@ It only supports hadoop version **2.9.X+**. | max_rows_in_memory | int | no | - | When File Format is Excel,The maximum number of data items that can be cached in the memory.Only used when file_format is excel. | | sheet_name | string | no | Sheet${Random number} | Writer the sheet of the workbook. Only used when file_format is excel. | | sheet_max_rows | int | no | 1048576 | Only used when file format_type is excel. | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | ### Tips @@ -117,7 +121,7 @@ Please note that, If `is_enable_transaction` is `true`, we will auto add `${tran > We supported as the following file types: > -> `text` `json` `csv` `orc` `parquet` `excel` +> `text` `json` `csv` `orc` `parquet` `excel` `canal_json` `debezium_json` `maxwell_json` Please note that, The final file name will end with the file_format's suffix, the suffix of the text file is `txt`. @@ -164,6 +168,12 @@ Please note that, The final file name will end with the file_format's suffix, th Please note that excel type does not support any compression format +#### merge_update_event + +> Only used when file_format_type is canal_json,debezium_json,maxwell_json. +> set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; +> set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event + #### common options > Sink plugin common parameters, please refer to [Sink Common Options](../sink-common-options.md) for details. diff --git a/docs/en/connector-v2/sink/OssFile.md b/docs/en/connector-v2/sink/OssFile.md index 5fed86ecb5d..8b39e899dab 100644 --- a/docs/en/connector-v2/sink/OssFile.md +++ b/docs/en/connector-v2/sink/OssFile.md @@ -41,6 +41,9 @@ import ChangeLog from '../changelog/connector-file-oss.md'; - [x] excel - [x] xml - [x] binary + - [x] canal_json + - [x] debezium_json + - [x] maxwell_json ## Data Type Mapping @@ -110,27 +113,28 @@ If write to `csv`, `text`, `json` file type, All column will be string. | have_partition | boolean | no | false | Whether you need processing partitions. | | partition_by | array | no | - | Only used then have_partition is true | | partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | -| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true | -| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | -| is_enable_transaction | boolean | no | true | | -| batch_size | int | no | 1000000 | | -| compress_codec | string | no | none | | -| common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | -| sheet_max_rows | int | no | 1048576 | Only used when file format_type is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | -| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | -| xml_root_tag | string | no | RECORDS | Only used when file_format is xml. | -| xml_row_tag | string | no | RECORD | Only used when file_format is xml. | -| xml_use_attr_format | boolean | no | - | Only used when file_format is xml. | -| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | -| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | -| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | -| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | -| enable_header_write | boolean | no | false | Only used when file_format_type is text,csv.
false:don't write header,true:write header. | -| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | -| schema_save_mode | Enum | no | CREATE_SCHEMA_WHEN_NOT_EXIST | Before turning on the synchronous task, do different treatment of the target path | -| data_save_mode | Enum | no | APPEND_DATA | Before opening the synchronous task, the data file in the target path is differently processed | +| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true | +| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | +| is_enable_transaction | boolean | no | true | | +| batch_size | int | no | 1000000 | | +| compress_codec | string | no | none | | +| common-options | object | no | - | | +| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | +| sheet_max_rows | int | no | 1048576 | Only used when file format_type is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | +| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | +| xml_root_tag | string | no | RECORDS | Only used when file_format is xml. | +| xml_row_tag | string | no | RECORD | Only used when file_format is xml. | +| xml_use_attr_format | boolean | no | - | Only used when file_format is xml. | +| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | +| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | +| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | +| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | +| enable_header_write | boolean | no | false | Only used when file_format_type is text,csv.
false:don't write header,true:write header. | +| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | +| schema_save_mode | Enum | no | CREATE_SCHEMA_WHEN_NOT_EXIST | Before turning on the synchronous task, do different treatment of the target path | +| data_save_mode | Enum | no | APPEND_DATA | Before opening the synchronous task, the data file in the target path is differently processed | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | ### path [string] @@ -184,7 +188,7 @@ When the format in the `file_name_expression` parameter is `xxxx-${Now}` , `file We supported as the following file types: -`text` `csv` `parquet` `orc` `json` `excel` `xml` `binary` +`text` `csv` `parquet` `orc` `json` `excel` `xml` `binary` `canal_json` `debezium_json` `maxwell_json` Please note that, The final file name will end with the file_format_type's suffix, the suffix of the text file is `txt`. @@ -317,6 +321,12 @@ Option introduction: `APPEND_DATA`:use the path, and add new files in the path for write data. `ERROR_WHEN_DATA_EXISTS`:When there are some data files in the path, an error will is reported. +### merge_update_event [boolean] + +Only used when file_format_type is canal_json,debezium_json,maxwell_json. +set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; +set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event + ## How to Create an Oss Data Synchronization Jobs The following example demonstrates how to create a data synchronization job that reads data from Fake Source and writes diff --git a/docs/en/connector-v2/sink/OssJindoFile.md b/docs/en/connector-v2/sink/OssJindoFile.md index 07de7f0f877..b1e1f104e3b 100644 --- a/docs/en/connector-v2/sink/OssJindoFile.md +++ b/docs/en/connector-v2/sink/OssJindoFile.md @@ -41,6 +41,9 @@ It only supports hadoop version **2.9.X+**. - [x] excel - [x] xml - [x] binary + - [x] canal_json + - [x] debezium_json + - [x] maxwell_json ## Options @@ -80,6 +83,7 @@ It only supports hadoop version **2.9.X+**. | parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | | parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | | encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | ### path [string] @@ -133,7 +137,7 @@ When the format in the `file_name_expression` parameter is `xxxx-${now}` , `file We supported as the following file types: -`text` `csv` `parquet` `orc` `json` `excel` `xml` `binary` +`text` `csv` `parquet` `orc` `json` `excel` `xml` `binary` `canal_json` `debezium_json` `maxwell_json` Please note that, The final file name will end with the file_format_type's suffix, the suffix of the text file is `txt`. @@ -249,6 +253,13 @@ Support writing Parquet INT96 from a 12-byte field, only valid for parquet files Only used when file_format_type is json,text,csv,xml. The encoding of the file to write. This param will be parsed by `Charset.forName(encoding)`. +### merge_update_event [boolean] + +Only used when file_format_type is canal_json,debezium_json,maxwell_json. +set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; +set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event + + ## Example For text file format with `have_partition` and `custom_filename` and `sink_columns` diff --git a/docs/en/connector-v2/sink/S3File.md b/docs/en/connector-v2/sink/S3File.md index 3fc4f25459d..84f13040ea4 100644 --- a/docs/en/connector-v2/sink/S3File.md +++ b/docs/en/connector-v2/sink/S3File.md @@ -32,6 +32,9 @@ import ChangeLog from '../changelog/connector-file-s3.md'; - [x] excel - [x] xml - [x] binary + - [x] canal_json + - [x] debezium_json + - [x] maxwell_json ## Description @@ -141,6 +144,7 @@ If write to `csv`, `text` file type, All column will be string. | data_save_mode | Enum | no | APPEND_DATA | Before opening the synchronous task, the data file in the target path is differently processed | | enable_header_write | boolean | no | false | Only used when file_format_type is text,csv.
false:don't write header,true:write header. | | encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | ### path [string] @@ -189,7 +193,7 @@ When the format in the `file_name_expression` parameter is `xxxx-${now}` , `file We supported as the following file types: -`text` `csv` `parquet` `orc` `json` `excel` `xml` `binary` +`text` `csv` `parquet` `orc` `json` `excel` `xml` `binary` `canal_json` `debezium_json` `maxwell_json` Please note that, The final file name will end with the file_format_type's suffix, the suffix of the text file is `txt`. @@ -322,6 +326,12 @@ Option introduction: Only used when file_format_type is json,text,csv,xml. The encoding of the file to write. This param will be parsed by `Charset.forName(encoding)`. +### merge_update_event [boolean] + +Only used when file_format_type is canal_json,debezium_json,maxwell_json. +set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; +set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event + ## Example ### Simple diff --git a/docs/en/connector-v2/sink/SftpFile.md b/docs/en/connector-v2/sink/SftpFile.md index c939855a542..50827794c95 100644 --- a/docs/en/connector-v2/sink/SftpFile.md +++ b/docs/en/connector-v2/sink/SftpFile.md @@ -35,6 +35,9 @@ If you use SeaTunnel Engine, It automatically integrated the hadoop jar when you - [x] excel - [x] xml - [x] binary + - [x] canal_json + - [x] debezium_json + - [x] maxwell_json ## Options @@ -77,6 +80,7 @@ If you use SeaTunnel Engine, It automatically integrated the hadoop jar when you | encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | | schema_save_mode | string | no | CREATE_SCHEMA_WHEN_NOT_EXIST | Existing dir processing method | | data_save_mode | string | no | APPEND_DATA | Existing data processing method | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | ### host [string] @@ -130,7 +134,7 @@ When the format in the `file_name_expression` parameter is `xxxx-${now}` , `file We supported as the following file types: -`text` `csv` `parquet` `orc` `json` `excel` `xml` `binary` +`text` `csv` `parquet` `orc` `json` `excel` `xml` `binary` `canal_json` `debezium_json` `maxwell_json` Please note that, The final file name will end with the file_format_type's suffix, the suffix of the text file is `txt`. @@ -267,6 +271,12 @@ Existing data processing method. - APPEND_DATA: preserve dir, preserve data files - ERROR_WHEN_DATA_EXISTS: when there is data files, an error is reported +### merge_update_event [boolean] + +Only used when file_format_type is canal_json,debezium_json,maxwell_json. +set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; +set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event + ## Example For text file format with `have_partition` and `custom_filename` and `sink_columns` diff --git a/docs/zh/connector-v2/sink/CosFile.md b/docs/zh/connector-v2/sink/CosFile.md index 63e64fb1d7e..5cd3b79daa5 100644 --- a/docs/zh/connector-v2/sink/CosFile.md +++ b/docs/zh/connector-v2/sink/CosFile.md @@ -78,6 +78,7 @@ import ChangeLog from '../changelog/connector-file-cos.md'; | parquet_avro_write_timestamp_as_int96 | boolean | 否 | false | 仅在file_format为parquet时使用. | | parquet_avro_write_fixed_as_int96 | array | 否 | - | 仅在file_format为parquet时使用. | | encoding | string | 否 | "UTF-8" | 仅当file_format_type为json、text、csv、xml时使用. | +| merge_update_event | boolean | 否 | false | 仅当file_format_type为canal_json、debezium_json、maxwell_json. | ### path [string] @@ -242,6 +243,12 @@ Tips: excel 类型不支持任何压缩格式 仅当file_format_type为json、text、csv、xml时使用. 要写入的文件的编码。此参数将由`Charset.forName(encoding)` 解析. +### merge_update_event [boolean] + +仅当file_format_type为canal_json、debezium_json、maxwell_json时使用. +设置成true,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 会合并成 UPDATE; +设置成false,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 不会合并; + ## 示例 对于具有 `have_partition` 、 `custom_filename` 和 `sink_columns` 的文本文件格式 diff --git a/docs/zh/connector-v2/sink/HdfsFile.md b/docs/zh/connector-v2/sink/HdfsFile.md index 384a20b7f40..0e61ea97641 100644 --- a/docs/zh/connector-v2/sink/HdfsFile.md +++ b/docs/zh/connector-v2/sink/HdfsFile.md @@ -55,10 +55,10 @@ import ChangeLog from '../changelog/connector-file-hadoop.md'; | custom_filename | boolean | 否 | false | 是否需要自定义文件名 | | file_name_expression | string | 否 | "${transactionId}" | 仅在 `custom_filename` 为 `true` 时使用。`file_name_expression` 描述将创建到 `path` 中的文件表达式。我们可以在 `file_name_expression` 中添加变量 `${now}` 或 `${uuid}`,例如 `test_${uuid}_${now}`,`${now}` 表示当前时间,其格式可以通过指定选项 `filename_time_format` 来定义。请注意,如果 `is_enable_transaction` 为 `true`,我们将在文件头部自动添加 `${transactionId}_`。 | | filename_time_format | string | 否 | "yyyy.MM.dd" | 仅在 `custom_filename` 为 `true` 时使用。当 `file_name_expression` 参数中的格式为 `xxxx-${now}` 时,`filename_time_format` 可以指定路径的时间格式,默认值为 `yyyy.MM.dd`。常用的时间格式如下所示:[y:年,M:月,d:月中的一天,H:一天中的小时(0-23),m:小时中的分钟,s:分钟中的秒] | -| file_format_type | string | 否 | "csv" | 我们支持以下文件类型:`text` `json` `csv` `orc` `parquet` `excel` `canal_json` `debezium_json` `maxwell_json`。请注意,最终文件名将以文件格式的后缀结束,文本文件的后缀是 `txt`。 | +| file_format_type | string | 否 | "csv" | 我们支持以下文件类型:`text` `json` `csv` `orc` `parquet` `excel` `canal_json` `debezium_json` `maxwell_json`。请注意,最终文件名将以文件格式的后缀结束,文本文件的后缀是 `txt`。 | | filename_extension | string | 否 | - | 使用自定义的文件扩展名覆盖默认的文件扩展名。 例如:`.xml`, `.json`, `dat`, `.customtype` | | field_delimiter | string | 否 | '\001' | 仅在 file_format 为 text 时使用,数据行中列之间的分隔符。仅需要 `text` 文件格式。 | -| row_delimiter | string | 否 | "\n" | 仅在 file_format 为 text 时使用,文件中行之间的分隔符。仅需要 `text`、`csv`、`json` 文件格式。 | +| row_delimiter | string | 否 | "\n" | 仅在 file_format 为 text 时使用,文件中行之间的分隔符。仅需要 `text`、`csv`、`json` 文件格式。 | | have_partition | boolean | 否 | false | 是否需要处理分区。 | | partition_by | array | 否 | - | 仅在 have_partition 为 true 时使用,根据选定的字段对数据进行分区。 | | partition_dir_expression | string | 否 | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | 仅在 have_partition 为 true 时使用,如果指定了 `partition_by`,我们将根据分区信息生成相应的分区目录,并将最终文件放置在分区目录中。默认 `partition_dir_expression` 为 `${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/`。`k0` 是第一个分区字段,`v0` 是第一个分区字段的值。 | @@ -79,6 +79,7 @@ import ChangeLog from '../changelog/connector-file-hadoop.md'; | max_rows_in_memory | int | 否 | - | 仅当 file_format 为 excel 时使用。当文件格式为 Excel 时,可以缓存在内存中的最大数据项数。 | | sheet_name | string | 否 | Sheet${Random number} | 仅当 file_format 为 excel 时使用。将工作簿的表写入指定的表名 | | remote_user | string | 否 | - | Hdfs的远端用户名。 | +| merge_update_event | boolean | 否 | false | 仅当file_format_type为canal_json、debezium_json、maxwell_json. | ### 提示 @@ -86,6 +87,12 @@ import ChangeLog from '../changelog/connector-file-hadoop.md'; > 2.x。如果您使用 SeaTunnel Engine,则在下载和安装 SeaTunnel Engine 时会自动集成 hadoop > jar。您可以检查 `${SEATUNNEL_HOME}/lib` 下的 jar 包来确认这一点。 +### merge_update_event [boolean] + +仅当file_format_type为canal_json、debezium_json、maxwell_json时使用. +设置成true,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 会合并成 UPDATE; +设置成false,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 不会合并; + ## 任务示例 ### 简单示例 diff --git a/docs/zh/connector-v2/sink/LocalFile.md b/docs/zh/connector-v2/sink/LocalFile.md index 55e394a8100..7bc92a564c5 100644 --- a/docs/zh/connector-v2/sink/LocalFile.md +++ b/docs/zh/connector-v2/sink/LocalFile.md @@ -51,7 +51,7 @@ import ChangeLog from '../changelog/connector-file-local.md'; | file_format_type | string | 否 | "csv" | 文件格式类型 | | filename_extension | string | 否 | - | 使用自定义的文件扩展名覆盖默认的文件扩展名。 例如:`.xml`, `.json`, `dat`, `.customtype` | | field_delimiter | string | 否 | '\001' | 仅在 file_format_type 为 text 时使用 | -| row_delimiter | string | 否 | "\n" | 仅在 file_format_type 为 `text`、`csv`、`json` 时使用 | +| row_delimiter | string | 否 | "\n" | 仅在 file_format_type 为 `text`、`csv`、`json` 时使用 | | have_partition | boolean | 否 | false | 是否需要处理分区 | | partition_by | array | 否 | - | 仅在 have_partition 为 true 时使用 | | partition_dir_expression | string | 否 | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | 仅在 have_partition 为 true 时使用 | @@ -73,6 +73,7 @@ import ChangeLog from '../changelog/connector-file-local.md'; | parquet_avro_write_fixed_as_int96 | array | 否 | - | 仅在 file_format 为 parquet 时使用 | | enable_header_write | boolean | 否 | false | 仅在 file_format_type 为 text,csv 时使用。
false:不写入表头,true:写入表头。 | | encoding | string | 否 | "UTF-8" | 仅在 file_format_type 为 json,text,csv,xml 时使用 | +| merge_update_event | boolean | 否 | false | 仅当file_format_type为canal_json、debezium_json、maxwell_json. | ### path [string] @@ -225,6 +226,12 @@ _root_tag [string] 仅在 file_format_type 为 json,text,csv,xml 时使用。文件写入的编码。该参数将通过 `Charset.forName(encoding)` 解析。 +### merge_update_event [boolean] + +仅当file_format_type为canal_json、debezium_json、maxwell_json时使用. +设置成true,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 会合并成 UPDATE; +设置成false,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 不会合并; + ## 示例 对于 orc 文件格式的简单配置 diff --git a/docs/zh/connector-v2/sink/ObsFile.md b/docs/zh/connector-v2/sink/ObsFile.md index 4069ae27e32..1929bb306bd 100644 --- a/docs/zh/connector-v2/sink/ObsFile.md +++ b/docs/zh/connector-v2/sink/ObsFile.md @@ -59,32 +59,33 @@ import ChangeLog from '../changelog/connector-file-obs.md'; ## 参数 -| 名称 | 类型 | 是否必填 | 默认值 | 描述 | -|----------------------------------|---------|---------|--------------------------------------------|-------------------------------------------------------------------------| -| path | string | 是 | - | 目标目录路径。 | -| bucket | string | 是 | - | obs文件系统的bucket地址,例如:`obs://obs-bucket-name`. | -| access_key | string | 是 | - | obs文件系统的访问密钥。 | -| access_secret | string | 是 | - | obs文件系统的访问私钥。 | -| endpoint | string | 是 | - | obs文件系统的终端。 | -| custom_filename | boolean | 否 | false | 是否需要自定义文件名。 | -| file_name_expression | string | 否 | "${transactionId}" | 描述将在“路径”中创建的文件表达式。仅在custom_filename为true时使用。[提示](#file_name_expression) | -| filename_time_format | string | 否 | "yyyy.MM.dd" | 指定“path”的时间格式。仅在custom_filename为true时使用。[提示](#filename_time_format) | -| file_format_type | string | 否 | "csv" | 支持的文件类型。[提示](#file_format_type) | -| field_delimiter | string | 否 | '\001' | 数据行中列之间的分隔符。仅在file_format为文本时使用。 | -| row_delimiter | string | 否 | "\n" | 文件中行之间的分隔符。仅被 `text`、`csv`、`json` 文件格式需要。 | -| have_partition | boolean | 否 | false | 是否需要处理分区。 | -| partition_by | array | 否 | - | 根据所选字段对数据进行分区。只有在have_partition为true时才使用。 | -| partition_dir_expression | string | 否 | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | 只有在have_partition为真true时才使用。[提示](#partition_dir_expression) | -| is_partition_field_write_in_file | boolean | 否 | false | 只有在have_partition为true时才使用。[提示](#is_partition_field_write_in_file) | -| sink_columns | array | 否 | | 当此参数为空时,所有字段都是接收列。[提示](#sink_columns) | -| is_enable_transaction | boolean | 否 | true | [提示](#is_enable_transaction) | -| batch_size | int | 否 | 1000000 | [提示](#batch_size) | -| single_file_mode | boolean | 否 | false | 每个并行处理只会输出一个文件。启用此参数后,batch_size将不会生效。输出文件名没有文件块后缀。 | -| create_empty_file_when_no_data | boolean | 否 | false | 当上游没有数据同步时,仍然会生成相应的数据文件。 | -| compress_codec | string | 否 | none | [提示](#compress_codec) | -| common-options | object | 否 | - | [提示](#common_options) | -| max_rows_in_memory | int | 否 | - | 当文件格式为Excel时,内存中可以缓存的最大数据项数。仅在file_format为excel时使用。 | -| sheet_name | string | 否 | Sheet${Random number} | 标签页。仅在file_format为excel时使用。 | +| 名称 | 类型 | 是否必填 | 默认值 | 描述 | +|----------------------------------|---------|------|--------------------------------------------|-------------------------------------------------------------------------| +| path | string | 是 | - | 目标目录路径。 | +| bucket | string | 是 | - | obs文件系统的bucket地址,例如:`obs://obs-bucket-name`. | +| access_key | string | 是 | - | obs文件系统的访问密钥。 | +| access_secret | string | 是 | - | obs文件系统的访问私钥。 | +| endpoint | string | 是 | - | obs文件系统的终端。 | +| custom_filename | boolean | 否 | false | 是否需要自定义文件名。 | +| file_name_expression | string | 否 | "${transactionId}" | 描述将在“路径”中创建的文件表达式。仅在custom_filename为true时使用。[提示](#file_name_expression) | +| filename_time_format | string | 否 | "yyyy.MM.dd" | 指定“path”的时间格式。仅在custom_filename为true时使用。[提示](#filename_time_format) | +| file_format_type | string | 否 | "csv" | 支持的文件类型。[提示](#file_format_type) | +| field_delimiter | string | 否 | '\001' | 数据行中列之间的分隔符。仅在file_format为文本时使用。 | +| row_delimiter | string | 否 | "\n" | 文件中行之间的分隔符。仅被 `text`、`csv`、`json` 文件格式需要。 | +| have_partition | boolean | 否 | false | 是否需要处理分区。 | +| partition_by | array | 否 | - | 根据所选字段对数据进行分区。只有在have_partition为true时才使用。 | +| partition_dir_expression | string | 否 | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | 只有在have_partition为真true时才使用。[提示](#partition_dir_expression) | +| is_partition_field_write_in_file | boolean | 否 | false | 只有在have_partition为true时才使用。[提示](#is_partition_field_write_in_file) | +| sink_columns | array | 否 | | 当此参数为空时,所有字段都是接收列。[提示](#sink_columns) | +| is_enable_transaction | boolean | 否 | true | [提示](#is_enable_transaction) | +| batch_size | int | 否 | 1000000 | [提示](#batch_size) | +| single_file_mode | boolean | 否 | false | 每个并行处理只会输出一个文件。启用此参数后,batch_size将不会生效。输出文件名没有文件块后缀。 | +| create_empty_file_when_no_data | boolean | 否 | false | 当上游没有数据同步时,仍然会生成相应的数据文件。 | +| compress_codec | string | 否 | none | [提示](#compress_codec) | +| common-options | object | 否 | - | [提示](#common_options) | +| max_rows_in_memory | int | 否 | - | 当文件格式为Excel时,内存中可以缓存的最大数据项数。仅在file_format为excel时使用。 | +| sheet_name | string | 否 | Sheet${Random number} | 标签页。仅在file_format为excel时使用。 | +| merge_update_event | boolean | 否 | false | 仅当file_format_type为canal_json、debezium_json、maxwell_json. | ### 提示 @@ -165,6 +166,12 @@ import ChangeLog from '../changelog/connector-file-obs.md'; 请注意,excel类型不支持任何压缩格式 +#### merge_update_event + +>仅当file_format_type为canal_json、debezium_json、maxwell_json时使用. +>设置成true,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 会合并成 UPDATE; +>设置成false,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 不会合并; + #### common options >Sink插件常用参数,请参考[Sink common Options](../Sink-common-Options.md)了解详细信息。 diff --git a/docs/zh/connector-v2/sink/OssFile.md b/docs/zh/connector-v2/sink/OssFile.md index dd8455736ba..943abf6a9a6 100644 --- a/docs/zh/connector-v2/sink/OssFile.md +++ b/docs/zh/connector-v2/sink/OssFile.md @@ -97,43 +97,44 @@ import ChangeLog from '../changelog/connector-file-oss.md'; ## 选项 -| 名称 | 类型 | 必需 | 默认值 | 描述 | -|---------------------------------------|---------|----|--------------------------------------------|-------------------------------------------------------| -| path | string | 是 | 写入文件的oss路径。 | | -| tmp_path | string | 否 | /tmp/seatunnel | 结果文件将首先写入tmp路径,然后使用`mv`将tmp-dir提交到目标dir。因此需要一个OSS目录。 | -| bucket | string | 是 | - | | -| access_key | string | 是 | - | | -| access_secret | string | 是 | - | | -| endpoint | string | 是 | - | | -| custom_filename | boolean | 否 | false | 是否需要自定义文件名 | -| file_name_expression | string | 否 | "${transactionId}" | 仅在custom_filename为true时使用 | -| filename_time_format | string | 否 | "yyyy.MM.dd" | 仅在custom_filename为true时使用 | -| file_format_type | string | 否 | "csv" | | -| field_delimiter | string | 否 | '\001' | 仅当file_format_type为文本时使用 | -| row_delimiter | string | 否 | "\n" | 仅当file_format_type为 `text`、`csv`、`json` 时使用 | -| have_partition | boolean | 否 | false | 是否需要处理分区。 | -| partition_by | array | 否 | - | 只有在have_partition为true时才使用 | -| partition_dir_expression | string | 否 | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | 只有在have_partition为true时才使用 | -| is_partition_field_write_in_file | boolean | 否 | false | 只有在have_partition为true时才使用 | -| sink_columns | array | 否 | | 当此参数为空时,所有字段都是接收列 | -| is_enable_transaction | boolean | 否 | true | | -| batch_size | int | 否 | 1000000 | | -| compress_codec | string | 否 | none | | -| common-options | object | 否 | - | | -| max_rows_in_memory | int | 否 | - | 仅当file_format_type为excel时使用。 | -| sheet_name | string | 否 | Sheet${Random number} | 仅当file_format_type为excel时使用。 | -| csv_string_quote_mode | enum | 否 | MINIMAL | 仅在file_format为csv时使用。 | -| xml_root_tag | string | 否 | RECORDS | 仅在file_format为xml时使用。 | -| xml_row_tag | string | 否 | RECORD | 仅在file_format为xml时使用。 | -| xml_use_attr_format | boolean | 否 | - | 仅在file_format为xml时使用。 | -| single_file_mode | boolean | 否 | false | 每个并行处理只会输出一个文件。启用此参数后,batch_size将不会生效。输出文件名没有文件块后缀。 | -| create_empty_file_when_no_data | boolean | 否 | false | 当上游没有数据同步时,仍然会生成相应的数据文件。 | -| parquet_avro_write_timestamp_as_int96 | boolean | 否 | false | 仅在file_format为parquet时使用。 | -| parquet_avro_write_fixed_as_int96 | array | 否 | - | 仅在file_format为parquet时使用。 | -| enable_header_write | boolean | 否 | false | 仅当file_format_type为文本、csv时使用
false:不写标头,true:写标头。 | -| encoding | string | 否 | "UTF-8" | 仅当file_format_type为json、text、csv、xml时使用。 | -| schema_save_mode | Enum | 否 | CREATE_SCHEMA_WHEN_NOT_EXIST | 在开启同步任务之前,对目标路径进行不同的处理 | -| data_save_mode | Enum | 否 | APPEND_DATA | 在开启同步任务之前,对目标路径中的数据文件进行不同的处理 | +| 名称 | 类型 | 必需 | 默认值 | 描述 | +|---------------------------------------|---------|----|--------------------------------------------|-------------------------------------------------------------------| +| path | string | 是 | 写入文件的oss路径。 | | +| tmp_path | string | 否 | /tmp/seatunnel | 结果文件将首先写入tmp路径,然后使用`mv`将tmp-dir提交到目标dir。因此需要一个OSS目录。 | +| bucket | string | 是 | - | | +| access_key | string | 是 | - | | +| access_secret | string | 是 | - | | +| endpoint | string | 是 | - | | +| custom_filename | boolean | 否 | false | 是否需要自定义文件名 | +| file_name_expression | string | 否 | "${transactionId}" | 仅在custom_filename为true时使用 | +| filename_time_format | string | 否 | "yyyy.MM.dd" | 仅在custom_filename为true时使用 | +| file_format_type | string | 否 | "csv" | | +| field_delimiter | string | 否 | '\001' | 仅当file_format_type为文本时使用 | +| row_delimiter | string | 否 | "\n" | 仅当file_format_type为 `text`、`csv`、`json` 时使用 | +| have_partition | boolean | 否 | false | 是否需要处理分区。 | +| partition_by | array | 否 | - | 只有在have_partition为true时才使用 | +| partition_dir_expression | string | 否 | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | 只有在have_partition为true时才使用 | +| is_partition_field_write_in_file | boolean | 否 | false | 只有在have_partition为true时才使用 | +| sink_columns | array | 否 | | 当此参数为空时,所有字段都是接收列 | +| is_enable_transaction | boolean | 否 | true | | +| batch_size | int | 否 | 1000000 | | +| compress_codec | string | 否 | none | | +| common-options | object | 否 | - | | +| max_rows_in_memory | int | 否 | - | 仅当file_format_type为excel时使用。 | +| sheet_name | string | 否 | Sheet${Random number} | 仅当file_format_type为excel时使用。 | +| csv_string_quote_mode | enum | 否 | MINIMAL | 仅在file_format为csv时使用。 | +| xml_root_tag | string | 否 | RECORDS | 仅在file_format为xml时使用。 | +| xml_row_tag | string | 否 | RECORD | 仅在file_format为xml时使用。 | +| xml_use_attr_format | boolean | 否 | - | 仅在file_format为xml时使用。 | +| single_file_mode | boolean | 否 | false | 每个并行处理只会输出一个文件。启用此参数后,batch_size将不会生效。输出文件名没有文件块后缀。 | +| create_empty_file_when_no_data | boolean | 否 | false | 当上游没有数据同步时,仍然会生成相应的数据文件。 | +| parquet_avro_write_timestamp_as_int96 | boolean | 否 | false | 仅在file_format为parquet时使用。 | +| parquet_avro_write_fixed_as_int96 | array | 否 | - | 仅在file_format为parquet时使用。 | +| enable_header_write | boolean | 否 | false | 仅当file_format_type为文本、csv时使用
false:不写标头,true:写标头。 | +| encoding | string | 否 | "UTF-8" | 仅当file_format_type为json、text、csv、xml时使用。 | +| schema_save_mode | Enum | 否 | CREATE_SCHEMA_WHEN_NOT_EXIST | 在开启同步任务之前,对目标路径进行不同的处理 | +| data_save_mode | Enum | 否 | APPEND_DATA | 在开启同步任务之前,对目标路径中的数据文件进行不同的处理 | +| merge_update_event | boolean | 否 | false | 仅当file_format_type为canal_json、debezium_json、maxwell_json. | ### path [string] @@ -315,6 +316,12 @@ Sink插件常用参数,请参考[Sink common Options](../Sink common Options `APPEND_DATA`:使用路径,并在路径中添加新文件以写入数据。 `ERROR_WHEN_DATA_EXISTS`:当路径中存在数据文件时,将报错。 +### merge_update_event [boolean] + +仅当file_format_type为canal_json、debezium_json、maxwell_json时使用. +设置成true,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 会合并成 UPDATE; +设置成false,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 不会合并; + ## 如何创建Oss数据同步作业 diff --git a/docs/zh/connector-v2/sink/OssJindoFile.md b/docs/zh/connector-v2/sink/OssJindoFile.md index 48cc53567aa..6a9588b1451 100644 --- a/docs/zh/connector-v2/sink/OssJindoFile.md +++ b/docs/zh/connector-v2/sink/OssJindoFile.md @@ -46,40 +46,41 @@ import ChangeLog from '../changelog/connector-file-oss-jindo.md'; ## 选项 -| 名称 | 类型 | 必需 | 默认值 | 描述 | -|---------------------------------------|---------|----------|--------------------------------------------|-----------------------------------------------------| -| path | string | 是 | - | | -| tmp_path | string | 否 | /tmp/seatunnel | 结果文件将首先写入临时路径,然后使用`mv`将tmp-dir提交到目标目录。需要一个OSS 目录。 | -| bucket | string | 是 | - | | -| access_key | string | 是 | - | | -| access_secret | string | 是 | - | | -| endpoint | string | 是 | - | | -| custom_filename | boolean | 否 | false | 是否需要自定义文件名 | -| file_name_expression | string | 否 | "${transactionId}" | 仅在custom_filename为true时使用 | -| filename_time_format | string | 否 | "yyyy.MM.dd" | 仅在custom_filename为true时使用 | -| file_format_type | string | 否 | "csv" | | -| field_delimiter | string | 否 | '\001' | 仅当file_format_type为text时使用 | -| row_delimiter | string | 否 | "\n" | 仅当file_format_type为 `text`、`csv`、`json` 时使用 | -| have_partition | boolean | 否 | false | 是否需要处理分区。 | -| partition_by | array | 否 | - | 只有在have_partition为true时才使用 | -| partition_dir_expression | string | 否 | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | 只有在have_partition为true时才使用 | -| is_partition_field_write_in_file | boolean | 否 | false | 只有在have_partition为true时才使用 | -| sink_columns | array | 否 | | 当此参数为空时,所有字段都是Sink列 | -| is_enable_transaction | boolean | 否 | true | | -| batch_size | int | 否 | 1000000 | | -| compress_codec | string | 否 | none | | -| common-options | object | 否 | - | | -| max_rows_in_memory | int | 否 | - | 仅当file_format_type为excel时使用。 | -| sheet_name | string | 否 | Sheet${Random number} | 仅当file_format_type为excel时使用。 | -| csv_string_quote_mode | enum | 否 | MINIMAL | 仅在file_format为csv时使用。 | -| xml_root_tag | string | 否 | RECORDS | 仅在file_format为xml时使用。 | -| xml_row_tag | string | 否 | RECORD | 仅在file_format为xml时使用。 | -| xml_use_attr_format | boolean | 否 | - | 仅在file_format为xml时使用。 | -| single_file_mode | boolean | 否 | false | 每个并行处理只会输出一个文件。启用此参数后,batch_size将不会生效。输出文件名没有文件块后缀。 | -| create_empty_file_when_no_data | boolean | 否 | false | 当上游没有数据同步时,仍然会生成相应的数据文件。 | -| parquet_avro_write_timestamp_as_int96 | boolean | 否 | false | 仅在file_format为parquet时使用。 | -| parquet_avro_write_fixed_as_int96 | array | 否 | - | 仅在file_format为parquet时使用。 | -| encoding | string | 否 | "UTF-8" | 仅当file_format_type为json、text、csv、xml时使用。 | +| 名称 | 类型 | 必需 | 默认值 | 描述 | +|---------------------------------------|---------|----|--------------------------------------------|-----------------------------------------------------------| +| path | string | 是 | - | | +| tmp_path | string | 否 | /tmp/seatunnel | 结果文件将首先写入临时路径,然后使用`mv`将tmp-dir提交到目标目录。需要一个OSS 目录。 | +| bucket | string | 是 | - | | +| access_key | string | 是 | - | | +| access_secret | string | 是 | - | | +| endpoint | string | 是 | - | | +| custom_filename | boolean | 否 | false | 是否需要自定义文件名 | +| file_name_expression | string | 否 | "${transactionId}" | 仅在custom_filename为true时使用 | +| filename_time_format | string | 否 | "yyyy.MM.dd" | 仅在custom_filename为true时使用 | +| file_format_type | string | 否 | "csv" | | +| field_delimiter | string | 否 | '\001' | 仅当file_format_type为text时使用 | +| row_delimiter | string | 否 | "\n" | 仅当file_format_type为 `text`、`csv`、`json` 时使用 | +| have_partition | boolean | 否 | false | 是否需要处理分区。 | +| partition_by | array | 否 | - | 只有在have_partition为true时才使用 | +| partition_dir_expression | string | 否 | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | 只有在have_partition为true时才使用 | +| is_partition_field_write_in_file | boolean | 否 | false | 只有在have_partition为true时才使用 | +| sink_columns | array | 否 | | 当此参数为空时,所有字段都是Sink列 | +| is_enable_transaction | boolean | 否 | true | | +| batch_size | int | 否 | 1000000 | | +| compress_codec | string | 否 | none | | +| common-options | object | 否 | - | | +| max_rows_in_memory | int | 否 | - | 仅当file_format_type为excel时使用。 | +| sheet_name | string | 否 | Sheet${Random number} | 仅当file_format_type为excel时使用。 | +| csv_string_quote_mode | enum | 否 | MINIMAL | 仅在file_format为csv时使用。 | +| xml_root_tag | string | 否 | RECORDS | 仅在file_format为xml时使用。 | +| xml_row_tag | string | 否 | RECORD | 仅在file_format为xml时使用。 | +| xml_use_attr_format | boolean | 否 | - | 仅在file_format为xml时使用。 | +| single_file_mode | boolean | 否 | false | 每个并行处理只会输出一个文件。启用此参数后,batch_size将不会生效。输出文件名没有文件块后缀。 | +| create_empty_file_when_no_data | boolean | 否 | false | 当上游没有数据同步时,仍然会生成相应的数据文件。 | +| parquet_avro_write_timestamp_as_int96 | boolean | 否 | false | 仅在file_format为parquet时使用。 | +| parquet_avro_write_fixed_as_int96 | array | 否 | - | 仅在file_format为parquet时使用。 | +| encoding | string | 否 | "UTF-8" | 仅当file_format_type为json、text、csv、xml时使用。 | +| merge_update_event | boolean | 否 | false | 仅当file_format_type为canal_json、debezium_json、maxwell_json. | ### path [string] @@ -245,6 +246,13 @@ Sink插件常用参数,请参考[Sink common Options](../sink-common-options 仅当file_format_type为json、text、csv、xml时使用。 要写入的文件的编码。此参数将由`Charset.forName(encoding)`解析。 + +### merge_update_event [boolean] + +仅当file_format_type为canal_json、debezium_json、maxwell_json时使用. +设置成true,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 会合并成 UPDATE; +设置成false,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 不会合并; + ## 例子 适用于具有“have_partition”、“custom_filename”和“sink_columns”的文本文件格式 diff --git a/docs/zh/connector-v2/sink/S3File.md b/docs/zh/connector-v2/sink/S3File.md index e38e09f7d9b..f663a1e2cfc 100644 --- a/docs/zh/connector-v2/sink/S3File.md +++ b/docs/zh/connector-v2/sink/S3File.md @@ -102,45 +102,46 @@ import ChangeLog from '../changelog/connector-file-s3.md'; ## Sink 选项 -| 名称 | 类型 | 是否必填 | 默认值 | 描述 | -|---------------------------------------|---------|----------|-------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------| -| path | string | 是 | - | | -| tmp_path | string | 否 | /tmp/seatunnel | 结果文件将首先写入临时路径,然后使用 `mv` 将临时目录提交到目标目录。需要一个 S3 目录。 | -| bucket | string | 是 | - | | -| fs.s3a.endpoint | string | 是 | - | | -| fs.s3a.aws.credentials.provider | string | 是 | com.amazonaws.auth.InstanceProfileCredentialsProvider | 认证 s3a 的方式。目前仅支持 `org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider` 和 `com.amazonaws.auth.InstanceProfileCredentialsProvider`。 | -| access_key | string | 否 | - | 仅当 fs.s3a.aws.credentials.provider = org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider 时使用 | -| access_secret | string | 否 | - | 仅当 fs.s3a.aws.credentials.provider = org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider 时使用 | -| custom_filename | boolean | 否 | false | 是否需要自定义文件名 | -| file_name_expression | string | 否 | "${transactionId}" | 仅当 custom_filename 为 true 时使用 | -| filename_time_format | string | 否 | "yyyy.MM.dd" | 仅当 custom_filename 为 true 时使用 | -| file_format_type | string | 否 | "csv" | | -| field_delimiter | string | 否 | '\001' | 仅当 file_format 为 text 时使用 | -| row_delimiter | string | 否 | "\n" | 仅当 file_format 为 `text`、`csv`、`json` 时使用 | -| have_partition | boolean | 否 | false | 是否需要处理分区。 | -| partition_by | array | 否 | - | 仅当 have_partition 为 true 时使用 | -| partition_dir_expression | string | 否 | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | 仅当 have_partition 为 true 时使用 | -| is_partition_field_write_in_file | boolean | 否 | false | 仅当 have_partition 为 true 时使用 | -| sink_columns | array | 否 | | 当此参数为空时,所有字段均为 sink 列 | -| is_enable_transaction | boolean | 否 | true | | -| batch_size | int | 否 | 1000000 | | -| compress_codec | string | 否 | none | | -| common-options | object | 否 | - | | -| max_rows_in_memory | int | 否 | - | 仅当 file_format 为 excel 时使用 | -| sheet_name | string | 否 | Sheet${Random number} | 仅当 file_format 为 excel 时使用 | -| csv_string_quote_mode | enum | 否 | MINIMAL | 仅当 file_format 为 csv 时使用 | -| xml_root_tag | string | 否 | RECORDS | 仅当 file_format 为 xml 时使用,指定 XML 文件中根元素的标签名称。 | -| xml_row_tag | string | 否 | RECORD | 仅当 file_format 为 xml 时使用,指定 XML 文件中数据行的标签名称。 | -| xml_use_attr_format | boolean | 否 | - | 仅当 file_format 为 xml 时使用,指定是否使用标签属性格式处理数据。 | -| single_file_mode | boolean | 否 | false | 每个并行度只会输出一个文件。当此参数开启时,batch_size 将不会生效。输出文件名不会有文件块后缀。 | -| create_empty_file_when_no_data | boolean | 否 | false | 当上游没有数据同步时,仍然会生成相应的数据文件。 | -| parquet_avro_write_timestamp_as_int96 | boolean | 否 | false | 仅当 file_format 为 parquet 时使用 | -| parquet_avro_write_fixed_as_int96 | array | 否 | - | 仅当 file_format 为 parquet 时使用 | -| hadoop_s3_properties | map | 否 | | 如果您需要添加其他选项,可以在此处添加,并参考此[链接](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html) | -| schema_save_mode | Enum | 否 | CREATE_SCHEMA_WHEN_NOT_EXIST | 在开启同步任务之前,对目标路径进行不同的处理 | -| data_save_mode | Enum | 否 | APPEND_DATA | 在开启同步任务之前,对目标路径中的数据文件进行不同的处理 | -| enable_header_write | boolean | 否 | false | 仅当 file_format_type 为 text,csv 时使用。
false: 不写入表头, true: 写入表头。 | -| encoding | string | 否 | "UTF-8" | 仅当 file_format_type 为 json,text,csv,xml 时使用。 | +| 名称 | 类型 | 是否必填 | 默认值 | 描述 | +|---------------------------------------|---------|------|-------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------| +| path | string | 是 | - | | +| tmp_path | string | 否 | /tmp/seatunnel | 结果文件将首先写入临时路径,然后使用 `mv` 将临时目录提交到目标目录。需要一个 S3 目录。 | +| bucket | string | 是 | - | | +| fs.s3a.endpoint | string | 是 | - | | +| fs.s3a.aws.credentials.provider | string | 是 | com.amazonaws.auth.InstanceProfileCredentialsProvider | 认证 s3a 的方式。目前仅支持 `org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider` 和 `com.amazonaws.auth.InstanceProfileCredentialsProvider`。 | +| access_key | string | 否 | - | 仅当 fs.s3a.aws.credentials.provider = org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider 时使用 | +| access_secret | string | 否 | - | 仅当 fs.s3a.aws.credentials.provider = org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider 时使用 | +| custom_filename | boolean | 否 | false | 是否需要自定义文件名 | +| file_name_expression | string | 否 | "${transactionId}" | 仅当 custom_filename 为 true 时使用 | +| filename_time_format | string | 否 | "yyyy.MM.dd" | 仅当 custom_filename 为 true 时使用 | +| file_format_type | string | 否 | "csv" | | +| field_delimiter | string | 否 | '\001' | 仅当 file_format 为 text 时使用 | +| row_delimiter | string | 否 | "\n" | 仅当 file_format 为 `text`、`csv`、`json` 时使用 | +| have_partition | boolean | 否 | false | 是否需要处理分区。 | +| partition_by | array | 否 | - | 仅当 have_partition 为 true 时使用 | +| partition_dir_expression | string | 否 | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | 仅当 have_partition 为 true 时使用 | +| is_partition_field_write_in_file | boolean | 否 | false | 仅当 have_partition 为 true 时使用 | +| sink_columns | array | 否 | | 当此参数为空时,所有字段均为 sink 列 | +| is_enable_transaction | boolean | 否 | true | | +| batch_size | int | 否 | 1000000 | | +| compress_codec | string | 否 | none | | +| common-options | object | 否 | - | | +| max_rows_in_memory | int | 否 | - | 仅当 file_format 为 excel 时使用 | +| sheet_name | string | 否 | Sheet${Random number} | 仅当 file_format 为 excel 时使用 | +| csv_string_quote_mode | enum | 否 | MINIMAL | 仅当 file_format 为 csv 时使用 | +| xml_root_tag | string | 否 | RECORDS | 仅当 file_format 为 xml 时使用,指定 XML 文件中根元素的标签名称。 | +| xml_row_tag | string | 否 | RECORD | 仅当 file_format 为 xml 时使用,指定 XML 文件中数据行的标签名称。 | +| xml_use_attr_format | boolean | 否 | - | 仅当 file_format 为 xml 时使用,指定是否使用标签属性格式处理数据。 | +| single_file_mode | boolean | 否 | false | 每个并行度只会输出一个文件。当此参数开启时,batch_size 将不会生效。输出文件名不会有文件块后缀。 | +| create_empty_file_when_no_data | boolean | 否 | false | 当上游没有数据同步时,仍然会生成相应的数据文件。 | +| parquet_avro_write_timestamp_as_int96 | boolean | 否 | false | 仅当 file_format 为 parquet 时使用 | +| parquet_avro_write_fixed_as_int96 | array | 否 | - | 仅当 file_format 为 parquet 时使用 | +| hadoop_s3_properties | map | 否 | | 如果您需要添加其他选项,可以在此处添加,并参考此[链接](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html) | +| schema_save_mode | Enum | 否 | CREATE_SCHEMA_WHEN_NOT_EXIST | 在开启同步任务之前,对目标路径进行不同的处理 | +| data_save_mode | Enum | 否 | APPEND_DATA | 在开启同步任务之前,对目标路径中的数据文件进行不同的处理 | +| enable_header_write | boolean | 否 | false | 仅当 file_format_type 为 text,csv 时使用。
false: 不写入表头, true: 写入表头。 | +| encoding | string | 否 | "UTF-8" | 仅当 file_format_type 为 json,text,csv,xml 时使用。 | +| merge_update_event | boolean | 否 | false | 仅当file_format_type为canal_json、debezium_json、maxwell_json. | ### path [string] @@ -318,6 +319,13 @@ Sink 插件通用参数,请参考 [Sink 通用选项](../sink-common-options.m 仅当 file_format_type 为 json,text,csv,xml 时使用。 写入文件的编码。此参数将由 `Charset.forName(encoding)` 解析。 + +### merge_update_event [boolean] + +仅当file_format_type为canal_json、debezium_json、maxwell_json时使用. +设置成true,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 会合并成 UPDATE; +设置成false,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 不会合并; + ## 示例 ### 简单示例 diff --git a/docs/zh/connector-v2/sink/SftpFile.md b/docs/zh/connector-v2/sink/SftpFile.md index 880d2cef075..ebf2874c4de 100644 --- a/docs/zh/connector-v2/sink/SftpFile.md +++ b/docs/zh/connector-v2/sink/SftpFile.md @@ -40,43 +40,44 @@ import ChangeLog from '../changelog/connector-file-sftp.md'; ## 参数 -| 名称 | 类型 | 是否必填 | 默认值 | 备注 | -|---------------------------------------|---------|----------|--------------------------------------------|---------------------------------------------------------| -| host | string | 是 | - | | -| port | int | 是 | - | | -| user | string | 是 | - | | -| password | string | 是 | - | | -| path | string | 是 | - | | -| tmp_path | string | 是 | /tmp/seatunnel | 结果文件将首先写入临时路径,然后使用`mv`将临时目录剪切到目标目录。需要一个FTP目录。 | -| custom_filename | boolean | 否 | false | 是否需要自定义文件名 | -| file_name_expression | string | 否 | "${transactionId}" | 仅在custom_filename为true时使用 | -| filename_time_format | string | 否 | "yyyy.MM.dd" | 仅在custom_filename为true时使用 | -| file_format_type | string | 否 | "csv" | | -| field_delimiter | string | 否 | '\001' | 仅当file_format_type为text时使用 | -| row_delimiter | string | 否 | "\n" | 仅当file_format_type为 `text`、`csv`、`json` 时使用 | -| have_partition | boolean | 否 | false | 是否需要处理分区。 | -| partition_by | array | 否 | - | 只有在have_partition为true时才使用 | -| partition_dir_expression | string | 否 | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | 只有在have_partition为true时才使用 | -| is_partition_field_write_in_file | boolean | 否 | false | 只有在have_partition为true时才使用 | -| sink_columns | array | 否 | | 当此参数为空时,所有字段都是sink列 | -| is_enable_transaction | boolean | 否 | true | | -| batch_size | int | 否 | 1000000 | | -| compress_codec | string | 否 | none | | -| common-options | object | 否 | - | | -| max_rows_in_memory | int | 否 | - | 仅当file_format_type为excel时使用。 | -| sheet_name | string | 否 | Sheet${Random number} | 仅当file_format_type为excel时使用。 | -| csv_string_quote_mode | enum | 否 | MINIMAL | 仅当file_format_type为csv时使用。 | -| xml_root_tag | string | 否 | RECORDS | 仅当file_format_type为xml时使用 | -| xml_row_tag | string | 否 | RECORD | 仅当file_format_type为xml时使用 | -| xml_use_attr_format | boolean | 否 | - | 仅当file_format_type为xml时使用 | -| single_file_mode | boolean | 否 | false | 每个并行处理只会输出一个文件。启用此参数后,batch_size将不会生效。输出文件名没有文件块后缀。 | -| create_empty_file_when_no_data | boolean | 否 | false | 当上游没有数据同步时,仍然会生成相应的数据文件。 | -| parquet_avro_write_timestamp_as_int96 | boolean | 否 | false | 仅当file_format_type为parquet时使用 | -| enable_header_write | boolean | 否 | false | 仅当file_format_type为text、csv时使用
false:不写标头,true:写标头。 | -| parquet_avro_write_fixed_as_int96 | array | 否 | - | 仅当file_format_type为parquet时使用 | -| encoding | string | 否 | "UTF-8" | 仅当file_format_type为json、text、csv、xml时使用。 | -| schema_save_mode | string | 否 | CREATE_SCHEMA_WHEN_NOT_EXIST | 现有目录处理方式 | -| data_save_mode | string | 否 | APPEND_DATA | 现有数据处理方式 | +| 名称 | 类型 | 是否必填 | 默认值 | 备注 | +|---------------------------------------|---------|------|--------------------------------------------|-----------------------------------------------------------| +| host | string | 是 | - | | +| port | int | 是 | - | | +| user | string | 是 | - | | +| password | string | 是 | - | | +| path | string | 是 | - | | +| tmp_path | string | 是 | /tmp/seatunnel | 结果文件将首先写入临时路径,然后使用`mv`将临时目录剪切到目标目录。需要一个FTP目录。 | +| custom_filename | boolean | 否 | false | 是否需要自定义文件名 | +| file_name_expression | string | 否 | "${transactionId}" | 仅在custom_filename为true时使用 | +| filename_time_format | string | 否 | "yyyy.MM.dd" | 仅在custom_filename为true时使用 | +| file_format_type | string | 否 | "csv" | | +| field_delimiter | string | 否 | '\001' | 仅当file_format_type为text时使用 | +| row_delimiter | string | 否 | "\n" | 仅当file_format_type为 `text`、`csv`、`json` 时使用 | +| have_partition | boolean | 否 | false | 是否需要处理分区。 | +| partition_by | array | 否 | - | 只有在have_partition为true时才使用 | +| partition_dir_expression | string | 否 | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | 只有在have_partition为true时才使用 | +| is_partition_field_write_in_file | boolean | 否 | false | 只有在have_partition为true时才使用 | +| sink_columns | array | 否 | | 当此参数为空时,所有字段都是sink列 | +| is_enable_transaction | boolean | 否 | true | | +| batch_size | int | 否 | 1000000 | | +| compress_codec | string | 否 | none | | +| common-options | object | 否 | - | | +| max_rows_in_memory | int | 否 | - | 仅当file_format_type为excel时使用。 | +| sheet_name | string | 否 | Sheet${Random number} | 仅当file_format_type为excel时使用。 | +| csv_string_quote_mode | enum | 否 | MINIMAL | 仅当file_format_type为csv时使用。 | +| xml_root_tag | string | 否 | RECORDS | 仅当file_format_type为xml时使用 | +| xml_row_tag | string | 否 | RECORD | 仅当file_format_type为xml时使用 | +| xml_use_attr_format | boolean | 否 | - | 仅当file_format_type为xml时使用 | +| single_file_mode | boolean | 否 | false | 每个并行处理只会输出一个文件。启用此参数后,batch_size将不会生效。输出文件名没有文件块后缀。 | +| create_empty_file_when_no_data | boolean | 否 | false | 当上游没有数据同步时,仍然会生成相应的数据文件。 | +| parquet_avro_write_timestamp_as_int96 | boolean | 否 | false | 仅当file_format_type为parquet时使用 | +| enable_header_write | boolean | 否 | false | 仅当file_format_type为text、csv时使用
false:不写标头,true:写标头。 | +| parquet_avro_write_fixed_as_int96 | array | 否 | - | 仅当file_format_type为parquet时使用 | +| encoding | string | 否 | "UTF-8" | 仅当file_format_type为json、text、csv、xml时使用。 | +| schema_save_mode | string | 否 | CREATE_SCHEMA_WHEN_NOT_EXIST | 现有目录处理方式 | +| data_save_mode | string | 否 | APPEND_DATA | 现有数据处理方式 | +| merge_update_event | boolean | 否 | false | 仅当file_format_type为canal_json、debezium_json、maxwell_json. | ### host [string] @@ -261,6 +262,13 @@ Sink插件常用参数,请参考[Sink common Options](../sink-common-options -APPEND_DATA:保留目录,保留数据文件 -ERROR_WHEN_DATA_EXISTS:当有数据文件时,会报告错误 + +### merge_update_event [boolean] + +仅当file_format_type为canal_json、debezium_json、maxwell_json时使用. +设置成true,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 会合并成 UPDATE; +设置成false,序列化数据时,UPDATE_AFTER 和 UPDATE_BEFORE 不会合并; + ## 示例 对于具有`have_partition`、`custom_filename`和`sink_columns`的文本文件格式 diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileBaseSinkOptions.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileBaseSinkOptions.java index ce8d7d531c2..462d4671275 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileBaseSinkOptions.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileBaseSinkOptions.java @@ -329,4 +329,11 @@ public class FileBaseSinkOptions extends FileBaseOptions { .stringType() .noDefaultValue() .withDescription("When using kerberos, We should specify the keytab path"); + + public static final Option MERGE_UPDATE_EVENT = + Options.key("merge_update_event") + .booleanType() + .defaultValue(false) + .withDescription( + "Only used when file_format_type is canal_json,debezium_json,maxwell_json. set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data;if set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event "); } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/FileSinkConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/FileSinkConfig.java index 9ebb359743a..7b0d2df92f6 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/FileSinkConfig.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/FileSinkConfig.java @@ -90,6 +90,8 @@ public class FileSinkConfig extends BaseFileSinkConfig implements PartitionConfi private CsvStringQuoteMode csvStringQuoteMode = FileBaseSinkOptions.CSV_STRING_QUOTE_MODE.defaultValue(); + private Boolean mergeUpdateEvent = FileBaseSinkOptions.MERGE_UPDATE_EVENT.defaultValue(); + public FileSinkConfig(@NonNull Config config, @NonNull SeaTunnelRowType seaTunnelRowTypeInfo) { super(config); checkArgument( @@ -249,5 +251,13 @@ public FileSinkConfig(@NonNull Config config, @NonNull SeaTunnelRowType seaTunne config.getString(FileBaseSinkOptions.CSV_STRING_QUOTE_MODE.key())); } } + if (FileFormat.DEBEZIUM_JSON.equals(this.fileFormat) + || FileFormat.CANAL_JSON.equals(this.fileFormat) + || FileFormat.MAXWELL_JSON.equals(this.fileFormat)) { + if (config.hasPath(FileBaseSinkOptions.MERGE_UPDATE_EVENT.key())) { + this.mergeUpdateEvent = + config.getBoolean(FileBaseSinkOptions.MERGE_UPDATE_EVENT.key()); + } + } } } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/CanalJsonWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/CanalJsonWriteStrategy.java index d4eef0858f1..9a5d8ef07bd 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/CanalJsonWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/CanalJsonWriteStrategy.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.common.utils.EncodingUtils; import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; import org.apache.seatunnel.connectors.seatunnel.file.sink.config.FileSinkConfig; +import org.apache.seatunnel.format.json.canal.CanalJsonFormatOptions; import org.apache.seatunnel.format.json.canal.CanalJsonSerializationSchema; import org.apache.hadoop.fs.FSDataOutputStream; @@ -45,6 +46,7 @@ public class CanalJsonWriteStrategy extends AbstractWriteStrategy beingWrittenOutputStream; private final Map isFirstWrite; private final Charset charset; + private final Map options; public CanalJsonWriteStrategy(FileSinkConfig textFileSinkConfig) { super(textFileSinkConfig); @@ -52,6 +54,10 @@ public CanalJsonWriteStrategy(FileSinkConfig textFileSinkConfig) { this.isFirstWrite = new HashMap<>(); this.charset = EncodingUtils.tryParseCharset(textFileSinkConfig.getEncoding()); this.rowDelimiter = textFileSinkConfig.getRowDelimiter().getBytes(charset); + this.options = new HashMap<>(); + this.options.put( + CanalJsonFormatOptions.MERGE_UPDATE_EVENT.key(), + textFileSinkConfig.getMergeUpdateEvent().toString()); } @Override @@ -61,7 +67,8 @@ public void setCatalogTable(CatalogTable catalogTable) { new CanalJsonSerializationSchema( buildSchemaWithRowType( catalogTable.getSeaTunnelRowType(), sinkColumnsIndexInRow), - charset); + charset, + options); } @Override @@ -76,6 +83,9 @@ public void write(@NonNull SeaTunnelRow seaTunnelRow) { sinkColumnsIndexInRow.stream() .mapToInt(Integer::intValue) .toArray())); + if (rowBytes == null) { + return; + } if (isFirstWrite.get(filePath)) { isFirstWrite.put(filePath, false); } else { diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/DebeziumJsonWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/DebeziumJsonWriteStrategy.java index 548451f2497..cdeb43872f8 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/DebeziumJsonWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/DebeziumJsonWriteStrategy.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.common.utils.EncodingUtils; import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; import org.apache.seatunnel.connectors.seatunnel.file.sink.config.FileSinkConfig; +import org.apache.seatunnel.format.json.debezium.DebeziumJsonFormatOptions; import org.apache.seatunnel.format.json.debezium.DebeziumJsonSerializationSchema; import org.apache.hadoop.fs.FSDataOutputStream; @@ -45,6 +46,7 @@ public class DebeziumJsonWriteStrategy extends AbstractWriteStrategy beingWrittenOutputStream; private final Map isFirstWrite; private final Charset charset; + private final Map options; public DebeziumJsonWriteStrategy(FileSinkConfig textFileSinkConfig) { super(textFileSinkConfig); @@ -52,6 +54,10 @@ public DebeziumJsonWriteStrategy(FileSinkConfig textFileSinkConfig) { this.isFirstWrite = new HashMap<>(); this.charset = EncodingUtils.tryParseCharset(textFileSinkConfig.getEncoding()); this.rowDelimiter = textFileSinkConfig.getRowDelimiter().getBytes(charset); + this.options = new HashMap<>(); + this.options.put( + DebeziumJsonFormatOptions.MERGE_UPDATE_EVENT.key(), + textFileSinkConfig.getMergeUpdateEvent().toString()); } @Override @@ -61,7 +67,8 @@ public void setCatalogTable(CatalogTable catalogTable) { new DebeziumJsonSerializationSchema( buildSchemaWithRowType( catalogTable.getSeaTunnelRowType(), sinkColumnsIndexInRow), - charset); + charset, + options); } @Override @@ -76,6 +83,9 @@ public void write(@NonNull SeaTunnelRow seaTunnelRow) { sinkColumnsIndexInRow.stream() .mapToInt(Integer::intValue) .toArray())); + if (rowBytes == null) { + return; + } if (isFirstWrite.get(filePath)) { isFirstWrite.put(filePath, false); } else { diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/MaxWellJsonWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/MaxWellJsonWriteStrategy.java index 0d610183bd9..3d185144e53 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/MaxWellJsonWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/MaxWellJsonWriteStrategy.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.common.utils.EncodingUtils; import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; import org.apache.seatunnel.connectors.seatunnel.file.sink.config.FileSinkConfig; +import org.apache.seatunnel.format.json.maxwell.MaxWellJsonFormatOptions; import org.apache.seatunnel.format.json.maxwell.MaxWellJsonSerializationSchema; import org.apache.hadoop.fs.FSDataOutputStream; @@ -45,6 +46,7 @@ public class MaxWellJsonWriteStrategy extends AbstractWriteStrategy beingWrittenOutputStream; private final Map isFirstWrite; private final Charset charset; + private final Map options; public MaxWellJsonWriteStrategy(FileSinkConfig textFileSinkConfig) { super(textFileSinkConfig); @@ -52,6 +54,10 @@ public MaxWellJsonWriteStrategy(FileSinkConfig textFileSinkConfig) { this.isFirstWrite = new HashMap<>(); this.charset = EncodingUtils.tryParseCharset(textFileSinkConfig.getEncoding()); this.rowDelimiter = textFileSinkConfig.getRowDelimiter().getBytes(charset); + this.options = new HashMap<>(); + this.options.put( + MaxWellJsonFormatOptions.MERGE_UPDATE_EVENT.key(), + textFileSinkConfig.getMergeUpdateEvent().toString()); } @Override @@ -61,7 +67,8 @@ public void setCatalogTable(CatalogTable catalogTable) { new MaxWellJsonSerializationSchema( buildSchemaWithRowType( catalogTable.getSeaTunnelRowType(), sinkColumnsIndexInRow), - charset); + charset, + options); } @Override @@ -76,6 +83,9 @@ public void write(@NonNull SeaTunnelRow seaTunnelRow) { sinkColumnsIndexInRow.stream() .mapToInt(Integer::intValue) .toArray())); + if (rowBytes == null) { + return; + } if (isFirstWrite.get(filePath)) { isFirstWrite.put(filePath, false); } else { diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonFormatOptions.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonFormatOptions.java index 3af85af1d06..09d946bb61e 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonFormatOptions.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonFormatOptions.java @@ -45,6 +45,14 @@ public class CanalJsonFormatOptions { "An optional regular expression to only read the specific tables changelog rows by regular matching the \"table\" meta field in the Canal record." + "The pattern string is compatible with Java's Pattern."); + public static final Option MERGE_UPDATE_EVENT = + Options.key("merge_update_event") + .booleanType() + .defaultValue(false) + .withDescription( + "set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data;" + + "if set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event "); + public static String getTableInclude(Map options) { return options.getOrDefault(TABLE_INCLUDE.key(), null); } @@ -57,4 +65,10 @@ public static boolean getIgnoreParseErrors(Map options) { return Boolean.parseBoolean( options.getOrDefault(IGNORE_PARSE_ERRORS.key(), IGNORE_PARSE_ERRORS.toString())); } + + public static boolean getMergeUpdateEvent(Map options) { + return Boolean.parseBoolean( + options.getOrDefault( + MERGE_UPDATE_EVENT.key(), MERGE_UPDATE_EVENT.defaultValue().toString())); + } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java index 03575d052dc..4744b14fc4b 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java @@ -33,6 +33,7 @@ import org.apache.commons.lang3.StringUtils; import java.nio.charset.Charset; +import java.util.Map; import static org.apache.seatunnel.api.table.type.BasicType.LONG_TYPE; import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; @@ -46,34 +47,52 @@ public class CanalJsonSerializationSchema implements SerializationSchema { private static final String OP_INSERT = "INSERT"; private static final String OP_DELETE = "DELETE"; + private static final String OP_UPDATE = "UPDATE"; private transient SeaTunnelRow reuse; private final JsonSerializationSchema jsonSerializer; + boolean mergeUpdateEventFlag; + SeaTunnelRow cacheUpdateBeforeRow; + public CanalJsonSerializationSchema(SeaTunnelRowType rowType) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType)); - this.reuse = new SeaTunnelRow(5); + this.reuse = new SeaTunnelRow(6); + mergeUpdateEventFlag = CanalJsonFormatOptions.MERGE_UPDATE_EVENT.defaultValue(); } - public CanalJsonSerializationSchema(SeaTunnelRowType rowType, Charset charset) { + public CanalJsonSerializationSchema( + SeaTunnelRowType rowType, Charset charset, Map options) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType), charset); - this.reuse = new SeaTunnelRow(5); + this.reuse = new SeaTunnelRow(6); + mergeUpdateEventFlag = CanalJsonFormatOptions.getMergeUpdateEvent(options); } @Override public byte[] serialize(SeaTunnelRow row) { try { - String opType = rowKind2String(row.getRowKind()); - reuse.setField(0, new SeaTunnelRow[] {row}); - reuse.setField(1, opType); + if (mergeUpdateEventFlag && row.getRowKind() == RowKind.UPDATE_BEFORE) { + cacheUpdateBeforeRow = row; + return null; + } + + if (mergeUpdateEventFlag && row.getRowKind() == RowKind.UPDATE_AFTER) { + reuse.setField(0, cacheUpdateBeforeRow); + } else { + reuse.setField(0, null); + } + + reuse.setField(1, new SeaTunnelRow[] {row}); + reuse.setField(2, rowKind2String(row.getRowKind())); + if (!StringUtils.isEmpty(row.getTableId())) { - reuse.setField(2, TablePath.of(row.getTableId()).getDatabaseName()); - reuse.setField(3, TablePath.of(row.getTableId()).getTableName()); + reuse.setField(3, TablePath.of(row.getTableId()).getDatabaseName()); + reuse.setField(4, TablePath.of(row.getTableId()).getTableName()); } if (row.getOptions() != null && row.getOptions().containsKey(EVENT_TIME.getName())) { - reuse.setField(4, row.getOptions().get(EVENT_TIME.getName())); + reuse.setField(5, row.getOptions().get(EVENT_TIME.getName())); } return jsonSerializer.serialize(reuse); @@ -86,6 +105,9 @@ private String rowKind2String(RowKind rowKind) { switch (rowKind) { case INSERT: case UPDATE_AFTER: + if (mergeUpdateEventFlag && rowKind.equals(RowKind.UPDATE_AFTER)) { + return OP_UPDATE; + } return OP_INSERT; case UPDATE_BEFORE: case DELETE: @@ -98,11 +120,8 @@ private String rowKind2String(RowKind rowKind) { } private static SeaTunnelRowType createJsonRowType(SeaTunnelRowType databaseSchema) { - // Canal JSON contains other information, e.g. "database", "ts" - // but we don't need them - // and we don't need "old" , because can not support UPDATE_BEFORE,UPDATE_AFTER return new SeaTunnelRowType( - new String[] {"data", "type", "database", "table", "ts"}, + new String[] {"old", "data", "type", "database", "table", "ts"}, new SeaTunnelDataType[] { new ArrayType<>(SeaTunnelRowType[].class, databaseSchema), STRING_TYPE, diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatOptions.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatOptions.java index 43cab50c9d4..61a5edac1bc 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatOptions.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatOptions.java @@ -39,6 +39,14 @@ public class DebeziumJsonFormatOptions { + "This option indicates the Debezium JSON data include the schema in the message or not. " + "Default is false."); + public static final Option MERGE_UPDATE_EVENT = + Options.key("merge_update_event") + .booleanType() + .defaultValue(false) + .withDescription( + "set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data;" + + "if set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event "); + public static boolean getSchemaInclude(Map options) { return Boolean.parseBoolean( options.getOrDefault( @@ -50,4 +58,10 @@ public static boolean getIgnoreParseErrors(Map options) { options.getOrDefault( IGNORE_PARSE_ERRORS.key(), IGNORE_PARSE_ERRORS.defaultValue().toString())); } + + public static boolean getMergeUpdateEvent(Map options) { + return Boolean.parseBoolean( + options.getOrDefault( + MERGE_UPDATE_EVENT.key(), MERGE_UPDATE_EVENT.defaultValue().toString())); + } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerializationSchema.java index a768d29e3e4..35694b54eba 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerializationSchema.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -43,20 +44,27 @@ public class DebeziumJsonSerializationSchema implements SerializationSchema { private static final String OP_INSERT = "c"; // insert private static final String OP_DELETE = "d"; // delete + private static final String OP_UPDATE = "u"; // update public static final String FORMAT = "Debezium"; private final JsonSerializationSchema jsonSerializer; private transient SeaTunnelRow genericRow; + boolean mergeUpdateEventFlag; + SeaTunnelRow cacheUpdateBeforeRow; + public DebeziumJsonSerializationSchema(SeaTunnelRowType rowType) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType)); this.genericRow = new SeaTunnelRow(GENERATE_ROW_SIZE); + mergeUpdateEventFlag = DebeziumJsonFormatOptions.MERGE_UPDATE_EVENT.defaultValue(); } - public DebeziumJsonSerializationSchema(SeaTunnelRowType rowType, Charset charset) { + public DebeziumJsonSerializationSchema( + SeaTunnelRowType rowType, Charset charset, Map options) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType), charset); this.genericRow = new SeaTunnelRow(GENERATE_ROW_SIZE); + this.mergeUpdateEventFlag = DebeziumJsonFormatOptions.getMergeUpdateEvent(options); } @Override @@ -71,9 +79,14 @@ public byte[] serialize(SeaTunnelRow row) { switch (row.getRowKind()) { case INSERT: case UPDATE_AFTER: - genericRow.setField(0, null); + if (mergeUpdateEventFlag && row.getRowKind().equals(RowKind.UPDATE_AFTER)) { + genericRow.setField(0, cacheUpdateBeforeRow); + genericRow.setField(2, OP_UPDATE); + } else { + genericRow.setField(0, null); + genericRow.setField(2, OP_INSERT); + } genericRow.setField(1, row); - genericRow.setField(2, OP_INSERT); genericRow.setField(3, source); if (row.getOptions() != null @@ -84,6 +97,10 @@ public byte[] serialize(SeaTunnelRow row) { } return jsonSerializer.serialize(genericRow); case UPDATE_BEFORE: + if (mergeUpdateEventFlag) { + cacheUpdateBeforeRow = row; + return null; + } case DELETE: genericRow.setField(0, row); genericRow.setField(1, null); diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonFormatOptions.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonFormatOptions.java index 0b6a5cd2879..830437a7bf5 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonFormatOptions.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonFormatOptions.java @@ -45,6 +45,14 @@ public class MaxWellJsonFormatOptions { "An optional regular expression to only read the specific tables changelog rows by regular matching the \"table\" meta field in the MaxWell record." + "The pattern string is compatible with Java's Pattern."); + public static final Option MERGE_UPDATE_EVENT = + Options.key("merge_update_event") + .booleanType() + .defaultValue(false) + .withDescription( + "set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data;" + + "if set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event "); + public static String getTableInclude(Map options) { return options.getOrDefault(TABLE_INCLUDE.key(), null); } @@ -57,4 +65,10 @@ public static boolean getIgnoreParseErrors(Map options) { return Boolean.parseBoolean( options.getOrDefault(IGNORE_PARSE_ERRORS.key(), IGNORE_PARSE_ERRORS.toString())); } + + public static boolean getMergeUpdateEvent(Map options) { + return Boolean.parseBoolean( + options.getOrDefault( + MERGE_UPDATE_EVENT.key(), MERGE_UPDATE_EVENT.defaultValue().toString())); + } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java index 191512c49cc..f4130cf5334 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java @@ -32,6 +32,7 @@ import org.apache.commons.lang3.StringUtils; import java.nio.charset.Charset; +import java.util.Map; import static org.apache.seatunnel.api.table.type.BasicType.LONG_TYPE; import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; @@ -41,8 +42,9 @@ public class MaxWellJsonSerializationSchema implements SerializationSchema { private static final long serialVersionUID = 1L; - private static final String OP_INSERT = "INSERT"; - private static final String OP_DELETE = "DELETE"; + private static final String OP_INSERT = "insert"; + private static final String OP_DELETE = "delete"; + private static final String OP_UPDATE = "update"; public static final String FORMAT = "MAXWELL"; @@ -50,29 +52,42 @@ public class MaxWellJsonSerializationSchema implements SerializationSchema { private final JsonSerializationSchema jsonSerializer; + boolean mergeUpdateEventFlag; + SeaTunnelRow cacheUpdateBeforeRow; + public MaxWellJsonSerializationSchema(SeaTunnelRowType rowType) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType)); - this.reuse = new SeaTunnelRow(5); + this.reuse = new SeaTunnelRow(6); } - public MaxWellJsonSerializationSchema(SeaTunnelRowType rowType, Charset charset) { + public MaxWellJsonSerializationSchema( + SeaTunnelRowType rowType, Charset charset, Map options) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType), charset); - this.reuse = new SeaTunnelRow(5); + this.reuse = new SeaTunnelRow(6); } @Override public byte[] serialize(SeaTunnelRow row) { try { - String opType = rowKind2String(row.getRowKind()); - reuse.setField(0, row); - reuse.setField(1, opType); - reuse.setField(2, row.getTableId()); + if (mergeUpdateEventFlag && row.getRowKind() == RowKind.UPDATE_BEFORE) { + cacheUpdateBeforeRow = row; + return null; + } + + if (mergeUpdateEventFlag && row.getRowKind() == RowKind.UPDATE_AFTER) { + reuse.setField(0, cacheUpdateBeforeRow); + } else { + reuse.setField(0, null); + } + + reuse.setField(1, row); + reuse.setField(2, rowKind2String(row.getRowKind())); if (!StringUtils.isEmpty(row.getTableId())) { - reuse.setField(2, TablePath.of(row.getTableId()).getDatabaseName()); - reuse.setField(3, TablePath.of(row.getTableId()).getTableName()); + reuse.setField(3, TablePath.of(row.getTableId()).getDatabaseName()); + reuse.setField(4, TablePath.of(row.getTableId()).getTableName()); } if (row.getOptions() != null && row.getOptions().containsKey(EVENT_TIME.getName())) { - reuse.setField(4, row.getOptions().get(EVENT_TIME.getName())); + reuse.setField(5, row.getOptions().get(EVENT_TIME.getName())); } return jsonSerializer.serialize(reuse); } catch (Throwable t) { @@ -84,6 +99,9 @@ private String rowKind2String(RowKind rowKind) { switch (rowKind) { case INSERT: case UPDATE_AFTER: + if (mergeUpdateEventFlag && rowKind.equals(RowKind.UPDATE_AFTER)) { + return OP_UPDATE; + } return OP_INSERT; case UPDATE_BEFORE: case DELETE: @@ -96,11 +114,8 @@ private String rowKind2String(RowKind rowKind) { } private static SeaTunnelRowType createJsonRowType(SeaTunnelRowType databaseSchema) { - // MaxWell JSON contains other information, e.g. "database", "ts" - // but we don't need them - // and we don't need "old" , because can not support UPDATE_BEFORE,UPDATE_AFTER return new SeaTunnelRowType( - new String[] {"data", "type", "database", "table", "ts"}, + new String[] {"old", "data", "type", "database", "table", "ts"}, new SeaTunnelDataType[] { databaseSchema, STRING_TYPE, STRING_TYPE, STRING_TYPE, LONG_TYPE }); diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonFormatOptions.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonFormatOptions.java index 462ccf8aa65..9e62840c40a 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonFormatOptions.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonFormatOptions.java @@ -45,6 +45,14 @@ public class OggJsonFormatOptions { "An optional regular expression to only read the specific tables changelog rows by regular matching the \"table\" meta field in the Ogg record." + "The pattern string is compatible with Java's Pattern."); + public static final Option MERGE_UPDATE_EVENT = + Options.key("merge_update_event") + .booleanType() + .defaultValue(false) + .withDescription( + "set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data;" + + "if set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event "); + public static String getTableInclude(Map options) { return options.getOrDefault(TABLE_INCLUDE.key(), null); } @@ -57,4 +65,10 @@ public static boolean getIgnoreParseErrors(Map options) { return Boolean.parseBoolean( options.getOrDefault(IGNORE_PARSE_ERRORS.key(), IGNORE_PARSE_ERRORS.toString())); } + + public static boolean getMergeUpdateEvent(Map options) { + return Boolean.parseBoolean( + options.getOrDefault( + MERGE_UPDATE_EVENT.key(), MERGE_UPDATE_EVENT.defaultValue().toString())); + } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java index 05218b6d434..ef3bd2bc3d5 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java @@ -30,6 +30,9 @@ import org.apache.commons.lang3.StringUtils; +import java.nio.charset.Charset; +import java.util.Map; + import static org.apache.seatunnel.api.table.type.BasicType.LONG_TYPE; import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; import static org.apache.seatunnel.api.table.type.CommonOptions.EVENT_TIME; @@ -38,31 +41,53 @@ public class OggJsonSerializationSchema implements SerializationSchema { private static final long serialVersionUID = 1L; - private static final String OP_INSERT = "INSERT"; - private static final String OP_DELETE = "DELETE"; + private static final String OP_INSERT = "I"; + private static final String OP_DELETE = "D"; + private static final String OP_UPDATE = "U"; public static final String FORMAT = "Ogg"; private transient SeaTunnelRow reuse; private final JsonSerializationSchema jsonSerializer; + boolean mergeUpdateEventFlag; + SeaTunnelRow cacheUpdateBeforeRow; + public OggJsonSerializationSchema(SeaTunnelRowType rowType) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType)); - this.reuse = new SeaTunnelRow(4); + this.reuse = new SeaTunnelRow(5); + mergeUpdateEventFlag = OggJsonFormatOptions.MERGE_UPDATE_EVENT.defaultValue(); + } + + public OggJsonSerializationSchema( + SeaTunnelRowType rowType, Charset charset, Map options) { + this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType), charset); + this.reuse = new SeaTunnelRow(5); + mergeUpdateEventFlag = OggJsonFormatOptions.getMergeUpdateEvent(options); } @Override public byte[] serialize(SeaTunnelRow row) { try { - String opType = rowKind2String(row.getRowKind()); - reuse.setField(0, row); - reuse.setField(1, opType); + if (mergeUpdateEventFlag && row.getRowKind() == RowKind.UPDATE_BEFORE) { + cacheUpdateBeforeRow = row; + return null; + } + + if (mergeUpdateEventFlag && row.getRowKind() == RowKind.UPDATE_AFTER) { + reuse.setField(0, cacheUpdateBeforeRow); + } else { + reuse.setField(0, null); + } + + reuse.setField(1, row); + reuse.setField(2, rowKind2String(row.getRowKind())); if (!StringUtils.isEmpty(row.getTableId())) { - reuse.setField(2, row.getTableId()); + reuse.setField(3, row.getTableId()); } if (row.getOptions() != null && row.getOptions().containsKey(EVENT_TIME.getName())) { - reuse.setField(3, row.getOptions().get(EVENT_TIME.getName())); + reuse.setField(4, row.getOptions().get(EVENT_TIME.getName())); } return jsonSerializer.serialize(reuse); } catch (Throwable t) { @@ -74,6 +99,9 @@ private String rowKind2String(RowKind rowKind) { switch (rowKind) { case INSERT: case UPDATE_AFTER: + if (mergeUpdateEventFlag && rowKind.equals(RowKind.UPDATE_AFTER)) { + return OP_UPDATE; + } return OP_INSERT; case UPDATE_BEFORE: case DELETE: @@ -86,11 +114,8 @@ private String rowKind2String(RowKind rowKind) { } private static SeaTunnelRowType createJsonRowType(SeaTunnelRowType databaseSchema) { - // Ogg JSON contains other information, e.g. "database", "ts" - // but we don't need them - // and we don't need "old" , because can not support UPDATE_BEFORE,UPDATE_AFTER return new SeaTunnelRowType( - new String[] {"data", "type", "table", "op_ts"}, + new String[] {"before", "after", "op_type", "table", "op_ts"}, new SeaTunnelDataType[] {databaseSchema, STRING_TYPE, STRING_TYPE, LONG_TYPE}); } } From 4157583248f67c43c2ed1a6dcfd4071db3296c0c Mon Sep 17 00:00:00 2001 From: dyp12 Date: Fri, 29 Aug 2025 18:08:48 +0800 Subject: [PATCH 12/37] init --- .../format/json/canal/CanalJsonSerializationSchema.java | 1 + .../format/json/maxwell/MaxWellJsonSerializationSchema.java | 2 +- .../seatunnel/format/json/ogg/OggJsonSerializationSchema.java | 4 +++- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java index 4744b14fc4b..7cd627d0403 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java @@ -123,6 +123,7 @@ private static SeaTunnelRowType createJsonRowType(SeaTunnelRowType databaseSchem return new SeaTunnelRowType( new String[] {"old", "data", "type", "database", "table", "ts"}, new SeaTunnelDataType[] { + new ArrayType<>(SeaTunnelRowType[].class, databaseSchema), new ArrayType<>(SeaTunnelRowType[].class, databaseSchema), STRING_TYPE, STRING_TYPE, diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java index f4130cf5334..93e03ef363b 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java @@ -117,7 +117,7 @@ private static SeaTunnelRowType createJsonRowType(SeaTunnelRowType databaseSchem return new SeaTunnelRowType( new String[] {"old", "data", "type", "database", "table", "ts"}, new SeaTunnelDataType[] { - databaseSchema, STRING_TYPE, STRING_TYPE, STRING_TYPE, LONG_TYPE + databaseSchema, databaseSchema, STRING_TYPE, STRING_TYPE, STRING_TYPE, LONG_TYPE }); } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java index ef3bd2bc3d5..e69231f8474 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java @@ -116,6 +116,8 @@ private String rowKind2String(RowKind rowKind) { private static SeaTunnelRowType createJsonRowType(SeaTunnelRowType databaseSchema) { return new SeaTunnelRowType( new String[] {"before", "after", "op_type", "table", "op_ts"}, - new SeaTunnelDataType[] {databaseSchema, STRING_TYPE, STRING_TYPE, LONG_TYPE}); + new SeaTunnelDataType[] { + databaseSchema, databaseSchema, STRING_TYPE, STRING_TYPE, LONG_TYPE + }); } } From f098cac250c476774fa9c08063ec4e426c2b6131 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Fri, 29 Aug 2025 18:28:38 +0800 Subject: [PATCH 13/37] init --- .../seatunnel/file/local/LocalFileTest.java | 121 ++++++++++++++++-- .../canal/CanalJsonSerializationSchema.java | 2 +- .../MaxWellJsonSerializationSchema.java | 2 + 3 files changed, 112 insertions(+), 13 deletions(-) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/local/LocalFileTest.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/local/LocalFileTest.java index 089c32cad04..41fd6fa20b1 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/local/LocalFileTest.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/local/LocalFileTest.java @@ -391,22 +391,53 @@ void testCanalJsonSink() throws IOException { String dataStr = FileUtils.readFileToStr(path); Assertions.assertTrue( dataStr.contains( - "{\"data\":[{\"a\":1,\"b\":\"A\",\"c\":100}],\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + "{\"old\":null,\"data\":[{\"a\":1,\"b\":\"A\",\"c\":100}],\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); Assertions.assertTrue( dataStr.contains( - "{\"data\":[{\"a\":2,\"b\":\"B\",\"c\":100}],\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + "{\"old\":null,\"data\":[{\"a\":2,\"b\":\"B\",\"c\":100}],\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); Assertions.assertTrue( dataStr.contains( - "{\"data\":[{\"a\":3,\"b\":\"C\",\"c\":100}],\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + "{\"old\":null,\"data\":[{\"a\":3,\"b\":\"C\",\"c\":100}],\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); Assertions.assertTrue( dataStr.contains( - "{\"data\":[{\"a\":1,\"b\":\"A\",\"c\":100}],\"type\":\"DELETE\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + "{\"old\":null,\"data\":[{\"a\":1,\"b\":\"A\",\"c\":100}],\"type\":\"DELETE\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); Assertions.assertTrue( dataStr.contains( - "{\"data\":[{\"a\":1,\"b\":\"A_1\",\"c\":100}],\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + "{\"old\":null,\"data\":[{\"a\":1,\"b\":\"A_1\",\"c\":100}],\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); Assertions.assertTrue( dataStr.contains( - "{\"data\":[{\"a\":2,\"b\":\"B\",\"c\":100}],\"type\":\"DELETE\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + "{\"old\":null,\"data\":[{\"a\":2,\"b\":\"B\",\"c\":100}],\"type\":\"DELETE\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + + // test merge_update_event + options.put("merge_update_event",true); + FileUtils.deleteFile("/tmp/seatunnel/LocalFileTest"); + SinkFlowTestUtils.runBatchWithCheckpointDisabled( + catalogTable, + ReadonlyConfig.fromMap(options), + new LocalFileSinkFactory(), + Arrays.asList(row1, row2, row3, row1UpdateBefore, row1UpdateAfter, row2Delete)); + Assertions.assertEquals( + 5, + (long) + FileUtils.getFileLineNumber( + "/tmp/seatunnel/LocalFileTest/canal_json_file.canal_json")); + path = Paths.get("/tmp/seatunnel/LocalFileTest/canal_json_file.canal_json"); + dataStr = FileUtils.readFileToStr(path); + Assertions.assertTrue( + dataStr.contains( + "{\"old\":null,\"data\":[{\"a\":1,\"b\":\"A\",\"c\":100}],\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + Assertions.assertTrue( + dataStr.contains( + "{\"old\":null,\"data\":[{\"a\":2,\"b\":\"B\",\"c\":100}],\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + Assertions.assertTrue( + dataStr.contains( + "{\"old\":null,\"data\":[{\"a\":3,\"b\":\"C\",\"c\":100}],\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + Assertions.assertTrue( + dataStr.contains( + "{\"old\":[{\"a\":1,\"b\":\"A\",\"c\":100}],\"data\":[{\"a\":1,\"b\":\"A_1\",\"c\":100}],\"type\":\"UPDATE\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + Assertions.assertTrue( + dataStr.contains( + "{\"old\":null,\"data\":[{\"a\":2,\"b\":\"B\",\"c\":100}],\"type\":\"DELETE\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); } @Test @@ -501,6 +532,39 @@ void testDebeziumJsonSink() throws IOException { Assertions.assertTrue( dataStr.contains( "{\"before\":{\"a\":2,\"b\":\"B\",\"c\":100},\"after\":null,\"op\":\"d\",\"source\":{\"schema\":\"default\",\"database\":\"default\",\"table\":\"default\"},\"ts_ms\":1}")); + + // test merge_update_event + options.put("merge_update_event",true); + FileUtils.deleteFile("/tmp/seatunnel/LocalFileTest"); + SinkFlowTestUtils.runBatchWithCheckpointDisabled( + catalogTable, + ReadonlyConfig.fromMap(options), + new LocalFileSinkFactory(), + Arrays.asList(row1, row2, row3, row1UpdateBefore, row1UpdateAfter, row2Delete)); + Assertions.assertEquals( + 5, + (long) + FileUtils.getFileLineNumber( + "/tmp/seatunnel/LocalFileTest/debezium_json_file.debezium_json")); + path = Paths.get("/tmp/seatunnel/LocalFileTest/debezium_json_file.debezium_json"); + dataStr = FileUtils.readFileToStr(path); + Assertions.assertTrue( + dataStr.contains( + "{\"before\":null,\"after\":{\"a\":1,\"b\":\"A\",\"c\":100},\"op\":\"c\",\"source\":{\"schema\":\"default\",\"database\":\"default\",\"table\":\"default\"},\"ts_ms\":1}")); + Assertions.assertTrue( + dataStr.contains( + "{\"before\":null,\"after\":{\"a\":2,\"b\":\"B\",\"c\":100},\"op\":\"c\",\"source\":{\"schema\":\"default\",\"database\":\"default\",\"table\":\"default\"},\"ts_ms\":1}")); + Assertions.assertTrue( + dataStr.contains( + "{\"before\":null,\"after\":{\"a\":3,\"b\":\"C\",\"c\":100},\"op\":\"c\",\"source\":{\"schema\":\"default\",\"database\":\"default\",\"table\":\"default\"},\"ts_ms\":1}")); + Assertions.assertTrue( + dataStr.contains( + "{\"before\":{\"a\":1,\"b\":\"A\",\"c\":100},\"after\":{\"a\":1,\"b\":\"A_1\",\"c\":100},\"op\":\"u\",\"source\":{\"schema\":\"default\",\"database\":\"default\",\"table\":\"default\"},\"ts_ms\":1}")); + Assertions.assertTrue( + dataStr.contains( + "{\"before\":{\"a\":2,\"b\":\"B\",\"c\":100},\"after\":null,\"op\":\"d\",\"source\":{\"schema\":\"default\",\"database\":\"default\",\"table\":\"default\"},\"ts_ms\":1}")); + + } @Test @@ -578,22 +642,55 @@ void testMaxWellJsonSink() throws IOException { String dataStr = FileUtils.readFileToStr(path); Assertions.assertTrue( dataStr.contains( - "{\"data\":{\"a\":1,\"b\":\"A\",\"c\":100},\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + "{\"old\":null,\"data\":{\"a\":1,\"b\":\"A\",\"c\":100},\"type\":\"insert\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); Assertions.assertTrue( dataStr.contains( - "{\"data\":{\"a\":2,\"b\":\"B\",\"c\":100},\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + "{\"old\":null,\"data\":{\"a\":2,\"b\":\"B\",\"c\":100},\"type\":\"insert\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); Assertions.assertTrue( dataStr.contains( - "{\"data\":{\"a\":3,\"b\":\"C\",\"c\":100},\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + "{\"old\":null,\"data\":{\"a\":3,\"b\":\"C\",\"c\":100},\"type\":\"insert\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); Assertions.assertTrue( dataStr.contains( - "{\"data\":{\"a\":1,\"b\":\"A\",\"c\":100},\"type\":\"DELETE\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + "{\"old\":null,\"data\":{\"a\":1,\"b\":\"A\",\"c\":100},\"type\":\"delete\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); Assertions.assertTrue( dataStr.contains( - "{\"data\":{\"a\":1,\"b\":\"A_1\",\"c\":100},\"type\":\"INSERT\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + "{\"old\":null,\"data\":{\"a\":1,\"b\":\"A_1\",\"c\":100},\"type\":\"insert\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); Assertions.assertTrue( dataStr.contains( - "{\"data\":{\"a\":2,\"b\":\"B\",\"c\":100},\"type\":\"DELETE\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + "{\"old\":null,\"data\":{\"a\":2,\"b\":\"B\",\"c\":100},\"type\":\"delete\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + + // test merge_update_event + options.put("merge_update_event",true); + FileUtils.deleteFile("/tmp/seatunnel/LocalFileTest"); + SinkFlowTestUtils.runBatchWithCheckpointDisabled( + catalogTable, + ReadonlyConfig.fromMap(options), + new LocalFileSinkFactory(), + Arrays.asList(row1, row2, row3, row1UpdateBefore, row1UpdateAfter, row2Delete)); + Assertions.assertEquals( + 5, + (long) + FileUtils.getFileLineNumber( + "/tmp/seatunnel/LocalFileTest/maxwell_json_file.maxwell_json")); + path = Paths.get("/tmp/seatunnel/LocalFileTest/maxwell_json_file.maxwell_json"); + dataStr = FileUtils.readFileToStr(path); + Assertions.assertTrue( + dataStr.contains( + "{\"old\":null,\"data\":{\"a\":1,\"b\":\"A\",\"c\":100},\"type\":\"insert\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + Assertions.assertTrue( + dataStr.contains( + "{\"old\":null,\"data\":{\"a\":2,\"b\":\"B\",\"c\":100},\"type\":\"insert\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + Assertions.assertTrue( + dataStr.contains( + "{\"old\":null,\"data\":{\"a\":3,\"b\":\"C\",\"c\":100},\"type\":\"insert\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + Assertions.assertTrue( + dataStr.contains( + "{\"old\":{\"a\":1,\"b\":\"A\",\"c\":100},\"data\":{\"a\":1,\"b\":\"A_1\",\"c\":100},\"type\":\"update\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + Assertions.assertTrue( + dataStr.contains( + "{\"old\":null,\"data\":{\"a\":2,\"b\":\"B\",\"c\":100},\"type\":\"delete\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); + + } @Test diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java index 7cd627d0403..1924f677817 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java @@ -78,7 +78,7 @@ public byte[] serialize(SeaTunnelRow row) { } if (mergeUpdateEventFlag && row.getRowKind() == RowKind.UPDATE_AFTER) { - reuse.setField(0, cacheUpdateBeforeRow); + reuse.setField(0, new SeaTunnelRow[] {cacheUpdateBeforeRow}); } else { reuse.setField(0, null); } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java index 93e03ef363b..44f8fde4973 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java @@ -58,12 +58,14 @@ public class MaxWellJsonSerializationSchema implements SerializationSchema { public MaxWellJsonSerializationSchema(SeaTunnelRowType rowType) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType)); this.reuse = new SeaTunnelRow(6); + this.mergeUpdateEventFlag = MaxWellJsonFormatOptions.MERGE_UPDATE_EVENT.defaultValue(); } public MaxWellJsonSerializationSchema( SeaTunnelRowType rowType, Charset charset, Map options) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType), charset); this.reuse = new SeaTunnelRow(6); + this.mergeUpdateEventFlag = MaxWellJsonFormatOptions.getMergeUpdateEvent(options); } @Override From 529dc6c96fa0b0b1e9f05d745f8094d396b782f3 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Fri, 29 Aug 2025 18:30:15 +0800 Subject: [PATCH 14/37] init --- .../connectors/seatunnel/file/local/LocalFileTest.java | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/local/LocalFileTest.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/local/LocalFileTest.java index 41fd6fa20b1..2d4c4a4a8ca 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/local/LocalFileTest.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/local/LocalFileTest.java @@ -409,7 +409,7 @@ void testCanalJsonSink() throws IOException { "{\"old\":null,\"data\":[{\"a\":2,\"b\":\"B\",\"c\":100}],\"type\":\"DELETE\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); // test merge_update_event - options.put("merge_update_event",true); + options.put("merge_update_event", true); FileUtils.deleteFile("/tmp/seatunnel/LocalFileTest"); SinkFlowTestUtils.runBatchWithCheckpointDisabled( catalogTable, @@ -534,7 +534,7 @@ void testDebeziumJsonSink() throws IOException { "{\"before\":{\"a\":2,\"b\":\"B\",\"c\":100},\"after\":null,\"op\":\"d\",\"source\":{\"schema\":\"default\",\"database\":\"default\",\"table\":\"default\"},\"ts_ms\":1}")); // test merge_update_event - options.put("merge_update_event",true); + options.put("merge_update_event", true); FileUtils.deleteFile("/tmp/seatunnel/LocalFileTest"); SinkFlowTestUtils.runBatchWithCheckpointDisabled( catalogTable, @@ -563,8 +563,6 @@ void testDebeziumJsonSink() throws IOException { Assertions.assertTrue( dataStr.contains( "{\"before\":{\"a\":2,\"b\":\"B\",\"c\":100},\"after\":null,\"op\":\"d\",\"source\":{\"schema\":\"default\",\"database\":\"default\",\"table\":\"default\"},\"ts_ms\":1}")); - - } @Test @@ -660,7 +658,7 @@ void testMaxWellJsonSink() throws IOException { "{\"old\":null,\"data\":{\"a\":2,\"b\":\"B\",\"c\":100},\"type\":\"delete\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); // test merge_update_event - options.put("merge_update_event",true); + options.put("merge_update_event", true); FileUtils.deleteFile("/tmp/seatunnel/LocalFileTest"); SinkFlowTestUtils.runBatchWithCheckpointDisabled( catalogTable, @@ -689,8 +687,6 @@ void testMaxWellJsonSink() throws IOException { Assertions.assertTrue( dataStr.contains( "{\"old\":null,\"data\":{\"a\":2,\"b\":\"B\",\"c\":100},\"type\":\"delete\",\"database\":\"default\",\"table\":\"default\",\"ts\":1}")); - - } @Test From 0e7ee4cd66382e4f218aaf4579eceabe1b133c46 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Fri, 29 Aug 2025 18:47:44 +0800 Subject: [PATCH 15/37] init --- .../json/canal/CanalJsonSerDeSchemaTest.java | 52 +++++++++---------- .../maxwell/MaxWellJsonSerDeSchemaTest.java | 52 +++++++++---------- .../json/ogg/OggJsonSerDeSchemaTest.java | 42 +++++++-------- 3 files changed, 73 insertions(+), 73 deletions(-) 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 ad0c1ec9601..1a951aae608 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 @@ -207,32 +207,32 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali List expectedResult = Arrays.asList( - "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", - "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", - "{\"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\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", - "{\"data\":[{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", - "{\"data\":[{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", - "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":null,\"weight\":1.0}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", - "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", - "{\"data\":[{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", - "{\"data\":[{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", - "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":null,\"weight\":1.0}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944202218}", - "{\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944202218}", - "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944279665}", - "{\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944279665}", - "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944288394}", - "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944288394}", - "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944288717}", - "{\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944288717}", - "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337341}", - "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337341}", - "{\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337341}", - "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337663}", - "{\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337663}", - "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337663}", - "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337663}", - "{\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944418418}", - "{\"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\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944418418}"); + "{\"old\":null,\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"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\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":null,\"weight\":1.0}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":null,\"weight\":1.0}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944202218}", + "{\"old\":null,\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944202218}", + "{\"old\":null,\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944279665}", + "{\"old\":null,\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944279665}", + "{\"old\":null,\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944288394}", + "{\"old\":null,\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944288394}", + "{\"old\":null,\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944288717}", + "{\"old\":null,\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944288717}", + "{\"old\":null,\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337341}", + "{\"old\":null,\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337341}", + "{\"old\":null,\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337341}", + "{\"old\":null,\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337663}", + "{\"old\":null,\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337663}", + "{\"old\":null,\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337663}", + "{\"old\":null,\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337663}", + "{\"old\":null,\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944418418}", + "{\"old\":null,\"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\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944418418}"); assertEquals(expectedResult, result); } diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java index 96416475337..bf60e452ac6 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java @@ -121,32 +121,32 @@ public void runTest(List lines, MaxWellJsonDeserializationSchema deseria List expectedResult = Arrays.asList( - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", - "{\"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\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", - "{\"data\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", - "{\"data\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", - "{\"data\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", - "{\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684893000}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684893000}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684897000}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684897000}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684900000}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684904000}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684906000}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684906000}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684912000}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684912000}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684914000}", - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684928000}", - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684928000}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684928000}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684928000}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684938000}", - "{\"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\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684938000}"); + "{\"old\":null,\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"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\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684893000}", + "{\"old\":null,\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684893000}", + "{\"old\":null,\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684897000}", + "{\"old\":null,\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684897000}", + "{\"old\":null,\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684900000}", + "{\"old\":null,\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684904000}", + "{\"old\":null,\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684906000}", + "{\"old\":null,\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684906000}", + "{\"old\":null,\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684912000}", + "{\"old\":null,\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684912000}", + "{\"old\":null,\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684914000}", + "{\"old\":null,\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684928000}", + "{\"old\":null,\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684928000}", + "{\"old\":null,\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684928000}", + "{\"old\":null,\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684928000}", + "{\"old\":null,\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684938000}", + "{\"old\":null,\"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\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684938000}"); assertEquals(expectedResult, result); } 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 b4da3d4ae24..726a5c7bfa3 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 @@ -144,7 +144,7 @@ public void testDeserializeUnknownTypeJson() throws Exception { final SimpleCollector collector = new SimpleCollector(); String unknownType = "XX"; String unknownOperationMsg = - "{\"data\":{\"id\":101,\"name\":\"scooter\"},\"op_type\":\"" + unknownType + "\"}"; + "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\"},\"op_type\":\"" + unknownType + "\"}"; SeaTunnelRuntimeException expected = CommonError.jsonOperationError(FORMAT, unknownOperationMsg); SeaTunnelRuntimeException cause = @@ -206,26 +206,26 @@ public void runTest(List lines, OggJsonDeserializationSchema deserializa List expectedResult = Arrays.asList( - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589384406000}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589384407000}", - "{\"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\",\"table\":\"..test\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"DELETE\",\"table\":\"..test\",\"op_ts\":1589390787000}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589390787000}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"DELETE\",\"table\":\"..test\",\"op_ts\":1589390899000}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589390899000}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589391010000}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589391043000}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"DELETE\",\"table\":\"..test\",\"op_ts\":1589391140000}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589391140000}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"DELETE\",\"table\":\"..test\",\"op_ts\":1589391130000}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"INSERT\",\"table\":\"..test\",\"op_ts\":1589391130000}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"DELETE\",\"table\":\"..test\",\"op_ts\":1589391144000}"); + "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384406000}", + "{\"before\":null,\"after\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"op_type\":\"D\",\"table\":\"..test\",\"op_ts\":1589390787000}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589390787000}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"op_type\":\"D\",\"table\":\"..test\",\"op_ts\":1589390899000}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589390899000}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589391010000}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589391043000}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"op_type\":\"D\",\"table\":\"..test\",\"op_ts\":1589391140000}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589391140000}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"op_type\":\"D\",\"table\":\"..test\",\"op_ts\":1589391130000}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589391130000}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op_type\":\"D\",\"table\":\"..test\",\"op_ts\":1589391144000}"); assertEquals(expectedResult, result); } From 73bec44f15bb0fbe3bbe574b7f69c315c7a0f872 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Fri, 29 Aug 2025 18:48:54 +0800 Subject: [PATCH 16/37] init --- .../seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 726a5c7bfa3..afa9abb9b3e 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 @@ -144,7 +144,9 @@ public void testDeserializeUnknownTypeJson() throws Exception { final SimpleCollector collector = new SimpleCollector(); String unknownType = "XX"; String unknownOperationMsg = - "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\"},\"op_type\":\"" + unknownType + "\"}"; + "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\"},\"op_type\":\"" + + unknownType + + "\"}"; SeaTunnelRuntimeException expected = CommonError.jsonOperationError(FORMAT, unknownOperationMsg); SeaTunnelRuntimeException cause = From b263e7acc66957ba17ab4d43323bbc22039dbbca Mon Sep 17 00:00:00 2001 From: dyp12 Date: Sat, 30 Aug 2025 15:51:09 +0800 Subject: [PATCH 17/37] init --- .../e2e/connector/kafka/KafkaFormatIT.java | 68 +++++++++---------- .../json/canal/CanalJsonSerDeSchemaTest.java | 39 +++++++++++ .../debezium/DebeziumJsonSerDeSchemaTest.java | 35 ++++++++++ .../maxwell/MaxWellJsonSerDeSchemaTest.java | 39 +++++++++++ .../json/ogg/OggJsonSerDeSchemaTest.java | 34 ++++++++++ 5 files changed, 181 insertions(+), 34 deletions(-) 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 5025d676c76..3922a98a55f 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 @@ -503,20 +503,20 @@ private void checkFormatCanalAndOgg() { private void checkCanalFormat() { List expectedResult = Arrays.asList( - "{\"data\":[{\"id\":1101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", - "{\"data\":[{\"id\":1102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":\"8.1\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", - "{\"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\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", - "{\"data\":[{\"id\":1104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":\"0.75\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", - "{\"data\":[{\"id\":1105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":\"0.875\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", - "{\"data\":[{\"id\":1106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":\"1.0\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", - "{\"data\":[{\"id\":1107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", - "{\"data\":[{\"id\":1108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":\"0.1\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", - "{\"data\":[{\"id\":1109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", - "{\"data\":[{\"id\":1101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"}],\"type\":\"DELETE\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900619}", - "{\"data\":[{\"id\":1101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"4.56\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900619}", - "{\"data\":[{\"id\":1107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"}],\"type\":\"DELETE\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900619}", - "{\"data\":[{\"id\":1107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"7.88\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900619}", - "{\"data\":[{\"id\":1109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"}],\"type\":\"DELETE\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900619}"); + "{\"old\":null,\"data\":[{\"id\":1101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", + "{\"old\":null,\"data\":[{\"id\":1102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":\"8.1\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", + "{\"old\":null,\"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\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", + "{\"old\":null,\"data\":[{\"id\":1104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":\"0.75\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", + "{\"old\":null,\"data\":[{\"id\":1105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":\"0.875\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", + "{\"old\":null,\"data\":[{\"id\":1106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":\"1.0\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", + "{\"old\":null,\"data\":[{\"id\":1107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", + "{\"old\":null,\"data\":[{\"id\":1108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":\"0.1\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", + "{\"old\":null,\"data\":[{\"id\":1109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900618}", + "{\"old\":null,\"data\":[{\"id\":1101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"}],\"type\":\"DELETE\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900619}", + "{\"old\":null,\"data\":[{\"id\":1101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"4.56\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900619}", + "{\"old\":null,\"data\":[{\"id\":1107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"}],\"type\":\"DELETE\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900619}", + "{\"old\":null,\"data\":[{\"id\":1107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"7.88\"}],\"type\":\"INSERT\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900619}", + "{\"old\":null,\"data\":[{\"id\":1109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"}],\"type\":\"DELETE\",\"database\":null,\"table\":\"test-cdc_mds\",\"ts\":1697788900619}"); ArrayList result = new ArrayList<>(); ArrayList topics = new ArrayList<>(); @@ -619,26 +619,26 @@ private void checkMaxWellFormat() { private void checkOggFormat() { List kafkaExpectedResult = Arrays.asList( - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.140000104904175\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589384406000}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":\"8.100000381469727\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":\"0.800000011920929\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":\"0.75\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":\"0.875\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":\"1\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.300000190734863\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":\"0.10000000149011612\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.200000762939453\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":\"1\"},\"type\":\"DELETE\",\"table\":\"test-ogg-source\",\"op_ts\":1589390787000}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":\"1\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589390787000}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.300000190734863\"},\"type\":\"DELETE\",\"table\":\"test-ogg-source\",\"op_ts\":1589390899000}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.099999904632568\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589390899000}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":\"0.20000000298023224\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589391010000}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":\"5.179999828338623\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589391043000}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":\"0.20000000298023224\"},\"type\":\"DELETE\",\"table\":\"test-ogg-source\",\"op_ts\":1589391140000}", - "{\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":\"0.5\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589391140000}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":\"5.179999828338623\"},\"type\":\"DELETE\",\"table\":\"test-ogg-source\",\"op_ts\":1589391130000}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":\"5.170000076293945\"},\"type\":\"INSERT\",\"table\":\"test-ogg-source\",\"op_ts\":1589391130000}", - "{\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":\"5.170000076293945\"},\"type\":\"DELETE\",\"table\":\"test-ogg-source\",\"op_ts\":1589391144000}"); + "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.140000104904175\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589384406000}", + "{\"before\":null,\"after\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":\"8.100000381469727\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":\"0.800000011920929\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":\"0.75\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":\"0.875\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":\"1\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.300000190734863\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":\"0.10000000149011612\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.200000762939453\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":\"1\"},\"op_type\":\"D\",\"table\":\"test-ogg-source\",\"op_ts\":1589390787000}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":\"1\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589390787000}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.300000190734863\"},\"op_type\":\"D\",\"table\":\"test-ogg-source\",\"op_ts\":1589390899000}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.099999904632568\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589390899000}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":\"0.20000000298023224\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589391010000}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":\"5.179999828338623\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589391043000}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":\"0.20000000298023224\"},\"op_type\":\"D\",\"table\":\"test-ogg-source\",\"op_ts\":1589391140000}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":\"0.5\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589391140000}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":\"5.179999828338623\"},\"op_type\":\"D\",\"table\":\"test-ogg-source\",\"op_ts\":1589391130000}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":\"5.170000076293945\"},\"op_type\":\"I\",\"table\":\"test-ogg-source\",\"op_ts\":1589391130000}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":\"5.170000076293945\"},\"op_type\":\"D\",\"table\":\"test-ogg-source\",\"op_ts\":1589391144000}"); ArrayList checkKafkaConsumerResult = new ArrayList<>(); ArrayList topics = new ArrayList<>(); 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 1a951aae608..bd851b1f072 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 @@ -37,7 +37,9 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import static org.apache.seatunnel.api.table.type.BasicType.FLOAT_TYPE; @@ -234,6 +236,43 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali "{\"old\":null,\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944418418}", "{\"old\":null,\"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\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944418418}"); assertEquals(expectedResult, result); + + // test merge_update_event + Map options = new HashMap<>(); + options.put(CanalJsonFormatOptions.MERGE_UPDATE_EVENT.key(), "true"); + serializationSchema = + new CanalJsonSerializationSchema( + SEATUNNEL_ROW_TYPE, StandardCharsets.UTF_8, options); + result.clear(); + for (SeaTunnelRow rowData : collector.list) { + if (serializationSchema.serialize(rowData) != null) { + result.add( + new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); + } + } + expectedResult = + Arrays.asList( + "{\"old\":null,\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"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\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":null,\"weight\":1.0}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":null,\"data\":[{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944146308}", + "{\"old\":[{\"id\":106,\"name\":\"hammer\",\"description\":null,\"weight\":1.0}],\"data\":[{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0}],\"type\":\"UPDATE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944202218}", + "{\"old\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3}],\"data\":[{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1}],\"type\":\"UPDATE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944279665}", + "{\"old\":null,\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944288394}", + "{\"old\":null,\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"type\":\"INSERT\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944288394}", + "{\"old\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2}],\"data\":[{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5}],\"type\":\"UPDATE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944288717}", + "{\"old\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18}],\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"UPDATE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337341}", + "{\"old\":null,\"data\":[{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337341}", + "{\"old\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14}],\"data\":[{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17}],\"type\":\"UPDATE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337663}", + "{\"old\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1}],\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"UPDATE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944337663}", + "{\"old\":null,\"data\":[{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17}],\"type\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944418418}", + "{\"old\":null,\"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\":\"DELETE\",\"database\":\"\",\"table\":\"test\",\"ts\":1598944418418}"); + assertEquals(expectedResult, result); } // -------------------------------------------------------------------------------------------- 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 8607aedbf0b..47acdd4066e 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 @@ -43,7 +43,9 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import static org.apache.seatunnel.api.table.type.BasicType.BOOLEAN_TYPE; @@ -637,6 +639,39 @@ private void testSerializationDeserialization(String resourceFile, boolean schem "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op\":\"c\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589362330904}", "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"after\":null,\"op\":\"d\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589362344455}"); assertEquals(expected, actual); + + // test merge_update_event + Map options = new HashMap<>(); + options.put(DebeziumJsonFormatOptions.MERGE_UPDATE_EVENT.key(), "true"); + serializationSchema = + new DebeziumJsonSerializationSchema( + SEATUNNEL_ROW_TYPE, StandardCharsets.UTF_8, options); + actual.clear(); + for (SeaTunnelRow rowData : collector.list) { + if (serializationSchema.serialize(rowData) != null) { + actual.add( + new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); + } + } + expected = + Arrays.asList( + "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"op\":\"c\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589355606100}", + "{\"before\":null,\"after\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"op\":\"c\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589355606101}", + "{\"before\":null,\"after\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"op\":\"c\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589355606101}", + "{\"before\":null,\"after\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"op\":\"c\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589355606101}", + "{\"before\":null,\"after\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"op\":\"c\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589355606101}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"op\":\"c\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589355606101}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"op\":\"c\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589355606101}", + "{\"before\":null,\"after\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"op\":\"c\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589355606101}", + "{\"before\":null,\"after\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"op\":\"c\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589355606101}", + "{\"before\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"op\":\"u\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589361987936}", + "{\"before\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"op\":\"u\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589362099505}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"op\":\"c\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589362210230}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"op\":\"c\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589362243428}", + "{\"before\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"op\":\"u\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589362293539}", + "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op\":\"u\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589362330904}", + "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"after\":null,\"op\":\"d\",\"source\":{\"schema\":\"\",\"database\":\"\",\"table\":\"test\"},\"ts_ms\":1589362344455}"); + assertEquals(expected, actual); } // -------------------------------------------------------------------------------------------- // Utilities diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java index bf60e452ac6..0832df85318 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java @@ -35,7 +35,9 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import static org.apache.seatunnel.api.table.type.BasicType.FLOAT_TYPE; @@ -148,6 +150,43 @@ public void runTest(List lines, MaxWellJsonDeserializationSchema deseria "{\"old\":null,\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684938000}", "{\"old\":null,\"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\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684938000}"); assertEquals(expectedResult, result); + + // test merge_update_event + Map options = new HashMap<>(); + options.put(MaxWellJsonFormatOptions.MERGE_UPDATE_EVENT.key(), "true"); + serializationSchema = + new MaxWellJsonSerializationSchema( + catalogTables.getSeaTunnelRowType(), StandardCharsets.UTF_8, options); + actual.clear(); + for (SeaTunnelRow rowData : collector.list) { + if (serializationSchema.serialize(rowData) != null) { + actual.add( + new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); + } + } + expected = + Arrays.asList( + "{\"old\":null,\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"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\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":null,\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684883000}", + "{\"old\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"type\":\"update\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684893000}", + "{\"old\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"type\":\"update\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684897000}", + "{\"old\":null,\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684900000}", + "{\"old\":null,\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"type\":\"insert\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684904000}", + "{\"old\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"data\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"type\":\"update\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684906000}", + "{\"old\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"update\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684912000}", + "{\"old\":null,\"data\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"type\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684914000}", + "{\"old\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":5.17},\"type\":\"update\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684928000}", + "{\"old\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"update\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684928000}", + "{\"old\":null,\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":5.17},\"type\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684938000}", + "{\"old\":null,\"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\":\"delete\",\"database\":\"\",\"table\":\"test\",\"ts\":1596684938000}"); + assertEquals(expected, actual); } // -------------------------------------------------------------------------------------------- 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 afa9abb9b3e..9abff0d726e 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 @@ -38,7 +38,9 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import static org.apache.seatunnel.api.table.type.BasicType.FLOAT_TYPE; @@ -229,6 +231,38 @@ public void runTest(List lines, OggJsonDeserializationSchema deserializa "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589391130000}", "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op_type\":\"D\",\"table\":\"..test\",\"op_ts\":1589391144000}"); assertEquals(expectedResult, result); + + // test merge_update_event + Map options = new HashMap<>(); + options.put(OggJsonFormatOptions.MERGE_UPDATE_EVENT.key(), "true"); + serializationSchema = + new OggJsonSerializationSchema(SEATUNNEL_ROW_TYPE, StandardCharsets.UTF_8, options); + actual.clear(); + for (SeaTunnelRow rowData : collector.list) { + if (serializationSchema.serialize(rowData) != null) { + actual.add( + new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); + } + } + expected = + Arrays.asList( + "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384406000}", + "{\"before\":null,\"after\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":null,\"after\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589384407000}", + "{\"before\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"op_type\":\"U\",\"table\":\"..test\",\"op_ts\":1589390787000}", + "{\"before\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"op_type\":\"U\",\"table\":\"..test\",\"op_ts\":1589390899000}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589391010000}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"op_type\":\"I\",\"table\":\"..test\",\"op_ts\":1589391043000}", + "{\"before\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"op_type\":\"U\",\"table\":\"..test\",\"op_ts\":1589391140000}", + "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op_type\":\"U\",\"table\":\"..test\",\"op_ts\":1589391130000}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op_type\":\"D\",\"table\":\"..test\",\"op_ts\":1589391144000}"); + assertEquals(expected, actual); } // -------------------------------------------------------------------------------------------- From 660a21e1f6bb8b1a3381878042298a445e7f739a Mon Sep 17 00:00:00 2001 From: dyp12 Date: Sat, 30 Aug 2025 16:01:08 +0800 Subject: [PATCH 18/37] init --- .../e2e/connector/kafka/KafkaFormatIT.java | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) 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 3922a98a55f..59eff0951e1 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 @@ -90,7 +90,7 @@ @DisabledOnContainer( value = {}, - type = {EngineType.SPARK}, + type = {EngineType.SPARK,EngineType.SEATUNNEL}, disabledReason = "Spark engine will lose the row kind of record") public class KafkaFormatIT extends TestSuiteBase implements TestResource { @@ -562,20 +562,20 @@ private void checkCanalFormat() { private void checkMaxWellFormat() { List expectedResult = Arrays.asList( - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"},\"type\":\"INSERT\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", - "{\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":\"8.1\"},\"type\":\"INSERT\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", - "{\"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\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", - "{\"data\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":\"0.75\"},\"type\":\"INSERT\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", - "{\"data\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":\"0.875\"},\"type\":\"INSERT\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", - "{\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":\"1.0\"},\"type\":\"INSERT\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"},\"type\":\"INSERT\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", - "{\"data\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":\"0.1\"},\"type\":\"INSERT\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", - "{\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"},\"type\":\"INSERT\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"},\"type\":\"DELETE\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", - "{\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"4.56\"},\"type\":\"INSERT\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"},\"type\":\"DELETE\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", - "{\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"7.88\"},\"type\":\"INSERT\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", - "{\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"},\"type\":\"DELETE\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}"); + "{\"old\":null,\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"},\"type\":\"insert\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", + "{\"old\":null,\"data\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":\"8.1\"},\"type\":\"insert\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", + "{\"old\":null,\"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\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", + "{\"old\":null,\"data\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":\"0.75\"},\"type\":\"insert\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", + "{\"old\":null,\"data\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":\"0.875\"},\"type\":\"insert\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", + "{\"old\":null,\"data\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":\"1.0\"},\"type\":\"insert\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", + "{\"old\":null,\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"},\"type\":\"insert\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", + "{\"old\":null,\"data\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":\"0.1\"},\"type\":\"insert\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", + "{\"old\":null,\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"},\"type\":\"insert\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", + "{\"old\":null,\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"3.14\"},\"type\":\"delete\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", + "{\"old\":null,\"data\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":\"4.56\"},\"type\":\"insert\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", + "{\"old\":null,\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"5.3\"},\"type\":\"delete\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", + "{\"old\":null,\"data\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":\"7.88\"},\"type\":\"insert\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}", + "{\"old\":null,\"data\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":\"22.2\"},\"type\":\"delete\",\"database\":null,\"table\":\"maxwell-test-cdc_mds\",\"ts\":1699253290000}"); ArrayList result = new ArrayList<>(); ArrayList topics = new ArrayList<>(); From 1add2d1da870ca37ad396ff4b98239e7ac8f3a33 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Sat, 30 Aug 2025 16:11:08 +0800 Subject: [PATCH 19/37] init --- .../org/apache/seatunnel/e2e/connector/kafka/KafkaFormatIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 59eff0951e1..5cda6362664 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 @@ -90,7 +90,7 @@ @DisabledOnContainer( value = {}, - type = {EngineType.SPARK,EngineType.SEATUNNEL}, + type = {EngineType.SPARK}, disabledReason = "Spark engine will lose the row kind of record") public class KafkaFormatIT extends TestSuiteBase implements TestResource { From d4bfa259fd65ffe6c8890e6a3fbc5b4e406fe9e2 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Sun, 31 Aug 2025 09:51:16 +0800 Subject: [PATCH 20/37] init --- .../format/json/debezium/DebeziumJsonDeserializationSchema.java | 2 +- .../seatunnel/format/json/ogg/OggJsonDeserializationSchema.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 122af68d236..d5fd640ce3e 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 @@ -147,7 +147,6 @@ private void parsePayload(Collector out, TablePath tablePath, Json if (tsNode != null) { MetadataUtil.setEventTime(before, tsNode.asLong()); } - out.collect(before); SeaTunnelRow after = debeziumRowConverter.parse(payload.get(DATA_AFTER)); after.setRowKind(RowKind.UPDATE_AFTER); @@ -158,6 +157,7 @@ private void parsePayload(Collector out, TablePath tablePath, Json if (tsNode != null) { MetadataUtil.setEventTime(after, tsNode.asLong()); } + out.collect(before); out.collect(after); break; case OP_DELETE: 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 f1731e3872b..b7b99af2566 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 @@ -195,7 +195,6 @@ public void deserializeMessage( if (tsNode != null) { MetadataUtil.setEventTime(before, ts); } - out.collect(before); after.setRowKind(RowKind.UPDATE_AFTER); if (tablePath != null) { @@ -204,6 +203,7 @@ public void deserializeMessage( if (tsNode != null) { MetadataUtil.setEventTime(after, ts); } + out.collect(before); out.collect(after); break; case OP_DELETE: From 16d2a3373fb616ba2b08e831d427d2d41e7dc846 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Thu, 4 Sep 2025 10:27:06 +0800 Subject: [PATCH 21/37] feat:update doc --- docs/en/connector-v2/sink/CosFile.md | 79 ++++++++++---------- docs/en/connector-v2/sink/HdfsFile.md | 7 +- docs/en/connector-v2/sink/LocalFile.md | 77 ++++++++++---------- docs/en/connector-v2/sink/ObsFile.md | 61 ++++++++-------- docs/en/connector-v2/sink/OssFile.md | 85 +++++++++++----------- docs/en/connector-v2/sink/OssJindoFile.md | 79 ++++++++++---------- docs/en/connector-v2/sink/S3File.md | 89 +++++++++++------------ docs/en/connector-v2/sink/SftpFile.md | 85 +++++++++++----------- 8 files changed, 277 insertions(+), 285 deletions(-) diff --git a/docs/en/connector-v2/sink/CosFile.md b/docs/en/connector-v2/sink/CosFile.md index efa3b60d270..44ffc3f9a7a 100644 --- a/docs/en/connector-v2/sink/CosFile.md +++ b/docs/en/connector-v2/sink/CosFile.md @@ -43,43 +43,43 @@ To use this connector you need put hadoop-cos-{hadoop.version}-{version}.jar and ## Options -| Name | Type | Required | Default | Description | -|---------------------------------------|---------|----------|--------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| path | string | yes | - | | -| tmp_path | string | no | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. Need a COS dir. | -| bucket | string | yes | - | | -| secret_id | string | yes | - | | -| secret_key | string | yes | - | | -| region | string | yes | - | | -| custom_filename | boolean | no | false | Whether you need custom the filename | -| file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | -| filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | -| file_format_type | string | no | "csv" | | -| filename_extension | string | no | - | Override the default file name extensions with custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype` | -| field_delimiter | string | no | '\001' for text and ',' for csv | Only used when file_format is text and csv | -| row_delimiter | string | no | "\n" | Only used when file_format is `text`, `csv` and `json` | -| have_partition | boolean | no | false | Whether you need processing partitions. | -| partition_by | array | no | - | Only used then have_partition is true | -| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | -| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true | -| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | -| is_enable_transaction | boolean | no | true | | -| batch_size | int | no | 1000000 | | -| compress_codec | string | no | none | | -| common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format is excel. | -| sheet_max_rows | int | no | 1048576 | Only used when file_format is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format is excel. | -| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | -| xml_root_tag | string | no | RECORDS | Only used when file_format is xml. | -| xml_row_tag | string | no | RECORD | Only used when file_format is xml. | -| xml_use_attr_format | boolean | no | - | Only used when file_format is xml. | -| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | -| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | -| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | -| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | -| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | -| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | +| Name | Type | Required | Default | Description | +|---------------------------------------|---------|----------|--------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| path | string | yes | - | | +| tmp_path | string | no | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. Need a COS dir. | +| bucket | string | yes | - | | +| secret_id | string | yes | - | | +| secret_key | string | yes | - | | +| region | string | yes | - | | +| custom_filename | boolean | no | false | Whether you need custom the filename | +| file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | +| filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | +| file_format_type | string | no | "csv" | | +| filename_extension | string | no | - | Override the default file name extensions with custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype` | +| field_delimiter | string | no | '\001' for text and ',' for csv | Only used when file_format is text and csv | +| row_delimiter | string | no | "\n" | Only used when file_format is `text`, `csv` and `json` | +| have_partition | boolean | no | false | Whether you need processing partitions. | +| partition_by | array | no | - | Only used then have_partition is true | +| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | +| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true | +| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | +| is_enable_transaction | boolean | no | true | | +| batch_size | int | no | 1000000 | | +| compress_codec | string | no | none | | +| common-options | object | no | - | | +| max_rows_in_memory | int | no | - | Only used when file_format is excel. | +| sheet_max_rows | int | no | 1048576 | Only used when file_format is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format is excel. | +| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | +| xml_root_tag | string | no | RECORDS | Only used when file_format is xml. | +| xml_row_tag | string | no | RECORD | Only used when file_format is xml. | +| xml_use_attr_format | boolean | no | - | Only used when file_format is xml. | +| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | +| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | +| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | +| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | +| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json or maxwell_json. When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data | ### path [string] @@ -251,9 +251,8 @@ The encoding of the file to write. This param will be parsed by `Charset.forName ### merge_update_event [boolean] -Only used when file_format_type is canal_json,debezium_json,maxwell_json. -set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; -set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event +Only used when file_format_type is canal_json,debezium_json or maxwell_json. +When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data ## Example diff --git a/docs/en/connector-v2/sink/HdfsFile.md b/docs/en/connector-v2/sink/HdfsFile.md index 0292f5a6b48..5031f13594f 100644 --- a/docs/en/connector-v2/sink/HdfsFile.md +++ b/docs/en/connector-v2/sink/HdfsFile.md @@ -88,7 +88,7 @@ Output data to hdfs file | enable_header_write | boolean | no | false | Only used when file_format_type is text,csv.
false:don't write header,true:write header. | | encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | | remote_user | string | no | - | The remote user name of hdfs. | -| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json or maxwell_json. When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data | ### Tips @@ -96,9 +96,8 @@ Output data to hdfs file ### merge_update_event [boolean] -Only used when file_format_type is canal_json,debezium_json,maxwell_json. -set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; -set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event +Only used when file_format_type is canal_json,debezium_json or maxwell_json. +When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data ## Task Example diff --git a/docs/en/connector-v2/sink/LocalFile.md b/docs/en/connector-v2/sink/LocalFile.md index 30cd929577e..f3d354f3fee 100644 --- a/docs/en/connector-v2/sink/LocalFile.md +++ b/docs/en/connector-v2/sink/LocalFile.md @@ -42,42 +42,42 @@ By default, we use 2PC commit to ensure `exactly-once` ## Options -| Name | Type | Required | Default | Description | -|---------------------------------------|---------|----------|--------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| path | string | yes | - | | -| tmp_path | string | no | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. | -| custom_filename | boolean | no | false | Whether you need custom the filename | -| file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | -| filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | -| file_format_type | string | no | "csv" | | -| filename_extension | string | no | - | Override the default file name extensions with custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype` | -| field_delimiter | string | no | '\001' for text and ',' for csv | Only used when file_format_type is text and csv | -| row_delimiter | string | no | "\n" | Only used when file_format_type is `text`, `csv` and `json` | -| have_partition | boolean | no | false | Whether you need processing partitions. | -| partition_by | array | no | - | Only used then have_partition is true | -| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | -| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true | -| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | -| is_enable_transaction | boolean | no | true | | -| batch_size | int | no | 1000000 | | -| compress_codec | string | no | none | | -| common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | -| sheet_max_rows | int | no | 1048576 | Only used when file_format_type is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | -| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | -| xml_root_tag | string | no | RECORDS | Only used when file_format is xml. | -| xml_row_tag | string | no | RECORD | Only used when file_format is xml. | -| xml_use_attr_format | boolean | no | - | Only used when file_format is xml. | -| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | -| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | -| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | -| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | -| enable_header_write | boolean | no | false | Only used when file_format_type is text,csv.
false:don't write header,true:write header. | -| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | -| schema_save_mode | string | no | CREATE_SCHEMA_WHEN_NOT_EXIST | Existing dir processing method | -| data_save_mode | string | no | APPEND_DATA | Existing data processing method | -| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | +| Name | Type | Required | Default | Description | +|---------------------------------------|---------|----------|--------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| path | string | yes | - | | +| tmp_path | string | no | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. | +| custom_filename | boolean | no | false | Whether you need custom the filename | +| file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | +| filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | +| file_format_type | string | no | "csv" | | +| filename_extension | string | no | - | Override the default file name extensions with custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype` | +| field_delimiter | string | no | '\001' for text and ',' for csv | Only used when file_format_type is text and csv | +| row_delimiter | string | no | "\n" | Only used when file_format_type is `text`, `csv` and `json` | +| have_partition | boolean | no | false | Whether you need processing partitions. | +| partition_by | array | no | - | Only used then have_partition is true | +| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | +| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true | +| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | +| is_enable_transaction | boolean | no | true | | +| batch_size | int | no | 1000000 | | +| compress_codec | string | no | none | | +| common-options | object | no | - | | +| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | +| sheet_max_rows | int | no | 1048576 | Only used when file_format_type is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | +| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | +| xml_root_tag | string | no | RECORDS | Only used when file_format is xml. | +| xml_row_tag | string | no | RECORD | Only used when file_format is xml. | +| xml_use_attr_format | boolean | no | - | Only used when file_format is xml. | +| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | +| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | +| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | +| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | +| enable_header_write | boolean | no | false | Only used when file_format_type is text,csv.
false:don't write header,true:write header. | +| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | +| schema_save_mode | string | no | CREATE_SCHEMA_WHEN_NOT_EXIST | Existing dir processing method | +| data_save_mode | string | no | APPEND_DATA | Existing data processing method | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json or maxwell_json. When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data | ### path [string] @@ -252,9 +252,8 @@ Existing data processing method. ### merge_update_event [boolean] -Only used when file_format_type is canal_json,debezium_json,maxwell_json. -set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; -set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event +Only used when file_format_type is canal_json,debezium_json or maxwell_json. +When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data ## Example diff --git a/docs/en/connector-v2/sink/ObsFile.md b/docs/en/connector-v2/sink/ObsFile.md index 88be0590866..a1621794309 100644 --- a/docs/en/connector-v2/sink/ObsFile.md +++ b/docs/en/connector-v2/sink/ObsFile.md @@ -59,34 +59,34 @@ It only supports hadoop version **2.9.X+**. ## Options -| name | type | required | default | description | -|----------------------------------|---------|----------|--------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| path | string | yes | - | The target dir path. | -| bucket | string | yes | - | The bucket address of obs file system, for example: `obs://obs-bucket-name`. | -| access_key | string | yes | - | The access key of obs file system. | -| access_secret | string | yes | - | The access secret of obs file system. | -| endpoint | string | yes | - | The endpoint of obs file system. | -| custom_filename | boolean | no | false | Whether you need custom the filename. | -| file_name_expression | string | no | "${transactionId}" | Describes the file expression which will be created into the `path`. Only used when custom_filename is true. [Tips](#file_name_expression) | -| filename_time_format | string | no | "yyyy.MM.dd" | Specify the time format of the `path`. Only used when custom_filename is true. [Tips](#filename_time_format) | -| file_format_type | string | no | "csv" | Supported file types. [Tips](#file_format_type) | -| field_delimiter | string | no | '\001' | The separator between columns in a row of data.Only used when file_format is text. | -| row_delimiter | string | no | "\n" | The separator between rows in a file. Only needed by `text`, `csv` and `json` file format. | -| have_partition | boolean | no | false | Whether you need processing partitions. | -| partition_by | array | no | - | Partition data based on selected fields. Only used then have_partition is true. | -| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true.[Tips](#partition_dir_expression) | -| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true.[Tips](#is_partition_field_write_in_file) | -| sink_columns | array | no | | When this parameter is empty, all fields are sink columns.[Tips](#sink_columns) | -| is_enable_transaction | boolean | no | true | [Tips](#is_enable_transaction) | -| batch_size | int | no | 1000000 | [Tips](#batch_size) | -| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | -| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | -| compress_codec | string | no | none | [Tips](#compress_codec) | -| common-options | object | no | - | [Tips](#common_options) | -| max_rows_in_memory | int | no | - | When File Format is Excel,The maximum number of data items that can be cached in the memory.Only used when file_format is excel. | -| sheet_name | string | no | Sheet${Random number} | Writer the sheet of the workbook. Only used when file_format is excel. | -| sheet_max_rows | int | no | 1048576 | Only used when file format_type is excel. | -| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | +| name | type | required | default | description | +|----------------------------------|---------|----------|--------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| path | string | yes | - | The target dir path. | +| bucket | string | yes | - | The bucket address of obs file system, for example: `obs://obs-bucket-name`. | +| access_key | string | yes | - | The access key of obs file system. | +| access_secret | string | yes | - | The access secret of obs file system. | +| endpoint | string | yes | - | The endpoint of obs file system. | +| custom_filename | boolean | no | false | Whether you need custom the filename. | +| file_name_expression | string | no | "${transactionId}" | Describes the file expression which will be created into the `path`. Only used when custom_filename is true. [Tips](#file_name_expression) | +| filename_time_format | string | no | "yyyy.MM.dd" | Specify the time format of the `path`. Only used when custom_filename is true. [Tips](#filename_time_format) | +| file_format_type | string | no | "csv" | Supported file types. [Tips](#file_format_type) | +| field_delimiter | string | no | '\001' | The separator between columns in a row of data.Only used when file_format is text. | +| row_delimiter | string | no | "\n" | The separator between rows in a file. Only needed by `text`, `csv` and `json` file format. | +| have_partition | boolean | no | false | Whether you need processing partitions. | +| partition_by | array | no | - | Partition data based on selected fields. Only used then have_partition is true. | +| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true.[Tips](#partition_dir_expression) | +| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true.[Tips](#is_partition_field_write_in_file) | +| sink_columns | array | no | | When this parameter is empty, all fields are sink columns.[Tips](#sink_columns) | +| is_enable_transaction | boolean | no | true | [Tips](#is_enable_transaction) | +| batch_size | int | no | 1000000 | [Tips](#batch_size) | +| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | +| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | +| compress_codec | string | no | none | [Tips](#compress_codec) | +| common-options | object | no | - | [Tips](#common_options) | +| max_rows_in_memory | int | no | - | When File Format is Excel,The maximum number of data items that can be cached in the memory.Only used when file_format is excel. | +| sheet_name | string | no | Sheet${Random number} | Writer the sheet of the workbook. Only used when file_format is excel. | +| sheet_max_rows | int | no | 1048576 | Only used when file format_type is excel. | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json or maxwell_json. When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data | ### Tips @@ -170,9 +170,8 @@ Please note that excel type does not support any compression format #### merge_update_event -> Only used when file_format_type is canal_json,debezium_json,maxwell_json. -> set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; -> set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event +> Only used when file_format_type is canal_json,debezium_json or maxwell_json. +> When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data #### common options diff --git a/docs/en/connector-v2/sink/OssFile.md b/docs/en/connector-v2/sink/OssFile.md index 8b39e899dab..0d83d14af91 100644 --- a/docs/en/connector-v2/sink/OssFile.md +++ b/docs/en/connector-v2/sink/OssFile.md @@ -95,46 +95,46 @@ If write to `csv`, `text`, `json` file type, All column will be string. ## Options -| Name | Type | Required | Default | Description | -|---------------------------------------|---------|----------|--------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| path | string | yes | The oss path to write file in. | | -| tmp_path | string | no | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. Need a OSS dir. | -| bucket | string | yes | - | | -| access_key | string | yes | - | | -| access_secret | string | yes | - | | -| endpoint | string | yes | - | | -| custom_filename | boolean | no | false | Whether you need custom the filename | -| file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | -| filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | -| file_format_type | string | no | "csv" | | -| filename_extension | string | no | - | Override the default file name extensions with custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype` | -| field_delimiter | string | no | '\001' for text and ',' for csv | Only used when file_format_type is text and csv | -| row_delimiter | string | no | "\n" | Only used when file_format_type is `text`, `csv` and `json` | -| have_partition | boolean | no | false | Whether you need processing partitions. | -| partition_by | array | no | - | Only used then have_partition is true | -| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | -| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true | -| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | -| is_enable_transaction | boolean | no | true | | -| batch_size | int | no | 1000000 | | -| compress_codec | string | no | none | | -| common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | -| sheet_max_rows | int | no | 1048576 | Only used when file format_type is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | -| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | -| xml_root_tag | string | no | RECORDS | Only used when file_format is xml. | -| xml_row_tag | string | no | RECORD | Only used when file_format is xml. | -| xml_use_attr_format | boolean | no | - | Only used when file_format is xml. | -| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | -| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | -| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | -| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | -| enable_header_write | boolean | no | false | Only used when file_format_type is text,csv.
false:don't write header,true:write header. | -| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | -| schema_save_mode | Enum | no | CREATE_SCHEMA_WHEN_NOT_EXIST | Before turning on the synchronous task, do different treatment of the target path | -| data_save_mode | Enum | no | APPEND_DATA | Before opening the synchronous task, the data file in the target path is differently processed | -| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | +| Name | Type | Required | Default | Description | +|---------------------------------------|---------|----------|--------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| path | string | yes | The oss path to write file in. | | +| tmp_path | string | no | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. Need a OSS dir. | +| bucket | string | yes | - | | +| access_key | string | yes | - | | +| access_secret | string | yes | - | | +| endpoint | string | yes | - | | +| custom_filename | boolean | no | false | Whether you need custom the filename | +| file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | +| filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | +| file_format_type | string | no | "csv" | | +| filename_extension | string | no | - | Override the default file name extensions with custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype` | +| field_delimiter | string | no | '\001' for text and ',' for csv | Only used when file_format_type is text and csv | +| row_delimiter | string | no | "\n" | Only used when file_format_type is `text`, `csv` and `json` | +| have_partition | boolean | no | false | Whether you need processing partitions. | +| partition_by | array | no | - | Only used then have_partition is true | +| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | +| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true | +| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | +| is_enable_transaction | boolean | no | true | | +| batch_size | int | no | 1000000 | | +| compress_codec | string | no | none | | +| common-options | object | no | - | | +| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | +| sheet_max_rows | int | no | 1048576 | Only used when file format_type is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | +| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | +| xml_root_tag | string | no | RECORDS | Only used when file_format is xml. | +| xml_row_tag | string | no | RECORD | Only used when file_format is xml. | +| xml_use_attr_format | boolean | no | - | Only used when file_format is xml. | +| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | +| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | +| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | +| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | +| enable_header_write | boolean | no | false | Only used when file_format_type is text,csv.
false:don't write header,true:write header. | +| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | +| schema_save_mode | Enum | no | CREATE_SCHEMA_WHEN_NOT_EXIST | Before turning on the synchronous task, do different treatment of the target path | +| data_save_mode | Enum | no | APPEND_DATA | Before opening the synchronous task, the data file in the target path is differently processed | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json or maxwell_json. When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data | ### path [string] @@ -323,9 +323,8 @@ Option introduction: ### merge_update_event [boolean] -Only used when file_format_type is canal_json,debezium_json,maxwell_json. -set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; -set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event +Only used when file_format_type is canal_json,debezium_json or maxwell_json. +When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data ## How to Create an Oss Data Synchronization Jobs diff --git a/docs/en/connector-v2/sink/OssJindoFile.md b/docs/en/connector-v2/sink/OssJindoFile.md index b1e1f104e3b..c9d961d38da 100644 --- a/docs/en/connector-v2/sink/OssJindoFile.md +++ b/docs/en/connector-v2/sink/OssJindoFile.md @@ -47,43 +47,43 @@ It only supports hadoop version **2.9.X+**. ## Options -| Name | Type | Required | Default | Description | -|---------------------------------------|---------|----------|--------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| path | string | yes | - | | -| tmp_path | string | no | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. Need a OSS dir. | -| bucket | string | yes | - | | -| access_key | string | yes | - | | -| access_secret | string | yes | - | | -| endpoint | string | yes | - | | -| custom_filename | boolean | no | false | Whether you need custom the filename | -| file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | -| filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | -| file_format_type | string | no | "csv" | | -| filename_extension | string | no | - | Override the default file name extensions with custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype` | -| field_delimiter | string | no | '\001' for text and ',' for csv | Only used when file_format_type is text and csv | -| row_delimiter | string | no | "\n" | Only used when file_format_type is `text`, `csv` and `json` | -| have_partition | boolean | no | false | Whether you need processing partitions. | -| partition_by | array | no | - | Only used then have_partition is true | -| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | -| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true | -| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | -| is_enable_transaction | boolean | no | true | | -| batch_size | int | no | 1000000 | | -| compress_codec | string | no | none | | -| common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | -| sheet_max_rows | int | no | 1048576 | Only used when file_format_type is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | -| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | -| xml_root_tag | string | no | RECORDS | Only used when file_format is xml. | -| xml_row_tag | string | no | RECORD | Only used when file_format is xml. | -| xml_use_attr_format | boolean | no | - | Only used when file_format is xml. | -| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | -| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | -| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | -| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | -| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | -| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | +| Name | Type | Required | Default | Description | +|---------------------------------------|---------|----------|--------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| path | string | yes | - | | +| tmp_path | string | no | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. Need a OSS dir. | +| bucket | string | yes | - | | +| access_key | string | yes | - | | +| access_secret | string | yes | - | | +| endpoint | string | yes | - | | +| custom_filename | boolean | no | false | Whether you need custom the filename | +| file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | +| filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | +| file_format_type | string | no | "csv" | | +| filename_extension | string | no | - | Override the default file name extensions with custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype` | +| field_delimiter | string | no | '\001' for text and ',' for csv | Only used when file_format_type is text and csv | +| row_delimiter | string | no | "\n" | Only used when file_format_type is `text`, `csv` and `json` | +| have_partition | boolean | no | false | Whether you need processing partitions. | +| partition_by | array | no | - | Only used then have_partition is true | +| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | +| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true | +| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | +| is_enable_transaction | boolean | no | true | | +| batch_size | int | no | 1000000 | | +| compress_codec | string | no | none | | +| common-options | object | no | - | | +| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | +| sheet_max_rows | int | no | 1048576 | Only used when file_format_type is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | +| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | +| xml_root_tag | string | no | RECORDS | Only used when file_format is xml. | +| xml_row_tag | string | no | RECORD | Only used when file_format is xml. | +| xml_use_attr_format | boolean | no | - | Only used when file_format is xml. | +| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | +| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | +| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | +| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | +| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json or maxwell_json. When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data | ### path [string] @@ -255,9 +255,8 @@ The encoding of the file to write. This param will be parsed by `Charset.forName ### merge_update_event [boolean] -Only used when file_format_type is canal_json,debezium_json,maxwell_json. -set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; -set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event +Only used when file_format_type is canal_json,debezium_json or maxwell_json. +When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data ## Example diff --git a/docs/en/connector-v2/sink/S3File.md b/docs/en/connector-v2/sink/S3File.md index 84f13040ea4..cba1f446fa9 100644 --- a/docs/en/connector-v2/sink/S3File.md +++ b/docs/en/connector-v2/sink/S3File.md @@ -103,48 +103,48 @@ If write to `csv`, `text` file type, All column will be string. ## Sink Options -| name | type | required | default value | Description | -|---------------------------------------|---------|----------|-------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| path | string | yes | - | | -| tmp_path | string | no | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. Need a S3 dir. | -| bucket | string | yes | - | | -| fs.s3a.endpoint | string | yes | - | | -| fs.s3a.aws.credentials.provider | string | yes | com.amazonaws.auth.InstanceProfileCredentialsProvider | The way to authenticate s3a. We only support `org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider` and `com.amazonaws.auth.InstanceProfileCredentialsProvider` now. | -| access_key | string | no | - | Only used when fs.s3a.aws.credentials.provider = org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider | -| secret_key | string | no | - | Only used when fs.s3a.aws.credentials.provider = org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider | -| custom_filename | boolean | no | false | Whether you need custom the filename | -| file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | -| filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | -| file_format_type | string | no | "csv" | | -| filename_extension | string | no | - | Override the default file name extensions with custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype` | -| field_delimiter | string | no | '\001' for text and ',' for csv | Only used when file_format is text and csv | -| row_delimiter | string | no | "\n" | Only used when file_format is `text`, `csv` and `json` | -| have_partition | boolean | no | false | Whether you need processing partitions. | -| partition_by | array | no | - | Only used when have_partition is true | -| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used when have_partition is true | -| is_partition_field_write_in_file | boolean | no | false | Only used when have_partition is true | -| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | -| is_enable_transaction | boolean | no | true | | -| batch_size | int | no | 1000000 | | -| compress_codec | string | no | none | | -| common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format is excel. | -| sheet_max_rows | int | no | 1048576 | Only used when file_format is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format is excel. | -| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | -| xml_root_tag | string | no | RECORDS | Only used when file_format is xml, specifies the tag name of the root element within the XML file. | -| xml_row_tag | string | no | RECORD | Only used when file_format is xml, specifies the tag name of the data rows within the XML file | -| xml_use_attr_format | boolean | no | - | Only used when file_format is xml, specifies Whether to process data using the tag attribute format. | -| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | -| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | -| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | -| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | -| hadoop_s3_properties | map | no | | If you need to add a other option, you could add it here and refer to this [link](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html) | -| schema_save_mode | Enum | no | CREATE_SCHEMA_WHEN_NOT_EXIST | Before turning on the synchronous task, do different treatment of the target path | -| data_save_mode | Enum | no | APPEND_DATA | Before opening the synchronous task, the data file in the target path is differently processed | -| enable_header_write | boolean | no | false | Only used when file_format_type is text,csv.
false:don't write header,true:write header. | -| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | -| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | +| name | type | required | default value | Description | +|---------------------------------------|---------|----------|-------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| path | string | yes | - | | +| tmp_path | string | no | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. Need a S3 dir. | +| bucket | string | yes | - | | +| fs.s3a.endpoint | string | yes | - | | +| fs.s3a.aws.credentials.provider | string | yes | com.amazonaws.auth.InstanceProfileCredentialsProvider | The way to authenticate s3a. We only support `org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider` and `com.amazonaws.auth.InstanceProfileCredentialsProvider` now. | +| access_key | string | no | - | Only used when fs.s3a.aws.credentials.provider = org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider | +| secret_key | string | no | - | Only used when fs.s3a.aws.credentials.provider = org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider | +| custom_filename | boolean | no | false | Whether you need custom the filename | +| file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | +| filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | +| file_format_type | string | no | "csv" | | +| filename_extension | string | no | - | Override the default file name extensions with custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype` | +| field_delimiter | string | no | '\001' for text and ',' for csv | Only used when file_format is text and csv | +| row_delimiter | string | no | "\n" | Only used when file_format is `text`, `csv` and `json` | +| have_partition | boolean | no | false | Whether you need processing partitions. | +| partition_by | array | no | - | Only used when have_partition is true | +| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used when have_partition is true | +| is_partition_field_write_in_file | boolean | no | false | Only used when have_partition is true | +| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | +| is_enable_transaction | boolean | no | true | | +| batch_size | int | no | 1000000 | | +| compress_codec | string | no | none | | +| common-options | object | no | - | | +| max_rows_in_memory | int | no | - | Only used when file_format is excel. | +| sheet_max_rows | int | no | 1048576 | Only used when file_format is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format is excel. | +| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | +| xml_root_tag | string | no | RECORDS | Only used when file_format is xml, specifies the tag name of the root element within the XML file. | +| xml_row_tag | string | no | RECORD | Only used when file_format is xml, specifies the tag name of the data rows within the XML file | +| xml_use_attr_format | boolean | no | - | Only used when file_format is xml, specifies Whether to process data using the tag attribute format. | +| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | +| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | +| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | +| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | +| hadoop_s3_properties | map | no | | If you need to add a other option, you could add it here and refer to this [link](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html) | +| schema_save_mode | Enum | no | CREATE_SCHEMA_WHEN_NOT_EXIST | Before turning on the synchronous task, do different treatment of the target path | +| data_save_mode | Enum | no | APPEND_DATA | Before opening the synchronous task, the data file in the target path is differently processed | +| enable_header_write | boolean | no | false | Only used when file_format_type is text,csv.
false:don't write header,true:write header. | +| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json or maxwell_json. When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data | ### path [string] @@ -328,9 +328,8 @@ The encoding of the file to write. This param will be parsed by `Charset.forName ### merge_update_event [boolean] -Only used when file_format_type is canal_json,debezium_json,maxwell_json. -set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; -set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event +Only used when file_format_type is canal_json,debezium_json or maxwell_json. +When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data ## Example diff --git a/docs/en/connector-v2/sink/SftpFile.md b/docs/en/connector-v2/sink/SftpFile.md index 50827794c95..b0cedd1749a 100644 --- a/docs/en/connector-v2/sink/SftpFile.md +++ b/docs/en/connector-v2/sink/SftpFile.md @@ -41,46 +41,46 @@ If you use SeaTunnel Engine, It automatically integrated the hadoop jar when you ## Options -| name | type | required | default value | remarks | -|---------------------------------------|---------|----------|--------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| host | string | yes | - | | -| port | int | yes | - | | -| user | string | yes | - | | -| password | string | yes | - | | -| path | string | yes | - | | -| tmp_path | string | yes | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. Need a FTP dir. | -| custom_filename | boolean | no | false | Whether you need custom the filename | -| file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | -| filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | -| file_format_type | string | no | "csv" | | -| filename_extension | string | no | - | Override the default file name extensions with custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype` | -| field_delimiter | string | no | '\001' for text and ',' for csv | Only used when file_format_type is text and csv | -| row_delimiter | string | no | "\n" | Only used when file_format_type is `text`, `csv` and `json` | -| have_partition | boolean | no | false | Whether you need processing partitions. | -| partition_by | array | no | - | Only used then have_partition is true | -| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | -| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true | -| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | -| is_enable_transaction | boolean | no | true | | -| batch_size | int | no | 1000000 | | -| compress_codec | string | no | none | | -| common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | -| sheet_max_rows | int | no | 1048576 | Only used when file_format_type is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | -| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | -| xml_root_tag | string | no | RECORDS | Only used when file_format is xml. | -| xml_row_tag | string | no | RECORD | Only used when file_format is xml. | -| xml_use_attr_format | boolean | no | - | Only used when file_format is xml. | -| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | -| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | -| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | -| enable_header_write | boolean | no | false | Only used when file_format_type is text,csv.
false:don't write header,true:write header. | -| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | -| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | -| schema_save_mode | string | no | CREATE_SCHEMA_WHEN_NOT_EXIST | Existing dir processing method | -| data_save_mode | string | no | APPEND_DATA | Existing data processing method | -| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json,maxwell_json.set true,when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data | +| name | type | required | default value | remarks | +|---------------------------------------|---------|----------|--------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| host | string | yes | - | | +| port | int | yes | - | | +| user | string | yes | - | | +| password | string | yes | - | | +| path | string | yes | - | | +| tmp_path | string | yes | /tmp/seatunnel | The result file will write to a tmp path first and then use `mv` to submit tmp dir to target dir. Need a FTP dir. | +| custom_filename | boolean | no | false | Whether you need custom the filename | +| file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | +| filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | +| file_format_type | string | no | "csv" | | +| filename_extension | string | no | - | Override the default file name extensions with custom file name extensions. E.g. `.xml`, `.json`, `dat`, `.customtype` | +| field_delimiter | string | no | '\001' for text and ',' for csv | Only used when file_format_type is text and csv | +| row_delimiter | string | no | "\n" | Only used when file_format_type is `text`, `csv` and `json` | +| have_partition | boolean | no | false | Whether you need processing partitions. | +| partition_by | array | no | - | Only used then have_partition is true | +| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | +| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true | +| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | +| is_enable_transaction | boolean | no | true | | +| batch_size | int | no | 1000000 | | +| compress_codec | string | no | none | | +| common-options | object | no | - | | +| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | +| sheet_max_rows | int | no | 1048576 | Only used when file_format_type is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | +| csv_string_quote_mode | enum | no | MINIMAL | Only used when file_format is csv. | +| xml_root_tag | string | no | RECORDS | Only used when file_format is xml. | +| xml_row_tag | string | no | RECORD | Only used when file_format is xml. | +| xml_use_attr_format | boolean | no | - | Only used when file_format is xml. | +| single_file_mode | boolean | no | false | Each parallelism will only output one file. When this parameter is turned on, batch_size will not take effect. The output file name does not have a file block suffix. | +| create_empty_file_when_no_data | boolean | no | false | When there is no data synchronization upstream, the corresponding data files are still generated. | +| parquet_avro_write_timestamp_as_int96 | boolean | no | false | Only used when file_format is parquet. | +| enable_header_write | boolean | no | false | Only used when file_format_type is text,csv.
false:don't write header,true:write header. | +| parquet_avro_write_fixed_as_int96 | array | no | - | Only used when file_format is parquet. | +| encoding | string | no | "UTF-8" | Only used when file_format_type is json,text,csv,xml. | +| schema_save_mode | string | no | CREATE_SCHEMA_WHEN_NOT_EXIST | Existing dir processing method | +| data_save_mode | string | no | APPEND_DATA | Existing data processing method | +| merge_update_event | boolean | no | false | Only used when file_format_type is canal_json,debezium_json or maxwell_json. When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data | ### host [string] @@ -273,9 +273,8 @@ Existing data processing method. ### merge_update_event [boolean] -Only used when file_format_type is canal_json,debezium_json,maxwell_json. -set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data; -set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event +Only used when file_format_type is canal_json,debezium_json or maxwell_json. +When value is true, the UPDATE_AFTER and UPDATE_BEFORE event will be merged into UPDATE event data ## Example From 76647a5674460f91377800ffe8ac6932c65c9d99 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Fri, 5 Sep 2025 09:51:26 +0800 Subject: [PATCH 22/37] feat:update --- .../file/sink/writer/CanalJsonWriteStrategy.java | 9 +++------ .../sink/writer/DebeziumJsonWriteStrategy.java | 9 +++------ .../sink/writer/MaxWellJsonWriteStrategy.java | 9 +++------ .../format/json/canal/CanalJsonFormatOptions.java | 15 --------------- .../json/canal/CanalJsonSerializationSchema.java | 6 +++--- .../json/debezium/DebeziumJsonFormatOptions.java | 14 -------------- .../debezium/DebeziumJsonSerializationSchema.java | 6 +++--- .../json/maxwell/MaxWellJsonFormatOptions.java | 15 --------------- .../maxwell/MaxWellJsonSerializationSchema.java | 8 ++++---- .../format/json/ogg/OggJsonFormatOptions.java | 15 --------------- .../json/ogg/OggJsonSerializationSchema.java | 8 ++++---- .../json/canal/CanalJsonSerDeSchemaTest.java | 4 +--- .../debezium/DebeziumJsonSerDeSchemaTest.java | 4 +--- .../json/maxwell/MaxWellJsonSerDeSchemaTest.java | 4 +--- .../format/json/ogg/OggJsonSerDeSchemaTest.java | 4 +--- 15 files changed, 27 insertions(+), 103 deletions(-) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/CanalJsonWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/CanalJsonWriteStrategy.java index 9a5d8ef07bd..a53b076fece 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/CanalJsonWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/CanalJsonWriteStrategy.java @@ -46,7 +46,7 @@ public class CanalJsonWriteStrategy extends AbstractWriteStrategy beingWrittenOutputStream; private final Map isFirstWrite; private final Charset charset; - private final Map options; + private final boolean mergeUpdateEventFlag; public CanalJsonWriteStrategy(FileSinkConfig textFileSinkConfig) { super(textFileSinkConfig); @@ -54,10 +54,7 @@ public CanalJsonWriteStrategy(FileSinkConfig textFileSinkConfig) { this.isFirstWrite = new HashMap<>(); this.charset = EncodingUtils.tryParseCharset(textFileSinkConfig.getEncoding()); this.rowDelimiter = textFileSinkConfig.getRowDelimiter().getBytes(charset); - this.options = new HashMap<>(); - this.options.put( - CanalJsonFormatOptions.MERGE_UPDATE_EVENT.key(), - textFileSinkConfig.getMergeUpdateEvent().toString()); + this.mergeUpdateEventFlag = textFileSinkConfig.getMergeUpdateEvent(); } @Override @@ -68,7 +65,7 @@ public void setCatalogTable(CatalogTable catalogTable) { buildSchemaWithRowType( catalogTable.getSeaTunnelRowType(), sinkColumnsIndexInRow), charset, - options); + mergeUpdateEventFlag); } @Override diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/DebeziumJsonWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/DebeziumJsonWriteStrategy.java index cdeb43872f8..60982c1e3d6 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/DebeziumJsonWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/DebeziumJsonWriteStrategy.java @@ -46,7 +46,7 @@ public class DebeziumJsonWriteStrategy extends AbstractWriteStrategy beingWrittenOutputStream; private final Map isFirstWrite; private final Charset charset; - private final Map options; + private final boolean mergeUpdateEventFlag; public DebeziumJsonWriteStrategy(FileSinkConfig textFileSinkConfig) { super(textFileSinkConfig); @@ -54,10 +54,7 @@ public DebeziumJsonWriteStrategy(FileSinkConfig textFileSinkConfig) { this.isFirstWrite = new HashMap<>(); this.charset = EncodingUtils.tryParseCharset(textFileSinkConfig.getEncoding()); this.rowDelimiter = textFileSinkConfig.getRowDelimiter().getBytes(charset); - this.options = new HashMap<>(); - this.options.put( - DebeziumJsonFormatOptions.MERGE_UPDATE_EVENT.key(), - textFileSinkConfig.getMergeUpdateEvent().toString()); + this.mergeUpdateEventFlag = textFileSinkConfig.getMergeUpdateEvent(); } @Override @@ -68,7 +65,7 @@ public void setCatalogTable(CatalogTable catalogTable) { buildSchemaWithRowType( catalogTable.getSeaTunnelRowType(), sinkColumnsIndexInRow), charset, - options); + mergeUpdateEventFlag); } @Override diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/MaxWellJsonWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/MaxWellJsonWriteStrategy.java index 3d185144e53..c04a5cc593c 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/MaxWellJsonWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/MaxWellJsonWriteStrategy.java @@ -46,7 +46,7 @@ public class MaxWellJsonWriteStrategy extends AbstractWriteStrategy beingWrittenOutputStream; private final Map isFirstWrite; private final Charset charset; - private final Map options; + private final boolean mergeUpdateEventFlag; public MaxWellJsonWriteStrategy(FileSinkConfig textFileSinkConfig) { super(textFileSinkConfig); @@ -54,10 +54,7 @@ public MaxWellJsonWriteStrategy(FileSinkConfig textFileSinkConfig) { this.isFirstWrite = new HashMap<>(); this.charset = EncodingUtils.tryParseCharset(textFileSinkConfig.getEncoding()); this.rowDelimiter = textFileSinkConfig.getRowDelimiter().getBytes(charset); - this.options = new HashMap<>(); - this.options.put( - MaxWellJsonFormatOptions.MERGE_UPDATE_EVENT.key(), - textFileSinkConfig.getMergeUpdateEvent().toString()); + this.mergeUpdateEventFlag = textFileSinkConfig.getMergeUpdateEvent(); } @Override @@ -68,7 +65,7 @@ public void setCatalogTable(CatalogTable catalogTable) { buildSchemaWithRowType( catalogTable.getSeaTunnelRowType(), sinkColumnsIndexInRow), charset, - options); + mergeUpdateEventFlag); } @Override diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonFormatOptions.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonFormatOptions.java index 09d946bb61e..fe0f8fd18ce 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonFormatOptions.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonFormatOptions.java @@ -44,15 +44,6 @@ public class CanalJsonFormatOptions { .withDescription( "An optional regular expression to only read the specific tables changelog rows by regular matching the \"table\" meta field in the Canal record." + "The pattern string is compatible with Java's Pattern."); - - public static final Option MERGE_UPDATE_EVENT = - Options.key("merge_update_event") - .booleanType() - .defaultValue(false) - .withDescription( - "set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data;" - + "if set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event "); - public static String getTableInclude(Map options) { return options.getOrDefault(TABLE_INCLUDE.key(), null); } @@ -65,10 +56,4 @@ public static boolean getIgnoreParseErrors(Map options) { return Boolean.parseBoolean( options.getOrDefault(IGNORE_PARSE_ERRORS.key(), IGNORE_PARSE_ERRORS.toString())); } - - public static boolean getMergeUpdateEvent(Map options) { - return Boolean.parseBoolean( - options.getOrDefault( - MERGE_UPDATE_EVENT.key(), MERGE_UPDATE_EVENT.defaultValue().toString())); - } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java index 1924f677817..a92e2203bb5 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java @@ -59,14 +59,14 @@ public class CanalJsonSerializationSchema implements SerializationSchema { public CanalJsonSerializationSchema(SeaTunnelRowType rowType) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType)); this.reuse = new SeaTunnelRow(6); - mergeUpdateEventFlag = CanalJsonFormatOptions.MERGE_UPDATE_EVENT.defaultValue(); + mergeUpdateEventFlag = false; } public CanalJsonSerializationSchema( - SeaTunnelRowType rowType, Charset charset, Map options) { + SeaTunnelRowType rowType, Charset charset, boolean mergeUpdateEventFlag) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType), charset); this.reuse = new SeaTunnelRow(6); - mergeUpdateEventFlag = CanalJsonFormatOptions.getMergeUpdateEvent(options); + this.mergeUpdateEventFlag = mergeUpdateEventFlag; } @Override diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatOptions.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatOptions.java index 61a5edac1bc..43cab50c9d4 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatOptions.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatOptions.java @@ -39,14 +39,6 @@ public class DebeziumJsonFormatOptions { + "This option indicates the Debezium JSON data include the schema in the message or not. " + "Default is false."); - public static final Option MERGE_UPDATE_EVENT = - Options.key("merge_update_event") - .booleanType() - .defaultValue(false) - .withDescription( - "set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data;" - + "if set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event "); - public static boolean getSchemaInclude(Map options) { return Boolean.parseBoolean( options.getOrDefault( @@ -58,10 +50,4 @@ public static boolean getIgnoreParseErrors(Map options) { options.getOrDefault( IGNORE_PARSE_ERRORS.key(), IGNORE_PARSE_ERRORS.defaultValue().toString())); } - - public static boolean getMergeUpdateEvent(Map options) { - return Boolean.parseBoolean( - options.getOrDefault( - MERGE_UPDATE_EVENT.key(), MERGE_UPDATE_EVENT.defaultValue().toString())); - } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerializationSchema.java index 35694b54eba..ce0bd965551 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerializationSchema.java @@ -57,14 +57,14 @@ public class DebeziumJsonSerializationSchema implements SerializationSchema { public DebeziumJsonSerializationSchema(SeaTunnelRowType rowType) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType)); this.genericRow = new SeaTunnelRow(GENERATE_ROW_SIZE); - mergeUpdateEventFlag = DebeziumJsonFormatOptions.MERGE_UPDATE_EVENT.defaultValue(); + this.mergeUpdateEventFlag = false; } public DebeziumJsonSerializationSchema( - SeaTunnelRowType rowType, Charset charset, Map options) { + SeaTunnelRowType rowType, Charset charset, boolean mergeUpdateEventFlag) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType), charset); this.genericRow = new SeaTunnelRow(GENERATE_ROW_SIZE); - this.mergeUpdateEventFlag = DebeziumJsonFormatOptions.getMergeUpdateEvent(options); + this.mergeUpdateEventFlag = mergeUpdateEventFlag; } @Override diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonFormatOptions.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonFormatOptions.java index 830437a7bf5..175691ff1aa 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonFormatOptions.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonFormatOptions.java @@ -44,15 +44,6 @@ public class MaxWellJsonFormatOptions { .withDescription( "An optional regular expression to only read the specific tables changelog rows by regular matching the \"table\" meta field in the MaxWell record." + "The pattern string is compatible with Java's Pattern."); - - public static final Option MERGE_UPDATE_EVENT = - Options.key("merge_update_event") - .booleanType() - .defaultValue(false) - .withDescription( - "set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data;" - + "if set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event "); - public static String getTableInclude(Map options) { return options.getOrDefault(TABLE_INCLUDE.key(), null); } @@ -65,10 +56,4 @@ public static boolean getIgnoreParseErrors(Map options) { return Boolean.parseBoolean( options.getOrDefault(IGNORE_PARSE_ERRORS.key(), IGNORE_PARSE_ERRORS.toString())); } - - public static boolean getMergeUpdateEvent(Map options) { - return Boolean.parseBoolean( - options.getOrDefault( - MERGE_UPDATE_EVENT.key(), MERGE_UPDATE_EVENT.defaultValue().toString())); - } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java index 44f8fde4973..60de2cc8879 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java @@ -52,20 +52,20 @@ public class MaxWellJsonSerializationSchema implements SerializationSchema { private final JsonSerializationSchema jsonSerializer; - boolean mergeUpdateEventFlag; + private final boolean mergeUpdateEventFlag; SeaTunnelRow cacheUpdateBeforeRow; public MaxWellJsonSerializationSchema(SeaTunnelRowType rowType) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType)); this.reuse = new SeaTunnelRow(6); - this.mergeUpdateEventFlag = MaxWellJsonFormatOptions.MERGE_UPDATE_EVENT.defaultValue(); + this.mergeUpdateEventFlag = false; } public MaxWellJsonSerializationSchema( - SeaTunnelRowType rowType, Charset charset, Map options) { + SeaTunnelRowType rowType, Charset charset, boolean mergeUpdateEventFlag) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType), charset); this.reuse = new SeaTunnelRow(6); - this.mergeUpdateEventFlag = MaxWellJsonFormatOptions.getMergeUpdateEvent(options); + this.mergeUpdateEventFlag = mergeUpdateEventFlag; } @Override diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonFormatOptions.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonFormatOptions.java index 9e62840c40a..d402cc8ee50 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonFormatOptions.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonFormatOptions.java @@ -44,15 +44,6 @@ public class OggJsonFormatOptions { .withDescription( "An optional regular expression to only read the specific tables changelog rows by regular matching the \"table\" meta field in the Ogg record." + "The pattern string is compatible with Java's Pattern."); - - public static final Option MERGE_UPDATE_EVENT = - Options.key("merge_update_event") - .booleanType() - .defaultValue(false) - .withDescription( - "set true,then when serialize data,UPDATE_AFTER and UPDATE_BEFORE event will merge into UPDATE data;" - + "if set false, when serialize data will get UPDATE_AFTER and UPDATE_BEFORE event "); - public static String getTableInclude(Map options) { return options.getOrDefault(TABLE_INCLUDE.key(), null); } @@ -65,10 +56,4 @@ public static boolean getIgnoreParseErrors(Map options) { return Boolean.parseBoolean( options.getOrDefault(IGNORE_PARSE_ERRORS.key(), IGNORE_PARSE_ERRORS.toString())); } - - public static boolean getMergeUpdateEvent(Map options) { - return Boolean.parseBoolean( - options.getOrDefault( - MERGE_UPDATE_EVENT.key(), MERGE_UPDATE_EVENT.defaultValue().toString())); - } } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java index e69231f8474..3edcb4f26bc 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java @@ -50,20 +50,20 @@ public class OggJsonSerializationSchema implements SerializationSchema { private final JsonSerializationSchema jsonSerializer; - boolean mergeUpdateEventFlag; + private final boolean mergeUpdateEventFlag; SeaTunnelRow cacheUpdateBeforeRow; public OggJsonSerializationSchema(SeaTunnelRowType rowType) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType)); this.reuse = new SeaTunnelRow(5); - mergeUpdateEventFlag = OggJsonFormatOptions.MERGE_UPDATE_EVENT.defaultValue(); + mergeUpdateEventFlag = false; } public OggJsonSerializationSchema( - SeaTunnelRowType rowType, Charset charset, Map options) { + SeaTunnelRowType rowType, Charset charset, boolean mergeUpdateEventFlag) { this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType), charset); this.reuse = new SeaTunnelRow(5); - mergeUpdateEventFlag = OggJsonFormatOptions.getMergeUpdateEvent(options); + this.mergeUpdateEventFlag = mergeUpdateEventFlag; } @Override 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 bd851b1f072..e564860174a 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 @@ -238,11 +238,9 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali assertEquals(expectedResult, result); // test merge_update_event - Map options = new HashMap<>(); - options.put(CanalJsonFormatOptions.MERGE_UPDATE_EVENT.key(), "true"); serializationSchema = new CanalJsonSerializationSchema( - SEATUNNEL_ROW_TYPE, StandardCharsets.UTF_8, options); + SEATUNNEL_ROW_TYPE, StandardCharsets.UTF_8, true); result.clear(); for (SeaTunnelRow rowData : collector.list) { if (serializationSchema.serialize(rowData) != null) { 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 47acdd4066e..05432fe1c6b 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 @@ -641,11 +641,9 @@ private void testSerializationDeserialization(String resourceFile, boolean schem assertEquals(expected, actual); // test merge_update_event - Map options = new HashMap<>(); - options.put(DebeziumJsonFormatOptions.MERGE_UPDATE_EVENT.key(), "true"); serializationSchema = new DebeziumJsonSerializationSchema( - SEATUNNEL_ROW_TYPE, StandardCharsets.UTF_8, options); + SEATUNNEL_ROW_TYPE, StandardCharsets.UTF_8, true); actual.clear(); for (SeaTunnelRow rowData : collector.list) { if (serializationSchema.serialize(rowData) != null) { diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java index 0832df85318..489bb7e0b8a 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java @@ -152,11 +152,9 @@ public void runTest(List lines, MaxWellJsonDeserializationSchema deseria assertEquals(expectedResult, result); // test merge_update_event - Map options = new HashMap<>(); - options.put(MaxWellJsonFormatOptions.MERGE_UPDATE_EVENT.key(), "true"); serializationSchema = new MaxWellJsonSerializationSchema( - catalogTables.getSeaTunnelRowType(), StandardCharsets.UTF_8, options); + catalogTables.getSeaTunnelRowType(), StandardCharsets.UTF_8, true); actual.clear(); for (SeaTunnelRow rowData : collector.list) { if (serializationSchema.serialize(rowData) != null) { 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 9abff0d726e..640e419b952 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 @@ -233,10 +233,8 @@ public void runTest(List lines, OggJsonDeserializationSchema deserializa assertEquals(expectedResult, result); // test merge_update_event - Map options = new HashMap<>(); - options.put(OggJsonFormatOptions.MERGE_UPDATE_EVENT.key(), "true"); serializationSchema = - new OggJsonSerializationSchema(SEATUNNEL_ROW_TYPE, StandardCharsets.UTF_8, options); + new OggJsonSerializationSchema(SEATUNNEL_ROW_TYPE, StandardCharsets.UTF_8, true); actual.clear(); for (SeaTunnelRow rowData : collector.list) { if (serializationSchema.serialize(rowData) != null) { From 79e9efe79c35e90ddf9650a96f5014ed2ae967b4 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Fri, 5 Sep 2025 09:59:13 +0800 Subject: [PATCH 23/37] feat:update --- .../seatunnel/file/sink/writer/CanalJsonWriteStrategy.java | 1 - .../file/sink/writer/DebeziumJsonWriteStrategy.java | 1 - .../seatunnel/file/sink/writer/MaxWellJsonWriteStrategy.java | 1 - .../seatunnel/format/json/canal/CanalJsonFormatOptions.java | 1 + .../format/json/canal/CanalJsonSerializationSchema.java | 1 - .../format/json/maxwell/MaxWellJsonFormatOptions.java | 1 + .../format/json/maxwell/MaxWellJsonSerializationSchema.java | 1 - .../seatunnel/format/json/ogg/OggJsonFormatOptions.java | 1 + .../format/json/ogg/OggJsonSerializationSchema.java | 1 - .../format/json/canal/CanalJsonSerDeSchemaTest.java | 5 +---- .../format/json/debezium/DebeziumJsonSerDeSchemaTest.java | 2 -- .../format/json/maxwell/MaxWellJsonSerDeSchemaTest.java | 2 -- .../seatunnel/format/json/ogg/OggJsonSerDeSchemaTest.java | 2 -- 13 files changed, 4 insertions(+), 16 deletions(-) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/CanalJsonWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/CanalJsonWriteStrategy.java index a53b076fece..f40e2fce907 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/CanalJsonWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/CanalJsonWriteStrategy.java @@ -25,7 +25,6 @@ import org.apache.seatunnel.common.utils.EncodingUtils; import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; import org.apache.seatunnel.connectors.seatunnel.file.sink.config.FileSinkConfig; -import org.apache.seatunnel.format.json.canal.CanalJsonFormatOptions; import org.apache.seatunnel.format.json.canal.CanalJsonSerializationSchema; import org.apache.hadoop.fs.FSDataOutputStream; diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/DebeziumJsonWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/DebeziumJsonWriteStrategy.java index 60982c1e3d6..5aa0a864453 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/DebeziumJsonWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/DebeziumJsonWriteStrategy.java @@ -25,7 +25,6 @@ import org.apache.seatunnel.common.utils.EncodingUtils; import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; import org.apache.seatunnel.connectors.seatunnel.file.sink.config.FileSinkConfig; -import org.apache.seatunnel.format.json.debezium.DebeziumJsonFormatOptions; import org.apache.seatunnel.format.json.debezium.DebeziumJsonSerializationSchema; import org.apache.hadoop.fs.FSDataOutputStream; diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/MaxWellJsonWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/MaxWellJsonWriteStrategy.java index c04a5cc593c..693e07fc1e4 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/MaxWellJsonWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/MaxWellJsonWriteStrategy.java @@ -25,7 +25,6 @@ import org.apache.seatunnel.common.utils.EncodingUtils; import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; import org.apache.seatunnel.connectors.seatunnel.file.sink.config.FileSinkConfig; -import org.apache.seatunnel.format.json.maxwell.MaxWellJsonFormatOptions; import org.apache.seatunnel.format.json.maxwell.MaxWellJsonSerializationSchema; import org.apache.hadoop.fs.FSDataOutputStream; diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonFormatOptions.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonFormatOptions.java index fe0f8fd18ce..3af85af1d06 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonFormatOptions.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonFormatOptions.java @@ -44,6 +44,7 @@ public class CanalJsonFormatOptions { .withDescription( "An optional regular expression to only read the specific tables changelog rows by regular matching the \"table\" meta field in the Canal record." + "The pattern string is compatible with Java's Pattern."); + public static String getTableInclude(Map options) { return options.getOrDefault(TABLE_INCLUDE.key(), null); } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java index a92e2203bb5..0d03dfd06ad 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/canal/CanalJsonSerializationSchema.java @@ -33,7 +33,6 @@ import org.apache.commons.lang3.StringUtils; import java.nio.charset.Charset; -import java.util.Map; import static org.apache.seatunnel.api.table.type.BasicType.LONG_TYPE; import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonFormatOptions.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonFormatOptions.java index 175691ff1aa..0b6a5cd2879 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonFormatOptions.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonFormatOptions.java @@ -44,6 +44,7 @@ public class MaxWellJsonFormatOptions { .withDescription( "An optional regular expression to only read the specific tables changelog rows by regular matching the \"table\" meta field in the MaxWell record." + "The pattern string is compatible with Java's Pattern."); + public static String getTableInclude(Map options) { return options.getOrDefault(TABLE_INCLUDE.key(), null); } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java index 60de2cc8879..d1d25e15d85 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerializationSchema.java @@ -32,7 +32,6 @@ import org.apache.commons.lang3.StringUtils; import java.nio.charset.Charset; -import java.util.Map; import static org.apache.seatunnel.api.table.type.BasicType.LONG_TYPE; import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonFormatOptions.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonFormatOptions.java index d402cc8ee50..462ccf8aa65 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonFormatOptions.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonFormatOptions.java @@ -44,6 +44,7 @@ public class OggJsonFormatOptions { .withDescription( "An optional regular expression to only read the specific tables changelog rows by regular matching the \"table\" meta field in the Ogg record." + "The pattern string is compatible with Java's Pattern."); + public static String getTableInclude(Map options) { return options.getOrDefault(TABLE_INCLUDE.key(), null); } diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java index 3edcb4f26bc..8216ab0e400 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/ogg/OggJsonSerializationSchema.java @@ -31,7 +31,6 @@ import org.apache.commons.lang3.StringUtils; import java.nio.charset.Charset; -import java.util.Map; import static org.apache.seatunnel.api.table.type.BasicType.LONG_TYPE; import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; 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 e564860174a..236bccc7d60 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 @@ -37,9 +37,7 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import static org.apache.seatunnel.api.table.type.BasicType.FLOAT_TYPE; @@ -239,8 +237,7 @@ public void runTest(List lines, CanalJsonDeserializationSchema deseriali // test merge_update_event serializationSchema = - new CanalJsonSerializationSchema( - SEATUNNEL_ROW_TYPE, StandardCharsets.UTF_8, true); + new CanalJsonSerializationSchema(SEATUNNEL_ROW_TYPE, StandardCharsets.UTF_8, true); result.clear(); for (SeaTunnelRow rowData : collector.list) { if (serializationSchema.serialize(rowData) != null) { 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 05432fe1c6b..fd3cbdf5614 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 @@ -43,9 +43,7 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import static org.apache.seatunnel.api.table.type.BasicType.BOOLEAN_TYPE; diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java index 489bb7e0b8a..9a481da23a6 100644 --- a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/maxwell/MaxWellJsonSerDeSchemaTest.java @@ -35,9 +35,7 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import static org.apache.seatunnel.api.table.type.BasicType.FLOAT_TYPE; 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 640e419b952..401ed655c5f 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 @@ -38,9 +38,7 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import static org.apache.seatunnel.api.table.type.BasicType.FLOAT_TYPE; From 6edb142cb9df4517d32768e57fcb0176492a100e Mon Sep 17 00:00:00 2001 From: dyp12 Date: Wed, 26 Nov 2025 18:30:56 +0800 Subject: [PATCH 24/37] fead:add fluss sink --- plugin-mapping.properties | 1 + .../connector-fluss/pom.xml | 37 ++ .../fluss/config/FlussBaseOptions.java | 49 +++ .../fluss/config/FlussSinkOptions.java | 20 + .../seatunnel/fluss/sink/FlussSink.java | 52 +++ .../fluss/sink/FlussSinkFactory.java | 50 +++ .../seatunnel/fluss/sink/FlussSinkWriter.java | 218 ++++++++++ seatunnel-connectors-v2/pom.xml | 1 + seatunnel-dist/pom.xml | 7 + .../connector-fluss-e2e/pom.xml | 68 +++ .../e2e/connector/fluss/FlussSinkIT.java | 410 ++++++++++++++++++ .../src/test/resources/fake_to_fluss.conf | 96 ++++ .../fake_to_multipletable_fluss.conf | 200 +++++++++ .../seatunnel-connector-v2-e2e/pom.xml | 1 + .../format/json/JsonToRowConverters.java | 13 + .../format/json/RowToJsonConverters.java | 10 + .../json/JsonRowDataSerDeSchemaTest.java | 47 +- 17 files changed, 1266 insertions(+), 14 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-fluss/pom.xml create mode 100644 seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/config/FlussBaseOptions.java create mode 100644 seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/config/FlussSinkOptions.java create mode 100644 seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java create mode 100644 seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkFactory.java create mode 100644 seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkWriter.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/resources/fake_to_fluss.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/resources/fake_to_multipletable_fluss.conf diff --git a/plugin-mapping.properties b/plugin-mapping.properties index 8a604a04773..0c27ae415b8 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -149,6 +149,7 @@ seatunnel.source.GraphQL = connector-graphql seatunnel.sink.GraphQL = connector-graphql seatunnel.sink.Aerospike = connector-aerospike seatunnel.sink.SensorsData = connector-sensorsdata +seatunnel.sink.Fluss = connector-fluss # For custom transforms, make sure to use the seatunnel.transform.[PluginIdentifier]=[JarPerfix] naming convention. For example: # seatunnel.transform.Sql = seatunnel-transforms-v2 diff --git a/seatunnel-connectors-v2/connector-fluss/pom.xml b/seatunnel-connectors-v2/connector-fluss/pom.xml new file mode 100644 index 00000000000..d2335444625 --- /dev/null +++ b/seatunnel-connectors-v2/connector-fluss/pom.xml @@ -0,0 +1,37 @@ + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connectors-v2 + ${revision} + + + connector-fluss + SeaTunnel : Connectors V2 : Fluss + + + 0.7.0 + connector.fluss + + + + + org.apache.seatunnel + seatunnel-api + ${project.version} + + + org.apache.seatunnel + connector-common + ${project.version} + + + com.alibaba.fluss + fluss-client + ${fluss.client.version} + + + + diff --git a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/config/FlussBaseOptions.java b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/config/FlussBaseOptions.java new file mode 100644 index 00000000000..e582c760ec3 --- /dev/null +++ b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/config/FlussBaseOptions.java @@ -0,0 +1,49 @@ +/* + * 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.fluss.config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +import java.io.Serializable; +import java.util.Map; + +public class FlussBaseOptions implements Serializable { + public static final String CONNECTOR_IDENTITY = "Fluss"; + public static final Option BOOTSTRAP_SERVERS = + Options.key("bootstrap.servers") + .stringType() + .noDefaultValue() + .withDescription("Fluss cluster address"); + public static final Option DATABASE = + Options.key("database") + .stringType() + .noDefaultValue() + .withDescription("The name of Fluss database"); + + public static final Option TABLE = + Options.key("table") + .stringType() + .noDefaultValue() + .withDescription("The name of Fluss table"); + + public static final Option> CLIENT_CONFIG = + Options.key("client.config") + .mapType() + .noDefaultValue() + .withDescription("The parameter of Fluss client add to Connection "); +} diff --git a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/config/FlussSinkOptions.java b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/config/FlussSinkOptions.java new file mode 100644 index 00000000000..293770eba96 --- /dev/null +++ b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/config/FlussSinkOptions.java @@ -0,0 +1,20 @@ +/* + * 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.fluss.config; + +public class FlussSinkOptions extends FlussBaseOptions {} diff --git a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java new file mode 100644 index 00000000000..f3a9be38570 --- /dev/null +++ b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java @@ -0,0 +1,52 @@ +/* + * 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.fluss.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.sink.SupportMultiTableSink; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; +import org.apache.seatunnel.connectors.seatunnel.fluss.config.FlussSinkOptions; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class FlussSink extends AbstractSimpleSink + implements SupportMultiTableSink { + + private final ReadonlyConfig pluginConfig; + private final CatalogTable catalogTable; + + public FlussSink(ReadonlyConfig pluginConfig, CatalogTable catalogTable) { + this.pluginConfig = pluginConfig; + this.catalogTable = catalogTable; + } + + @Override + public AbstractSinkWriter createWriter(SinkWriter.Context context) { + return new FlussSinkWriter(context, catalogTable, pluginConfig); + } + + @Override + public String getPluginName() { + return FlussSinkOptions.CONNECTOR_IDENTITY; + } +} diff --git a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkFactory.java b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkFactory.java new file mode 100644 index 00000000000..f7cc8c8d9c0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkFactory.java @@ -0,0 +1,50 @@ +/* + * 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.fluss.sink; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.connector.TableSink; +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.fluss.config.FlussSinkOptions; + +import com.google.auto.service.AutoService; + +@AutoService(Factory.class) +public class FlussSinkFactory implements TableSinkFactory { + @Override + public String factoryIdentifier() { + return FlussSinkOptions.CONNECTOR_IDENTITY; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required(FlussSinkOptions.BOOTSTRAP_SERVERS) + .optional(FlussSinkOptions.DATABASE) + .optional(FlussSinkOptions.TABLE) + .optional(FlussSinkOptions.CLIENT_CONFIG) + .build(); + } + + @Override + public TableSink createSink(TableSinkFactoryContext context) { + return () -> new FlussSink(context.getOptions(), context.getCatalogTable()); + } +} diff --git a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkWriter.java b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkWriter.java new file mode 100644 index 00000000000..92a85ba5a00 --- /dev/null +++ b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkWriter.java @@ -0,0 +1,218 @@ +/* + * 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.fluss.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.RowKind; +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.CommonError; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; +import org.apache.seatunnel.connectors.seatunnel.fluss.config.FlussSinkOptions; + +import com.alibaba.fluss.client.Connection; +import com.alibaba.fluss.client.ConnectionFactory; +import com.alibaba.fluss.client.table.Table; +import com.alibaba.fluss.client.table.writer.AppendWriter; +import com.alibaba.fluss.client.table.writer.TableWriter; +import com.alibaba.fluss.client.table.writer.UpsertWriter; +import com.alibaba.fluss.config.Configuration; +import com.alibaba.fluss.metadata.TablePath; +import com.alibaba.fluss.row.BinaryString; +import com.alibaba.fluss.row.Decimal; +import com.alibaba.fluss.row.GenericRow; +import com.alibaba.fluss.row.TimestampLtz; +import com.alibaba.fluss.row.TimestampNtz; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.math.BigDecimal; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.util.Map; +import java.util.Optional; + +@Slf4j +public class FlussSinkWriter extends AbstractSinkWriter + implements SupportMultiTableSinkWriter { + + private Connection connection; + private TableWriter writer; + private Table table; + private String dbName; + private String tableName; + private final SeaTunnelRowType seaTunnelRowType; + + public FlussSinkWriter( + SinkWriter.Context context, CatalogTable catalogTable, ReadonlyConfig pluginConfig) { + seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType(); + Configuration flussConfig = new Configuration(); + flussConfig.setString( + FlussSinkOptions.BOOTSTRAP_SERVERS.key(), + pluginConfig.get(FlussSinkOptions.BOOTSTRAP_SERVERS)); + Optional> clientConfig = + pluginConfig.getOptional(FlussSinkOptions.CLIENT_CONFIG); + if (clientConfig.isPresent()) { + clientConfig + .get() + .forEach( + (k, v) -> { + flussConfig.setString(k, v); + }); + } + log.info("Connect to Fluss with config: {}", flussConfig); + connection = ConnectionFactory.createConnection(flussConfig); + log.info("Connect to Fluss success"); + dbName = + pluginConfig + .getOptional(FlussSinkOptions.DATABASE) + .orElseGet(() -> catalogTable.getTableId().getDatabaseName()); + tableName = + pluginConfig + .getOptional(FlussSinkOptions.TABLE) + .orElseGet(() -> catalogTable.getTableId().getTableName()); + TablePath tablePath = TablePath.of(dbName, tableName); + table = connection.getTable(tablePath); + if (table.getTableInfo().hasPrimaryKey()) { + log.info("Table {} has primary key, use upsert writer", tableName); + writer = table.newUpsert().createWriter(); + } else { + log.info("Table {} has no primary key, use append writer", tableName); + writer = table.newAppend().createWriter(); + } + } + + @Override + public void write(SeaTunnelRow element) { + RowKind rowKind = element.getRowKind(); + GenericRow genericRow = new GenericRow(element.getFields().length); + for (int i = 0; i < element.getFields().length; i++) { + genericRow.setField( + i, + convert( + seaTunnelRowType.getFieldType(i), + seaTunnelRowType.getFieldName(i), + element.getField(i))); + } + + if (writer instanceof UpsertWriter) { + UpsertWriter upsertWriter = (UpsertWriter) writer; + switch (rowKind) { + case INSERT: + case UPDATE_AFTER: + upsertWriter.upsert(genericRow); + break; + case DELETE: + case UPDATE_BEFORE: + upsertWriter.delete(genericRow); + break; + default: + throw CommonError.unsupportedRowKind( + FlussSinkOptions.CONNECTOR_IDENTITY, tableName, rowKind.shortString()); + } + } else if (writer instanceof AppendWriter) { + AppendWriter appendWriter = (AppendWriter) writer; + switch (rowKind) { + case INSERT: + case UPDATE_AFTER: + appendWriter.append(genericRow); + break; + default: + throw CommonError.unsupportedRowKind( + FlussSinkOptions.CONNECTOR_IDENTITY, tableName, rowKind.shortString()); + } + } else { + throw CommonError.unsupportedOperation( + FlussSinkOptions.CONNECTOR_IDENTITY, writer.getClass().getName()); + } + } + + @Override + public Optional prepareCommit(long checkpointId) throws IOException { + writer.flush(); + return super.prepareCommit(checkpointId); + } + + @Override + public void close() { + log.info("Close FlussSinkWriter."); + try { + if (table != null) { + table.close(); + } + if (connection != null) { + connection.close(); + } + } catch (Exception e) { + throw CommonError.closeFailed("Close FlussSinkWriter failed.", e); + } + } + + protected Object convert(SeaTunnelDataType dataType, String fieldName, Object val) { + if (val == null) { + return null; + } + switch (dataType.getSqlType()) { + case BOOLEAN: + case TINYINT: + case SMALLINT: + case INT: + case BIGINT: + case FLOAT: + case DOUBLE: + case BYTES: + return val; + case STRING: + return BinaryString.fromString((String) val); + case DECIMAL: + return Decimal.fromBigDecimal( + (BigDecimal) val, + ((DecimalType) dataType).getPrecision(), + ((DecimalType) dataType).getScale()); + case DATE: + return (int) ((LocalDate) val).toEpochDay(); + case TIME: + return (int) (((LocalTime) val).toNanoOfDay() / 1_000_000); + case TIMESTAMP: + return TimestampNtz.fromLocalDateTime((LocalDateTime) val); + case TIMESTAMP_TZ: + if (val instanceof Instant) { + return TimestampLtz.fromInstant((Instant) val); + } else if (val instanceof OffsetDateTime) { + return TimestampLtz.fromInstant(((OffsetDateTime) val).toInstant()); + } + throw CommonError.unsupportedDataType( + FlussSinkOptions.CONNECTOR_IDENTITY, + dataType.getSqlType().name(), + fieldName); + default: + throw CommonError.unsupportedDataType( + FlussSinkOptions.CONNECTOR_IDENTITY, + dataType.getSqlType().name(), + fieldName); + } + } +} diff --git a/seatunnel-connectors-v2/pom.xml b/seatunnel-connectors-v2/pom.xml index 9cfbdcd62db..c80f9e096d9 100644 --- a/seatunnel-connectors-v2/pom.xml +++ b/seatunnel-connectors-v2/pom.xml @@ -88,6 +88,7 @@ connector-graphql connector-aerospike connector-sensorsdata + connector-fluss diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index 9367cfb29a8..3a1a2301909 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -644,6 +644,13 @@ provided + + org.apache.seatunnel + connector-fluss + ${project.version} + provided + + com.aliyun.phoenix diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/pom.xml new file mode 100644 index 00000000000..6dabd0c45d4 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/pom.xml @@ -0,0 +1,68 @@ + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + + connector-fluss-e2e + SeaTunnel : E2E : Connector V2 : Fluss + + + + + org.apache.seatunnel + connector-fluss + ${project.version} + test + + + org.apache.seatunnel + connector-fake + ${project.version} + test + + + org.apache.seatunnel + connector-cdc-mysql + ${project.version} + test-jar + test + + + org.apache.seatunnel + connector-assert + ${project.version} + test + + + + org.testcontainers + mysql + ${testcontainer.version} + test + + + org.testcontainers + postgresql + ${testcontainer.version} + test + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 8 + 8 + + + + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java new file mode 100644 index 00000000000..14a22b56f5e --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java @@ -0,0 +1,410 @@ +/* + * 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.fluss; + +import org.apache.seatunnel.shade.com.google.common.collect.Lists; + +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +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.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerLoggerFactory; + +import com.alibaba.fluss.client.Connection; +import com.alibaba.fluss.client.ConnectionFactory; +import com.alibaba.fluss.client.admin.Admin; +import com.alibaba.fluss.client.table.Table; +import com.alibaba.fluss.client.table.scanner.ScanRecord; +import com.alibaba.fluss.client.table.scanner.log.LogScanner; +import com.alibaba.fluss.client.table.scanner.log.ScanRecords; +import com.alibaba.fluss.config.Configuration; +import com.alibaba.fluss.metadata.DatabaseDescriptor; +import com.alibaba.fluss.metadata.Schema; +import com.alibaba.fluss.metadata.TableBucket; +import com.alibaba.fluss.metadata.TableDescriptor; +import com.alibaba.fluss.metadata.TablePath; +import com.alibaba.fluss.row.GenericRow; +import com.alibaba.fluss.row.InternalRow; +import com.alibaba.fluss.types.DataTypes; +import com.alibaba.fluss.utils.CloseableIterator; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.net.Socket; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +import static org.awaitility.Awaitility.given; + +@Slf4j +public class FlussSinkIT extends TestSuiteBase implements TestResource { + private static final String DOCKER_IMAGE = "fluss/fluss:0.7.0"; + private static final String DOCKER_ZK_IMAGE = "zookeeper:3.9.2"; + + private static final String FLUSS_Coordinator_HOST = "fluss_coordinator_e2e"; + private static final String FLUSS_Tablet_HOST = "fluss_tablet_e2e"; + private static final String ZK_HOST = "zk_e2e"; + private static final int ZK_PORT = 2181; + private static final int FLUSS_Coordinator_PORT = 9123; + private static final int FLUSS_Tablet_PORT = 9124; + private static final int FLUSS_Coordinator_LOCAL_PORT = 8123; + private static final int FLUSS_Tablet_LOCAL_PORT = 8124; + + private GenericContainer zookeeperServer; + private GenericContainer coordinatorServer; + private GenericContainer tabletServer; + + private Connection flussConnection; + + private static final String DB_NAME = "fluss_db_test"; + private static final String DB_NAME_2 = "fluss_db_test2"; + private static final String DB_NAME_3 = "fluss_db_test3"; + private static final String TABLE_NAME = "fluss_tb_table1"; + private static final String TABLE_NAME_2 = "fluss_tb_table2"; + private static final String TABLE_NAME_3 = "fluss_tb_table3"; + + @BeforeAll + @Override + public void startUp() { + createZookeeperContainer(); + createFlussContainer(); + } + + private void createFlussContainer() { + log.info("Starting FlussServer container..."); + String coordinatorEnv = + String.format( + "zookeeper.address: %s:%d\n" + + "bind.listeners: INTERNAL://%s:%d, LOCALCLIENT://%s:%d \n" + + "advertised.listeners: INTERNAL://%s:%d, LOCALCLIENT://localhost:%d\n" + + "internal.listener.name: INTERNAL", + ZK_HOST, + ZK_PORT, + FLUSS_Coordinator_HOST, + FLUSS_Coordinator_PORT, + FLUSS_Coordinator_HOST, + FLUSS_Coordinator_LOCAL_PORT, + FLUSS_Coordinator_HOST, + FLUSS_Coordinator_PORT, + FLUSS_Coordinator_LOCAL_PORT); + coordinatorServer = + new GenericContainer<>(DOCKER_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(FLUSS_Coordinator_HOST) + .withEnv("FLUSS_PROPERTIES", coordinatorEnv) + .withCommand("coordinatorServer") + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger("coordinatorServer"))); + coordinatorServer.setPortBindings( + Lists.newArrayList( + String.format( + "%s:%s", + FLUSS_Coordinator_LOCAL_PORT, FLUSS_Coordinator_LOCAL_PORT))); + Startables.deepStart(Stream.of(coordinatorServer)).join(); + given().ignoreExceptions() + .await() + .atMost(120, TimeUnit.SECONDS) + .pollInterval(5, TimeUnit.SECONDS) + .until( + () -> + checkPort( + coordinatorServer.getHost(), + FLUSS_Coordinator_LOCAL_PORT, + 1000)); + log.info("coordinatorServer container start success"); + + String tabletEnv = + String.format( + "zookeeper.address: %s:%d\n" + + "bind.listeners: INTERNAL://%s:%d, LOCALCLIENT://%s:%d\n" + + "advertised.listeners: INTERNAL://%s:%d, LOCALCLIENT://localhost:%d\n" + + "internal.listener.name: INTERNAL\n" + + "tablet-server.id: 0\n" + + "kv.snapshot.interval: 0s\n" + + "data.dir: /tmp/fluss/data\n" + + "remote.data.dir: /tmp/fluss/remote-data", + ZK_HOST, + ZK_PORT, + FLUSS_Tablet_HOST, + FLUSS_Tablet_PORT, + FLUSS_Tablet_HOST, + FLUSS_Tablet_LOCAL_PORT, + FLUSS_Tablet_HOST, + FLUSS_Tablet_PORT, + FLUSS_Tablet_LOCAL_PORT); + tabletServer = + new GenericContainer<>(DOCKER_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(FLUSS_Tablet_HOST) + .withEnv("FLUSS_PROPERTIES", tabletEnv) + .withCommand("tabletServer") + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger("tabletServer"))); + tabletServer.setPortBindings( + Lists.newArrayList( + String.format("%s:%s", FLUSS_Tablet_LOCAL_PORT, FLUSS_Tablet_LOCAL_PORT))); + Startables.deepStart(Stream.of(tabletServer)).join(); + given().ignoreExceptions() + .await() + .atMost(120, TimeUnit.SECONDS) + .pollInterval(5, TimeUnit.SECONDS) + .untilAsserted(this::initializeConnection); + log.info("tabletServer container start success"); + log.info("FlussServer Containers are started"); + } + + private void createZookeeperContainer() { + log.info("Starting ZookeeperServer container..."); + zookeeperServer = + new GenericContainer<>(DOCKER_ZK_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(ZK_HOST) + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger(DOCKER_ZK_IMAGE))); + zookeeperServer.setPortBindings( + Lists.newArrayList(String.format("%s:%s", ZK_PORT, ZK_PORT))); + Startables.deepStart(Stream.of(zookeeperServer)).join(); + given().ignoreExceptions() + .await() + .atMost(60, TimeUnit.SECONDS) + .pollInterval(5, TimeUnit.SECONDS) + .until(() -> checkPort(zookeeperServer.getHost(), ZK_PORT, 1000)); + log.info("ZookeeperServer Containers are started"); + } + + private void initializeConnection() throws ExecutionException, InterruptedException { + Configuration flussConfig = new Configuration(); + flussConfig.setString( + "bootstrap.servers", + coordinatorServer.getHost() + ":" + FLUSS_Coordinator_LOCAL_PORT); + flussConnection = ConnectionFactory.createConnection(flussConfig); + createDb(flussConnection, DB_NAME); + } + + public void createDb(Connection connection, String dbName) + throws ExecutionException, InterruptedException { + Admin admin = connection.getAdmin(); + DatabaseDescriptor descriptor = DatabaseDescriptor.builder().build(); + admin.dropDatabase(dbName, true, true).get(); + admin.createDatabase(dbName, descriptor, true).get(); + } + + public Schema getFlussSchema() { + return Schema.newBuilder() + .column("fbytes", DataTypes.BYTES()) + .column("fboolean", DataTypes.BOOLEAN()) + .column("fint", DataTypes.INT()) + .column("ftinyint", DataTypes.TINYINT()) + .column("fsmallint", DataTypes.SMALLINT()) + .column("fbigint", DataTypes.BIGINT()) + .column("ffloat", DataTypes.FLOAT()) + .column("fdouble", DataTypes.DOUBLE()) + .column("fdecimal", DataTypes.DECIMAL(30, 8)) + .column("fstring", DataTypes.STRING()) + .column("fdate", DataTypes.DATE()) + .column("ftime", DataTypes.TIME()) + .column("ftimestamp", DataTypes.TIMESTAMP()) + .column("ftimestamp_ltz", DataTypes.TIMESTAMP_LTZ()) + .primaryKey("fstring") + .build(); + } + + public void createTable(Connection connection, String dbName, String tableName, Schema schema) + throws ExecutionException, InterruptedException { + Admin admin = connection.getAdmin(); + TableDescriptor tableDescriptor = TableDescriptor.builder().schema(schema).build(); + TablePath tablePath = TablePath.of(dbName, tableName); + admin.dropTable(tablePath, true).get(); + admin.createTable(tablePath, tableDescriptor, true).get(); // blocking call + } + + public static boolean checkPort(String host, int port, int timeoutMs) throws IOException { + try (Socket socket = new Socket()) { + socket.connect(new java.net.InetSocketAddress(host, port), timeoutMs); + return true; + } catch (Exception e) { + throw e; + } + } + + @AfterAll + @Override + public void tearDown() throws Exception { + if (tabletServer != null) { + tabletServer.close(); + } + if (coordinatorServer != null) { + coordinatorServer.close(); + } + if (zookeeperServer != null) { + zookeeperServer.close(); + } + } + + @TestTemplate + @DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}) + public void testFlussSink(TestContainer container) throws Exception { + log.info(" create fluss table"); + createDb(flussConnection, DB_NAME); + createTable(flussConnection, DB_NAME, TABLE_NAME, getFlussSchema()); + Container.ExecResult execFake2fluss = container.executeJob("/fake_to_fluss.conf"); + Assertions.assertEquals(0, execFake2fluss.getExitCode(), execFake2fluss.getStderr()); + checkFlussData(DB_NAME, TABLE_NAME); + } + + @TestTemplate + @DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}) + public void testFlussMultiTableSink(TestContainer container) throws Exception { + log.info(" create fluss tables"); + createDb(flussConnection, DB_NAME_2); + createDb(flussConnection, DB_NAME_3); + createTable(flussConnection, DB_NAME_2, TABLE_NAME, getFlussSchema()); + createTable(flussConnection, DB_NAME_2, TABLE_NAME_2, getFlussSchema()); + createTable(flussConnection, DB_NAME_3, TABLE_NAME_3, getFlussSchema()); + + Container.ExecResult execFake2fluss = + container.executeJob("/fake_to_multipletable_fluss.conf"); + Assertions.assertEquals(0, execFake2fluss.getExitCode(), execFake2fluss.getStderr()); + checkFlussData(DB_NAME_2, TABLE_NAME); + checkFlussData(DB_NAME_2, TABLE_NAME_2); + checkFlussData(DB_NAME_3, TABLE_NAME_3); + } + + public void checkFlussData(String dbName, String tableName) throws IOException { + // check log data + List streamData = + getFlussTableStreamData(flussConnection, DB_NAME_2, TABLE_NAME, 10); + checkFlussTableStreamData(streamData); + // check data + List data = getFlussTableData(flussConnection, DB_NAME_2, TABLE_NAME, 10); + checkFlussTableData(data); + } + + public void checkFlussTableData(List streamData) { + Assertions.assertEquals(3, streamData.size()); + List expectedResult = + Arrays.asList( + "([109, 105, 73, 90, 106],true,1940337748,73,17489,7408919466156976747,9.434991E37,3.140411637757371E307,4029933791018936000000.00000000,aaaaa,20091,9010000,2025-05-27T21:56:09,2025-09-27T18:54:08Z)", + "([109, 105, 73, 90, 106],true,90650390,37,22504,5851888708829345169,2.6221706E36,1.8915341983748786E307,3093109630614623000000.00000000,bbbbb,20089,76964000,2025-05-08T05:26:18,2025-08-04T08:49:45Z)", + "([109, 105, 73, 90, 106],true,388742243,89,15831,159071788675312856,7.310445E37,1.2166972324288247E308,7994947075691901000000.00000000,ddddd,20092,55687000,2025-07-18T08:59:49,2025-09-12T15:46:25Z)"); + ArrayList result = new ArrayList<>(); + for (GenericRow streamDatum : streamData) { + result.add(streamDatum.toString()); + } + Assertions.assertEquals(expectedResult, result); + } + + public void checkFlussTableStreamData(List streamData) { + Assertions.assertEquals(7, streamData.size()); + List expectedResult = + Arrays.asList( + "([109, 105, 73, 90, 106],true,1940337748,73,17489,7408919466156976747,9.434991E37,3.140411637757371E307,4029933791018936000000.00000000,aaaaa,20091,9010000,2025-05-27T21:56:09,2025-09-27T18:54:08Z)", + "([109, 105, 73, 90, 106],true,90650390,37,22504,5851888708829345169,2.6221706E36,1.8915341983748786E307,3093109630614623000000.00000000,bbbbb,20089,76964000,2025-05-08T05:26:18,2025-08-04T08:49:45Z)", + "([109, 105, 73, 90, 106],true,2146418323,79,19821,6393905306944584839,2.0462337E38,1.4868114385836557E308,5594947262031770000000.00000000,ccccc,20367,79840000,2025-03-25T01:49:14,2025-07-03T03:52:06Z)", + "([109, 105, 73, 90, 106],true,2146418323,79,19821,6393905306944584839,2.0462337E38,1.4868114385836557E308,5594947262031770000000.00000000,ccccc,20367,79840000,2025-03-25T01:49:14,2025-07-03T03:52:06Z)", + "([109, 105, 73, 90, 106],true,82794384,27,30339,5826566947079347516,2.2137477E37,1.7737681870839753E308,3984670873242882300000.00000000,ddddd,20344,37972000,2025-01-27T19:20:51,2025-11-06T18:38:54Z)", + "([109, 105, 73, 90, 106],true,82794384,27,30339,5826566947079347516,2.2137477E37,1.7737681870839753E308,3984670873242882300000.00000000,ddddd,20344,37972000,2025-01-27T19:20:51,2025-11-06T18:38:54Z)", + "([109, 105, 73, 90, 106],true,388742243,89,15831,159071788675312856,7.310445E37,1.2166972324288247E308,7994947075691901000000.00000000,ddddd,20092,55687000,2025-07-18T08:59:49,2025-09-12T15:46:25Z)"); + ArrayList result = new ArrayList<>(); + for (GenericRow streamDatum : streamData) { + result.add(streamDatum.toString()); + } + Assertions.assertEquals(expectedResult, result); + } + + public List getFlussTableStreamData( + Connection connection, String dbName, String tableName, int scanNum) { + TablePath tablePath = TablePath.of(dbName, tableName); + Table table = connection.getTable(tablePath); + LogScanner logScanner = table.newScan().createLogScanner(); + int numBuckets = table.getTableInfo().getNumBuckets(); + for (int i = 0; i < numBuckets; i++) { + logScanner.subscribeFromBeginning(i); + } + int scanned = 0; + List rows = new ArrayList<>(); + + while (true) { + if (scanned > scanNum) break; + log.info("Polling for stream records..."); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (TableBucket bucket : scanRecords.buckets()) { + for (ScanRecord record : scanRecords.records(bucket)) { + GenericRow row = (GenericRow) record.getRow(); + rows.add(row); + } + } + scanned++; + } + return rows; + } + + public List getFlussTableData( + Connection connection, String dbName, String tableName, int scanNum) + throws IOException { + TablePath tablePath = TablePath.of(dbName, tableName); + Table table = connection.getTable(tablePath); + LogScanner logScanner = table.newScan().createLogScanner(); + int numBuckets = table.getTableInfo().getNumBuckets(); + for (int i = 0; i < numBuckets; i++) { + logScanner.subscribeFromBeginning(i); + } + int scanned = 0; + List rows = new ArrayList<>(); + + while (true) { + if (scanned > scanNum) break; + log.info("Polling for records..."); + ScanRecords scanRecords = logScanner.poll(Duration.ofSeconds(1)); + for (TableBucket bucket : scanRecords.buckets()) { + CloseableIterator data = + table.newScan() + .limit(10) + .createBatchScanner(bucket) + .pollBatch(Duration.ofSeconds(5)); + while (data.hasNext()) { + rows.add((GenericRow) data.next()); + } + } + scanned++; + } + return rows; + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/resources/fake_to_fluss.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/resources/fake_to_fluss.conf new file mode 100644 index 00000000000..27ea0435bb6 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/resources/fake_to_fluss.conf @@ -0,0 +1,96 @@ +# +# 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 { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + parallelism = 1 + tables_configs = [ + { + row.num = 7 + schema { + table = "test.table1" + fields { + fbytes = bytes + fboolean = boolean + fint = int + ftinyint = tinyint + fsmallint = smallint + fbigint = bigint + ffloat = float + fdouble = double + fdecimal = "decimal(30, 8)" + fstring = string + fdate = date + ftime = time + ftimestamp = timestamp + ftimestamp_ltz = timestamp_tz + } + } + rows = [ + { + kind = INSERT + fields = ["bWlJWmo=", true, 1940337748, 73, 17489, 7408919466156976747, 9.434991E37, 3.140411637757371E307, 4029933791018936061944.80602290, "aaaaa", "2025-01-03", "02:30:10", "2025-05-27T21:56:09", "2025-09-28T02:54:08+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 90650390, 37, 22504, 5851888708829345169, 2.6221706E36, 1.8915341983748786E307, 3093109630614622831876.71725344, "bbbbb", "2025-01-01", "21:22:44", "2025-05-08T05:26:18", "2025-08-04T16:49:45+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = DELETE + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_BEFORE + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_AFTER + fields = ["bWlJWmo=", true, 388742243, 89, 15831, 159071788675312856, 7.310445E37, 1.2166972324288247E308, 7994947075691901110245.55960937, "ddddd", "2025-01-04", "15:28:07", "2025-07-18T08:59:49", "2025-09-12T23:46:25+08:00"] + } + ] + } + ] +} +} + +transform { +} + +sink { + Fluss { + bootstrap.servers="fluss_coordinator_e2e:9123" + database = "fluss_db_${database_name}" + table = "fluss_tb_${table_name}" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/resources/fake_to_multipletable_fluss.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/resources/fake_to_multipletable_fluss.conf new file mode 100644 index 00000000000..b1b1307652f --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/resources/fake_to_multipletable_fluss.conf @@ -0,0 +1,200 @@ +# +# 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 { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + parallelism = 1 + tables_configs = [ + { + row.num = 7 + schema { + table = "test2.table1" + fields { + fbytes = bytes + fboolean = boolean + fint = int + ftinyint = tinyint + fsmallint = smallint + fbigint = bigint + ffloat = float + fdouble = double + fdecimal = "decimal(30, 8)" + fstring = string + fdate = date + ftime = time + ftimestamp = timestamp + ftimestamp_ltz = timestamp_tz + } + } + rows = [ + { + kind = INSERT + fields = ["bWlJWmo=", true, 1940337748, 73, 17489, 7408919466156976747, 9.434991E37, 3.140411637757371E307, 4029933791018936061944.80602290, "aaaaa", "2025-01-03", "02:30:10", "2025-05-27T21:56:09", "2025-09-28T02:54:08+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 90650390, 37, 22504, 5851888708829345169, 2.6221706E36, 1.8915341983748786E307, 3093109630614622831876.71725344, "bbbbb", "2025-01-01", "21:22:44", "2025-05-08T05:26:18", "2025-08-04T16:49:45+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = DELETE + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_BEFORE + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_AFTER + fields = ["bWlJWmo=", true, 388742243, 89, 15831, 159071788675312856, 7.310445E37, 1.2166972324288247E308, 7994947075691901110245.55960937, "ddddd", "2025-01-04", "15:28:07", "2025-07-18T08:59:49", "2025-09-12T23:46:25+08:00"] + } + ] + }, + { + row.num = 7 + schema { + table = "test2.table2" + fields { + fbytes = bytes + fboolean = boolean + fint = int + ftinyint = tinyint + fsmallint = smallint + fbigint = bigint + ffloat = float + fdouble = double + fdecimal = "decimal(30, 8)" + fstring = string + fdate = date + ftime = time + ftimestamp = timestamp + ftimestamp_ltz = timestamp_tz + } + } + rows = [ + { + kind = INSERT + fields = ["bWlJWmo=", true, 1940337748, 73, 17489, 7408919466156976747, 9.434991E37, 3.140411637757371E307, 4029933791018936061944.80602290, "aaaaa", "2025-01-03", "02:30:10", "2025-05-27T21:56:09", "2025-09-28T02:54:08+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 90650390, 37, 22504, 5851888708829345169, 2.6221706E36, 1.8915341983748786E307, 3093109630614622831876.71725344, "bbbbb", "2025-01-01", "21:22:44", "2025-05-08T05:26:18", "2025-08-04T16:49:45+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = DELETE + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_BEFORE + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_AFTER + fields = ["bWlJWmo=", true, 388742243, 89, 15831, 159071788675312856, 7.310445E37, 1.2166972324288247E308, 7994947075691901110245.55960937, "ddddd", "2025-01-04", "15:28:07", "2025-07-18T08:59:49", "2025-09-12T23:46:25+08:00"] + } + ] + }, + { + row.num = 7 + schema { + table = "test3.table3" + fields { + fbytes = bytes + fboolean = boolean + fint = int + ftinyint = tinyint + fsmallint = smallint + fbigint = bigint + ffloat = float + fdouble = double + fdecimal = "decimal(30, 8)" + fstring = string + fdate = date + ftime = time + ftimestamp = timestamp + ftimestamp_ltz = timestamp_tz + } + } + rows = [ + { + kind = INSERT + fields = ["bWlJWmo=", true, 1940337748, 73, 17489, 7408919466156976747, 9.434991E37, 3.140411637757371E307, 4029933791018936061944.80602290, "aaaaa", "2025-01-03", "02:30:10", "2025-05-27T21:56:09", "2025-09-28T02:54:08+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 90650390, 37, 22504, 5851888708829345169, 2.6221706E36, 1.8915341983748786E307, 3093109630614622831876.71725344, "bbbbb", "2025-01-01", "21:22:44", "2025-05-08T05:26:18", "2025-08-04T16:49:45+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = DELETE + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_BEFORE + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_AFTER + fields = ["bWlJWmo=", true, 388742243, 89, 15831, 159071788675312856, 7.310445E37, 1.2166972324288247E308, 7994947075691901110245.55960937, "ddddd", "2025-01-04", "15:28:07", "2025-07-18T08:59:49", "2025-09-12T23:46:25+08:00"] + } + ] + } + ] +} +} + +transform { +} + +sink { + Fluss { + bootstrap.servers="fluss_coordinator_e2e:9123" + database = "fluss_db_${database_name}" + table = "fluss_tb_${table_name}" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index 4cdb7af240d..31dc735fc75 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -88,6 +88,7 @@ connector-graphql-e2e connector-aerospike-e2e connector-sensorsdata-e2e + connector-fluss-e2e diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonToRowConverters.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonToRowConverters.java index 949392d1724..a452db64366 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonToRowConverters.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonToRowConverters.java @@ -41,6 +41,7 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.OffsetDateTime; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatterBuilder; import java.time.temporal.ChronoField; @@ -155,6 +156,13 @@ public Object convert(JsonNode jsonNode, String fieldName) { return convertToLocalDateTime(jsonNode, fieldName); } }; + case TIMESTAMP_TZ: + return new JsonToObjectConverter() { + @Override + public Object convert(JsonNode jsonNode, String fieldName) { + return convertToOffsetDateTime(jsonNode, fieldName); + } + }; case FLOAT: return new JsonToObjectConverter() { @Override @@ -284,6 +292,11 @@ private LocalDateTime convertToLocalDateTime(JsonNode jsonNode, String fieldName return LocalDateTime.of(localDate, localTime); } + private OffsetDateTime convertToOffsetDateTime(JsonNode jsonNode, String fieldName) { + String datetimeStr = jsonNode.asText(); + return OffsetDateTime.parse(datetimeStr); + } + private String convertToString(JsonNode jsonNode) { if (jsonNode.isContainerNode()) { return jsonNode.toString(); diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/RowToJsonConverters.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/RowToJsonConverters.java index 2cf8ae092e7..13a30442d17 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/RowToJsonConverters.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/RowToJsonConverters.java @@ -37,6 +37,7 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.OffsetDateTime; import java.util.Arrays; import java.util.Map; import java.util.function.Function; @@ -44,6 +45,7 @@ import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE_TIME; +import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME; public class RowToJsonConverters implements Serializable { @@ -183,6 +185,14 @@ public JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object value) { .textNode(ISO_LOCAL_DATE_TIME.format((LocalDateTime) value)); } }; + case TIMESTAMP_TZ: + return new RowToJsonConverter() { + @Override + public JsonNode convert(ObjectMapper mapper, JsonNode reuse, Object value) { + return mapper.getNodeFactory() + .textNode(ISO_OFFSET_DATE_TIME.format((OffsetDateTime) value)); + } + }; case ARRAY: return createArrayConverter((ArrayType) type); case MAP: 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 8471756b8b5..13d8b5c820c 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 @@ -47,6 +47,7 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.OffsetDateTime; import java.time.format.DateTimeFormatter; import java.time.temporal.TemporalQueries; import java.util.HashMap; @@ -76,6 +77,7 @@ public void testSerDe() throws Exception { String name = "asdlkjasjkdla998y1122"; LocalDate date = LocalDate.parse("1990-10-14"); LocalTime time = LocalTime.parse("12:12:43"); + OffsetDateTime offsetDateTime = OffsetDateTime.parse("2025-09-12T23:46:25+08:00"); Timestamp timestamp3 = Timestamp.valueOf("1990-10-14 12:12:43.123"); Timestamp timestamp9 = Timestamp.valueOf("1990-10-14 12:12:43.123456789"); Map map = new HashMap<>(); @@ -100,6 +102,7 @@ public void testSerDe() throws Exception { root.put("name", name); root.put("date", "1990-10-14"); root.put("time", "12:12:43"); + root.put("timestamp_tz", "2025-09-12T23:46:25+08:00"); root.put("timestamp3", "1990-10-14T12:12:43.123"); root.put("timestamp9", "1990-10-14T12:12:43.123456789"); root.putObject("map").put("element", 123); @@ -121,6 +124,7 @@ public void testSerDe() throws Exception { "name", "date", "time", + "timestamp_tz", "timestamp3", "timestamp9", "map", @@ -136,6 +140,7 @@ public void testSerDe() throws Exception { STRING_TYPE, LocalTimeType.LOCAL_DATE_TYPE, LocalTimeType.LOCAL_TIME_TYPE, + LocalTimeType.OFFSET_DATE_TIME_TYPE, LocalTimeType.LOCAL_DATE_TIME_TYPE, LocalTimeType.LOCAL_DATE_TIME_TYPE, new MapType(STRING_TYPE, LONG_TYPE), @@ -150,6 +155,7 @@ public void testSerDe() throws Exception { "name", "date", "time", + "timestamp_tz", "timestamp3", "timestamp9", "map", @@ -164,6 +170,7 @@ public void testSerDe() throws Exception { STRING_TYPE, LocalTimeType.LOCAL_DATE_TIME_TYPE, LocalTimeType.LOCAL_TIME_TYPE, + LocalTimeType.OFFSET_DATE_TIME_TYPE, LocalTimeType.LOCAL_DATE_TIME_TYPE, LocalTimeType.LOCAL_DATE_TIME_TYPE, new MapType(STRING_TYPE, LONG_TYPE), @@ -175,7 +182,7 @@ public void testSerDe() throws Exception { JsonDeserializationSchema deserializationSchema = new JsonDeserializationSchema(catalogTables, false, false); - SeaTunnelRow expected = new SeaTunnelRow(13); + SeaTunnelRow expected = new SeaTunnelRow(14); expected.setField(0, true); expected.setField(1, intValue); expected.setField(2, longValue); @@ -183,13 +190,14 @@ public void testSerDe() throws Exception { expected.setField(4, name); expected.setField(5, date); expected.setField(6, time); - expected.setField(7, timestamp3.toLocalDateTime()); - expected.setField(8, timestamp9.toLocalDateTime()); - expected.setField(9, map); - expected.setField(10, multiSet); - expected.setField(11, nestedMap); - - SeaTunnelRow rowFieldRow = new SeaTunnelRow(12); + expected.setField(7, offsetDateTime); + expected.setField(8, timestamp3.toLocalDateTime()); + expected.setField(9, timestamp9.toLocalDateTime()); + expected.setField(10, map); + expected.setField(11, multiSet); + expected.setField(12, nestedMap); + + SeaTunnelRow rowFieldRow = new SeaTunnelRow(13); rowFieldRow.setField(0, true); rowFieldRow.setField(1, intValue); rowFieldRow.setField(2, longValue); @@ -197,13 +205,14 @@ public void testSerDe() throws Exception { rowFieldRow.setField(4, name); rowFieldRow.setField(5, timestamp3.toLocalDateTime()); rowFieldRow.setField(6, time); - rowFieldRow.setField(7, timestamp3.toLocalDateTime()); - rowFieldRow.setField(8, timestamp9.toLocalDateTime()); - rowFieldRow.setField(9, map); - rowFieldRow.setField(10, multiSet); - rowFieldRow.setField(11, nestedMap); + rowFieldRow.setField(7, offsetDateTime); + rowFieldRow.setField(8, timestamp3.toLocalDateTime()); + rowFieldRow.setField(9, timestamp9.toLocalDateTime()); + rowFieldRow.setField(10, map); + rowFieldRow.setField(11, multiSet); + rowFieldRow.setField(12, nestedMap); - expected.setField(12, rowFieldRow); + expected.setField(13, rowFieldRow); SeaTunnelRow seaTunnelRow = deserializationSchema.deserialize(serializedJson); assertEquals(expected, seaTunnelRow); @@ -678,6 +687,16 @@ public void testSerializationWithTimestamp() { assertEquals( "{\"timestamp\":\"2022-09-24T22:45:00.000123456\"}", new String(new JsonSerializationSchema(schema, "\\N").serialize(row))); + + schema = + new SeaTunnelRowType( + new String[] {"timestamp_tz"}, + new SeaTunnelDataType[] {LocalTimeType.OFFSET_DATE_TIME_TYPE}); + OffsetDateTime offsetDateTime = OffsetDateTime.parse("2025-09-12T23:46:25+08:00"); + row = new SeaTunnelRow(new Object[] {offsetDateTime}); + assertEquals( + "{\"timestamp_tz\":\"2025-09-12T23:46:25+08:00\"}", + new String(new JsonSerializationSchema(schema, "\\N").serialize(row))); } @Test From c1594f5a3264f81a4672532636543db23e2ea4a7 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Wed, 26 Nov 2025 18:36:31 +0800 Subject: [PATCH 25/37] fead:add fluss sink --- seatunnel-connectors-v2/connector-fluss/pom.xml | 14 ++++++++++++++ .../connector-fluss-e2e/pom.xml | 14 ++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/seatunnel-connectors-v2/connector-fluss/pom.xml b/seatunnel-connectors-v2/connector-fluss/pom.xml index d2335444625..8db032b6f90 100644 --- a/seatunnel-connectors-v2/connector-fluss/pom.xml +++ b/seatunnel-connectors-v2/connector-fluss/pom.xml @@ -1,4 +1,18 @@ + 4.0.0 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/pom.xml index 6dabd0c45d4..4105b14d249 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/pom.xml @@ -1,4 +1,18 @@ + 4.0.0 From 6741580a78ddeeb6ccfde9d17ba868f7f85026b6 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Wed, 26 Nov 2025 23:50:55 +0800 Subject: [PATCH 26/37] fead:add fluss sink --- .../seatunnel/fluss/sink/FlussSinkWriter.java | 16 +++++++++++++--- .../e2e/connector/fluss/FlussSinkIT.java | 4 ++-- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkWriter.java b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkWriter.java index 92a85ba5a00..91881d30e5a 100644 --- a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkWriter.java +++ b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkWriter.java @@ -126,9 +126,10 @@ public void write(SeaTunnelRow element) { upsertWriter.upsert(genericRow); break; case DELETE: - case UPDATE_BEFORE: upsertWriter.delete(genericRow); break; + case UPDATE_BEFORE: + return; default: throw CommonError.unsupportedRowKind( FlussSinkOptions.CONNECTOR_IDENTITY, tableName, rowKind.shortString()); @@ -140,6 +141,9 @@ public void write(SeaTunnelRow element) { case UPDATE_AFTER: appendWriter.append(genericRow); break; + case DELETE: + case UPDATE_BEFORE: + return; default: throw CommonError.unsupportedRowKind( FlussSinkOptions.CONNECTOR_IDENTITY, tableName, rowKind.shortString()); @@ -158,16 +162,22 @@ public Optional prepareCommit(long checkpointId) throws IOException { @Override public void close() { - log.info("Close FlussSinkWriter."); + log.info("Close Fluss table."); try { if (table != null) { table.close(); } + } catch (Exception e) { + throw CommonError.closeFailed("Close Fluss table failed.", e); + } + + log.info("Close Fluss connection."); + try { if (connection != null) { connection.close(); } } catch (Exception e) { - throw CommonError.closeFailed("Close FlussSinkWriter failed.", e); + throw CommonError.closeFailed("Close Fluss connection failed.", e); } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java index 14a22b56f5e..01118991424 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java @@ -310,10 +310,10 @@ public void testFlussMultiTableSink(TestContainer container) throws Exception { public void checkFlussData(String dbName, String tableName) throws IOException { // check log data List streamData = - getFlussTableStreamData(flussConnection, DB_NAME_2, TABLE_NAME, 10); + getFlussTableStreamData(flussConnection, dbName, tableName, 10); checkFlussTableStreamData(streamData); // check data - List data = getFlussTableData(flussConnection, DB_NAME_2, TABLE_NAME, 10); + List data = getFlussTableData(flussConnection, dbName, tableName, 10); checkFlussTableData(data); } From 63cd19f6397e88f126df4f7b7c0a279aa4a62ea0 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Thu, 27 Nov 2025 09:46:24 +0800 Subject: [PATCH 27/37] fead:add fluss sink doc --- docs/en/connector-v2/sink/Fluss.md | 352 ++++++++++++++++++ docs/zh/connector-v2/sink/Fluss.md | 351 +++++++++++++++++ .../e2e/connector/fluss/FlussSinkIT.java | 6 - 3 files changed, 703 insertions(+), 6 deletions(-) create mode 100644 docs/en/connector-v2/sink/Fluss.md create mode 100644 docs/zh/connector-v2/sink/Fluss.md diff --git a/docs/en/connector-v2/sink/Fluss.md b/docs/en/connector-v2/sink/Fluss.md new file mode 100644 index 00000000000..06f70d0a272 --- /dev/null +++ b/docs/en/connector-v2/sink/Fluss.md @@ -0,0 +1,352 @@ +import ChangeLog from '../changelog/connector-fluss.md'; + +# Fluss + +> Fluss sink connector + +## Support These Engines + +> Spark
+> Flink
+> SeaTunnel Zeta
+ +## Key Features + +- [ ] [exactly-once](../../concept/connector-v2-features.md) +- [x] [cdc](../../concept/connector-v2-features.md) +- [x] [support multiple table write](../../concept/connector-v2-features.md) + +## Description + +Used to send data to Fluss. Both support streaming and batch mode. + +## Using Dependency + + com.alibaba.fluss + fluss-client + 0.7.0 + + +## Sink Options + +| Name | Type | Required | Default | Description | +|-------------------|--------|----------|---------|-------------------------------------------------------------------------------------------------------------| +| bootstrap.servers | string | yes | - | The bootstrap servers for the Fluss sink connection. | +| database | string | no | - | The name of Fluss database, If not set, the table name will be the name of the upstream db | +| table | string | no | - | The name of Fluss table, If not set, the table name will be the name of the upstream table | +| client.config | Map | no | - | set other client config. Please refer to https://fluss.apache.org/docs/engine-flink/options/#other-options | + + +### database [string] + +The name of Fluss database, If not set, the table name will be the name of the upstream db + +for example: + +1. test_${schema_name}_test +2. sink_sinkdb +3. ss_${database_name} + + +### table [string] + +The name of Fluss table, If not set, the table name will be the name of the upstream table + +for example: +1. test_${table_name}_test +2. sink_sinktable +3. ss_${table_name} + + +## Data Type Mapping + +| StarRocks Data type | Fluss Data type | +|---------------------|-----------------| +| BOOLEAN | BOOLEAN | +| TINYINT | TINYINT | +| SMALLINT | SMALLINT | +| INT | INT | +| BIGINT | BIGINT | +| FLOAT | FLOAT | +| DOUBLE | DOUBLE | +| DOUBLE | DOUBLE | +| BYTES | BYTES | +| DATE | DATE | +| TIME | TIME | +| TIMESTAMP | TIMESTAMP | +| TIMESTAMP_TZ | TIMESTAMP_TZ | +| STRING | STRING | + +## Task Example + +### Simple + +```hocon +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + parallelism = 1 + tables_configs = [ + { + row.num = 7 + schema { + table = "test.table1" + fields { + fbytes = bytes + fboolean = boolean + fint = int + ftinyint = tinyint + fsmallint = smallint + fbigint = bigint + ffloat = float + fdouble = double + fdecimal = "decimal(30, 8)" + fstring = string + fdate = date + ftime = time + ftimestamp = timestamp + ftimestamp_ltz = timestamp_tz + } + } + rows = [ + { + kind = INSERT + fields = ["bWlJWmo=", true, 1940337748, 73, 17489, 7408919466156976747, 9.434991E37, 3.140411637757371E307, 4029933791018936061944.80602290, "aaaaa", "2025-01-03", "02:30:10", "2025-05-27T21:56:09", "2025-09-28T02:54:08+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 90650390, 37, 22504, 5851888708829345169, 2.6221706E36, 1.8915341983748786E307, 3093109630614622831876.71725344, "bbbbb", "2025-01-01", "21:22:44", "2025-05-08T05:26:18", "2025-08-04T16:49:45+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = DELETE + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_BEFORE + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_AFTER + fields = ["bWlJWmo=", true, 388742243, 89, 15831, 159071788675312856, 7.310445E37, 1.2166972324288247E308, 7994947075691901110245.55960937, "ddddd", "2025-01-04", "15:28:07", "2025-07-18T08:59:49", "2025-09-12T23:46:25+08:00"] + } + ] + } + ] +} +} + +transform { +} + +sink { + Fluss { + bootstrap.servers="fluss_coordinator_e2e:9123" + database = "fluss_db_${database_name}" + table = "fluss_tb_${table_name}" + } +} +``` + +### Multiple table + +```hocon +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + parallelism = 1 + tables_configs = [ + { + row.num = 7 + schema { + table = "test2.table1" + fields { + fbytes = bytes + fboolean = boolean + fint = int + ftinyint = tinyint + fsmallint = smallint + fbigint = bigint + ffloat = float + fdouble = double + fdecimal = "decimal(30, 8)" + fstring = string + fdate = date + ftime = time + ftimestamp = timestamp + ftimestamp_ltz = timestamp_tz + } + } + rows = [ + { + kind = INSERT + fields = ["bWlJWmo=", true, 1940337748, 73, 17489, 7408919466156976747, 9.434991E37, 3.140411637757371E307, 4029933791018936061944.80602290, "aaaaa", "2025-01-03", "02:30:10", "2025-05-27T21:56:09", "2025-09-28T02:54:08+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 90650390, 37, 22504, 5851888708829345169, 2.6221706E36, 1.8915341983748786E307, 3093109630614622831876.71725344, "bbbbb", "2025-01-01", "21:22:44", "2025-05-08T05:26:18", "2025-08-04T16:49:45+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = DELETE + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_BEFORE + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_AFTER + fields = ["bWlJWmo=", true, 388742243, 89, 15831, 159071788675312856, 7.310445E37, 1.2166972324288247E308, 7994947075691901110245.55960937, "ddddd", "2025-01-04", "15:28:07", "2025-07-18T08:59:49", "2025-09-12T23:46:25+08:00"] + } + ] + }, + { + row.num = 7 + schema { + table = "test2.table2" + fields { + fbytes = bytes + fboolean = boolean + fint = int + ftinyint = tinyint + fsmallint = smallint + fbigint = bigint + ffloat = float + fdouble = double + fdecimal = "decimal(30, 8)" + fstring = string + fdate = date + ftime = time + ftimestamp = timestamp + ftimestamp_ltz = timestamp_tz + } + } + rows = [ + { + kind = INSERT + fields = ["bWlJWmo=", true, 1940337748, 73, 17489, 7408919466156976747, 9.434991E37, 3.140411637757371E307, 4029933791018936061944.80602290, "aaaaa", "2025-01-03", "02:30:10", "2025-05-27T21:56:09", "2025-09-28T02:54:08+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 90650390, 37, 22504, 5851888708829345169, 2.6221706E36, 1.8915341983748786E307, 3093109630614622831876.71725344, "bbbbb", "2025-01-01", "21:22:44", "2025-05-08T05:26:18", "2025-08-04T16:49:45+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = DELETE + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_BEFORE + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_AFTER + fields = ["bWlJWmo=", true, 388742243, 89, 15831, 159071788675312856, 7.310445E37, 1.2166972324288247E308, 7994947075691901110245.55960937, "ddddd", "2025-01-04", "15:28:07", "2025-07-18T08:59:49", "2025-09-12T23:46:25+08:00"] + } + ] + }, + { + row.num = 7 + schema { + table = "test3.table3" + fields { + fbytes = bytes + fboolean = boolean + fint = int + ftinyint = tinyint + fsmallint = smallint + fbigint = bigint + ffloat = float + fdouble = double + fdecimal = "decimal(30, 8)" + fstring = string + fdate = date + ftime = time + ftimestamp = timestamp + ftimestamp_ltz = timestamp_tz + } + } + rows = [ + { + kind = INSERT + fields = ["bWlJWmo=", true, 1940337748, 73, 17489, 7408919466156976747, 9.434991E37, 3.140411637757371E307, 4029933791018936061944.80602290, "aaaaa", "2025-01-03", "02:30:10", "2025-05-27T21:56:09", "2025-09-28T02:54:08+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 90650390, 37, 22504, 5851888708829345169, 2.6221706E36, 1.8915341983748786E307, 3093109630614622831876.71725344, "bbbbb", "2025-01-01", "21:22:44", "2025-05-08T05:26:18", "2025-08-04T16:49:45+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = DELETE + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_BEFORE + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_AFTER + fields = ["bWlJWmo=", true, 388742243, 89, 15831, 159071788675312856, 7.310445E37, 1.2166972324288247E308, 7994947075691901110245.55960937, "ddddd", "2025-01-04", "15:28:07", "2025-07-18T08:59:49", "2025-09-12T23:46:25+08:00"] + } + ] + } + ] +} +} + +transform { +} + +sink { + Fluss { + bootstrap.servers="fluss_coordinator_e2e:9123" + database = "fluss_db_${database_name}" + table = "fluss_tb_${table_name}" + } +} +``` + + +## Changelog + + + diff --git a/docs/zh/connector-v2/sink/Fluss.md b/docs/zh/connector-v2/sink/Fluss.md new file mode 100644 index 00000000000..bc32f8db8f9 --- /dev/null +++ b/docs/zh/connector-v2/sink/Fluss.md @@ -0,0 +1,351 @@ +import ChangeLog from '../changelog/connector-fluss.md'; + +# Fluss + +> Fluss 数据接收器 + +## 引擎支持 + +> Spark
+> Flink
+> SeaTunnel Zeta
+ +## 主要特性 + +- [ ] [精准一次](../../concept/connector-v2-features.md) +- [x] [cdc](../../concept/connector-v2-features.md) +- [x] [支持多表写入](../../concept/connector-v2-features.md) + +## 描述 + +该接收器用于将数据写入到Fluss中。支持批和流两种模式。 + +## 依赖 + + com.alibaba.fluss + fluss-client + 0.7.0 + + + +## 接收器选项 + +| 名称 | 类型 | 是否必须 | 默认值 | Description | +|-------------------|--------|------|-----|----------------------------------------------------------------------------------| +| bootstrap.servers | string | yes | - | fluss 集群地址 | +| database | string | no | - | 指定目标 Fluss 表所在的数据库的名称, 如果没有设置该值,则表名与上游库名相同 | +| table | string | no | - | 指定目标 Fluss 表的名称, 如果没有设置该值,则表名与上游表名相同 | +| client.config | Map | no | - | 设置其他客户端配置. 参考 https://fluss.apache.org/docs/engine-flink/options/#other-options | + + +### database [string] + +database选项参数可以填入一任意库名,这个名字最终会被用作目标表的库名,并且支持变量(`${database_name}`,`${schema_name}`)。 +替换规则如下:`${schema_name}` 将替换传递给目标端的 SCHEMA 名称,`${database_name}` 将替换传递给目标端的库名。 + +例如: +1. test_${schema_name}_test +2. sink_sinkdb +3. ss_${database_name} + + +### table [string] + +table选项参数可以填入一任意表名,这个名字最终会被用作目标表的表名,并且支持变量(`${table_name}`,`${schema_name}`)。 +替换规则如下:`${schema_name}` 将替换传递给目标端的 SCHEMA 名称,`${table_name}` 将替换传递给目标端的表名。 + +例如: +1. test_${schema_name}_test +2. sink_sinktable +3. ss_${table_name} + +## 数据类型映射 + +| FLuss数据类型 | SeaTunnel数据类型 | +|--------------|---------------| +| BOOLEAN | BOOLEAN | +| TINYINT | TINYINT | +| SMALLINT | SMALLINT | +| INT | INT | +| BIGINT | BIGINT | +| FLOAT | FLOAT | +| DOUBLE | DOUBLE | +| DOUBLE | DOUBLE | +| BYTES | BYTES | +| DATE | DATE | +| TIME | TIME | +| TIMESTAMP | TIMESTAMP | +| TIMESTAMP_TZ | TIMESTAMP_TZ | +| STRING | STRING | + + +## 任务示例 + +### 简单示例 + +```hocon +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + parallelism = 1 + tables_configs = [ + { + row.num = 7 + schema { + table = "test.table1" + fields { + fbytes = bytes + fboolean = boolean + fint = int + ftinyint = tinyint + fsmallint = smallint + fbigint = bigint + ffloat = float + fdouble = double + fdecimal = "decimal(30, 8)" + fstring = string + fdate = date + ftime = time + ftimestamp = timestamp + ftimestamp_ltz = timestamp_tz + } + } + rows = [ + { + kind = INSERT + fields = ["bWlJWmo=", true, 1940337748, 73, 17489, 7408919466156976747, 9.434991E37, 3.140411637757371E307, 4029933791018936061944.80602290, "aaaaa", "2025-01-03", "02:30:10", "2025-05-27T21:56:09", "2025-09-28T02:54:08+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 90650390, 37, 22504, 5851888708829345169, 2.6221706E36, 1.8915341983748786E307, 3093109630614622831876.71725344, "bbbbb", "2025-01-01", "21:22:44", "2025-05-08T05:26:18", "2025-08-04T16:49:45+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = DELETE + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_BEFORE + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_AFTER + fields = ["bWlJWmo=", true, 388742243, 89, 15831, 159071788675312856, 7.310445E37, 1.2166972324288247E308, 7994947075691901110245.55960937, "ddddd", "2025-01-04", "15:28:07", "2025-07-18T08:59:49", "2025-09-12T23:46:25+08:00"] + } + ] + } + ] +} +} + +transform { +} + +sink { + Fluss { + bootstrap.servers="fluss_coordinator_e2e:9123" + database = "fluss_db_${database_name}" + table = "fluss_tb_${table_name}" + } +} +``` +### 多表写入 + +```hocon +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + parallelism = 1 + tables_configs = [ + { + row.num = 7 + schema { + table = "test2.table1" + fields { + fbytes = bytes + fboolean = boolean + fint = int + ftinyint = tinyint + fsmallint = smallint + fbigint = bigint + ffloat = float + fdouble = double + fdecimal = "decimal(30, 8)" + fstring = string + fdate = date + ftime = time + ftimestamp = timestamp + ftimestamp_ltz = timestamp_tz + } + } + rows = [ + { + kind = INSERT + fields = ["bWlJWmo=", true, 1940337748, 73, 17489, 7408919466156976747, 9.434991E37, 3.140411637757371E307, 4029933791018936061944.80602290, "aaaaa", "2025-01-03", "02:30:10", "2025-05-27T21:56:09", "2025-09-28T02:54:08+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 90650390, 37, 22504, 5851888708829345169, 2.6221706E36, 1.8915341983748786E307, 3093109630614622831876.71725344, "bbbbb", "2025-01-01", "21:22:44", "2025-05-08T05:26:18", "2025-08-04T16:49:45+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = DELETE + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_BEFORE + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_AFTER + fields = ["bWlJWmo=", true, 388742243, 89, 15831, 159071788675312856, 7.310445E37, 1.2166972324288247E308, 7994947075691901110245.55960937, "ddddd", "2025-01-04", "15:28:07", "2025-07-18T08:59:49", "2025-09-12T23:46:25+08:00"] + } + ] + }, + { + row.num = 7 + schema { + table = "test2.table2" + fields { + fbytes = bytes + fboolean = boolean + fint = int + ftinyint = tinyint + fsmallint = smallint + fbigint = bigint + ffloat = float + fdouble = double + fdecimal = "decimal(30, 8)" + fstring = string + fdate = date + ftime = time + ftimestamp = timestamp + ftimestamp_ltz = timestamp_tz + } + } + rows = [ + { + kind = INSERT + fields = ["bWlJWmo=", true, 1940337748, 73, 17489, 7408919466156976747, 9.434991E37, 3.140411637757371E307, 4029933791018936061944.80602290, "aaaaa", "2025-01-03", "02:30:10", "2025-05-27T21:56:09", "2025-09-28T02:54:08+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 90650390, 37, 22504, 5851888708829345169, 2.6221706E36, 1.8915341983748786E307, 3093109630614622831876.71725344, "bbbbb", "2025-01-01", "21:22:44", "2025-05-08T05:26:18", "2025-08-04T16:49:45+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = DELETE + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_BEFORE + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_AFTER + fields = ["bWlJWmo=", true, 388742243, 89, 15831, 159071788675312856, 7.310445E37, 1.2166972324288247E308, 7994947075691901110245.55960937, "ddddd", "2025-01-04", "15:28:07", "2025-07-18T08:59:49", "2025-09-12T23:46:25+08:00"] + } + ] + }, + { + row.num = 7 + schema { + table = "test3.table3" + fields { + fbytes = bytes + fboolean = boolean + fint = int + ftinyint = tinyint + fsmallint = smallint + fbigint = bigint + ffloat = float + fdouble = double + fdecimal = "decimal(30, 8)" + fstring = string + fdate = date + ftime = time + ftimestamp = timestamp + ftimestamp_ltz = timestamp_tz + } + } + rows = [ + { + kind = INSERT + fields = ["bWlJWmo=", true, 1940337748, 73, 17489, 7408919466156976747, 9.434991E37, 3.140411637757371E307, 4029933791018936061944.80602290, "aaaaa", "2025-01-03", "02:30:10", "2025-05-27T21:56:09", "2025-09-28T02:54:08+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 90650390, 37, 22504, 5851888708829345169, 2.6221706E36, 1.8915341983748786E307, 3093109630614622831876.71725344, "bbbbb", "2025-01-01", "21:22:44", "2025-05-08T05:26:18", "2025-08-04T16:49:45+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = DELETE + fields = ["bWlJWmo=", true, 2146418323, 79, 19821, 6393905306944584839, 2.0462337E38, 1.4868114385836557E308, 5594947262031769994080.35717665, "ccccc", "2025-10-06", "22:10:40", "2025-03-25T01:49:14", "2025-07-03T11:52:06+08:00"] + } + { + kind = INSERT + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_BEFORE + fields = ["bWlJWmo=", true, 82794384, 27, 30339, 5826566947079347516, 2.2137477E37, 1.7737681870839753E308, 3984670873242882274814.90739768, "ddddd", "2025-09-13", "10:32:52", "2025-01-27T19:20:51", "2025-11-07T02:38:54+08:00"] + } + { + kind = UPDATE_AFTER + fields = ["bWlJWmo=", true, 388742243, 89, 15831, 159071788675312856, 7.310445E37, 1.2166972324288247E308, 7994947075691901110245.55960937, "ddddd", "2025-01-04", "15:28:07", "2025-07-18T08:59:49", "2025-09-12T23:46:25+08:00"] + } + ] + } + ] +} +} + +transform { +} + +sink { + Fluss { + bootstrap.servers="fluss_coordinator_e2e:9123" + database = "fluss_db_${database_name}" + table = "fluss_tb_${table_name}" + } +} +``` + +## 变更日志 + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java index 01118991424..7c310ea355d 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java @@ -275,9 +275,6 @@ public void tearDown() throws Exception { } @TestTemplate - @DisabledOnContainer( - value = {}, - type = {EngineType.SPARK, EngineType.FLINK}) public void testFlussSink(TestContainer container) throws Exception { log.info(" create fluss table"); createDb(flussConnection, DB_NAME); @@ -288,9 +285,6 @@ public void testFlussSink(TestContainer container) throws Exception { } @TestTemplate - @DisabledOnContainer( - value = {}, - type = {EngineType.SPARK, EngineType.FLINK}) public void testFlussMultiTableSink(TestContainer container) throws Exception { log.info(" create fluss tables"); createDb(flussConnection, DB_NAME_2); From 3bf8dbf7430c53dc6cc71857cebf52ec0f91281e Mon Sep 17 00:00:00 2001 From: dyp12 Date: Thu, 27 Nov 2025 09:50:47 +0800 Subject: [PATCH 28/37] fead:add fluss sink doc --- .../org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java index 7c310ea355d..04230c362e5 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-fluss-e2e/src/test/java/org/apache/seatunnel/e2e/connector/fluss/FlussSinkIT.java @@ -21,9 +21,7 @@ import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; -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.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; From f9616f2489557cd3262764afa3d5c4e47dd5a6c2 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Thu, 27 Nov 2025 09:57:15 +0800 Subject: [PATCH 29/37] fead:add fluss sink doc --- docs/en/connector-v2/changelog/connector-fluss.md | 6 ++++++ docs/zh/connector-v2/changelog/connector-fluss.md | 6 ++++++ 2 files changed, 12 insertions(+) create mode 100644 docs/en/connector-v2/changelog/connector-fluss.md create mode 100644 docs/zh/connector-v2/changelog/connector-fluss.md diff --git a/docs/en/connector-v2/changelog/connector-fluss.md b/docs/en/connector-v2/changelog/connector-fluss.md new file mode 100644 index 00000000000..97ff1428167 --- /dev/null +++ b/docs/en/connector-v2/changelog/connector-fluss.md @@ -0,0 +1,6 @@ +
Change Log + +| Change | Commit | Version | +|--------|--------|---------| + +
diff --git a/docs/zh/connector-v2/changelog/connector-fluss.md b/docs/zh/connector-v2/changelog/connector-fluss.md new file mode 100644 index 00000000000..97ff1428167 --- /dev/null +++ b/docs/zh/connector-v2/changelog/connector-fluss.md @@ -0,0 +1,6 @@ +
Change Log + +| Change | Commit | Version | +|--------|--------|---------| + +
From d72dd86081f562fafc12284bc483bfd92cde35f0 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Thu, 27 Nov 2025 17:14:24 +0800 Subject: [PATCH 30/37] fead:add createWriter --- .../connectors/seatunnel/fluss/sink/FlussSink.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java index f3a9be38570..33ddfcca117 100644 --- a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java +++ b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java @@ -28,6 +28,8 @@ import lombok.extern.slf4j.Slf4j; +import java.util.Optional; + @Slf4j public class FlussSink extends AbstractSimpleSink implements SupportMultiTableSink { @@ -45,6 +47,11 @@ public AbstractSinkWriter createWriter(SinkWriter.Context co return new FlussSinkWriter(context, catalogTable, pluginConfig); } + @Override + public Optional getWriteCatalogTable() { + return Optional.of(catalogTable); + } + @Override public String getPluginName() { return FlussSinkOptions.CONNECTOR_IDENTITY; From 5c6afb76cbf214c3811ca6577e0a8334b954f218 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Thu, 27 Nov 2025 19:57:25 +0800 Subject: [PATCH 31/37] fead:add createWriter --- .../connectors/seatunnel/fluss/sink/FlussSinkFactory.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkFactory.java b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkFactory.java index f7cc8c8d9c0..13ee1424685 100644 --- a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkFactory.java +++ b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSinkFactory.java @@ -26,6 +26,8 @@ import com.google.auto.service.AutoService; +import static org.apache.seatunnel.api.options.SinkConnectorCommonOptions.MULTI_TABLE_SINK_REPLICA; + @AutoService(Factory.class) public class FlussSinkFactory implements TableSinkFactory { @Override @@ -40,6 +42,7 @@ public OptionRule optionRule() { .optional(FlussSinkOptions.DATABASE) .optional(FlussSinkOptions.TABLE) .optional(FlussSinkOptions.CLIENT_CONFIG) + .optional(MULTI_TABLE_SINK_REPLICA) .build(); } From 39f9c0dfdd4bf747f49d9cd11a461a2aa83493a1 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Thu, 27 Nov 2025 20:06:12 +0800 Subject: [PATCH 32/37] fead:add createWriter --- .../seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java index 33ddfcca117..cfefbdd72ff 100644 --- a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java +++ b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java @@ -43,7 +43,7 @@ public FlussSink(ReadonlyConfig pluginConfig, CatalogTable catalogTable) { } @Override - public AbstractSinkWriter createWriter(SinkWriter.Context context) { + public FlussSinkWriter createWriter(SinkWriter.Context context) { return new FlussSinkWriter(context, catalogTable, pluginConfig); } From 914385bd9caf56f41167a49980b2b52c6890ff84 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Thu, 27 Nov 2025 20:06:55 +0800 Subject: [PATCH 33/37] fead:add createWriter --- .../seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java | 1 - 1 file changed, 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java index cfefbdd72ff..771d414c700 100644 --- a/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java +++ b/seatunnel-connectors-v2/connector-fluss/src/main/java/org/apache/seatunnel/connectors/seatunnel/fluss/sink/FlussSink.java @@ -23,7 +23,6 @@ import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; -import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; import org.apache.seatunnel.connectors.seatunnel.fluss.config.FlussSinkOptions; import lombok.extern.slf4j.Slf4j; From 4b0aacd50a749633cb3f9cd7dd0dd1e2c15a5108 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Thu, 27 Nov 2025 21:03:26 +0800 Subject: [PATCH 34/37] fead:add createWriter --- plugin-mapping.properties | 1 + seatunnel-dist/pom.xml | 7 +++++++ seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml | 1 + 3 files changed, 9 insertions(+) diff --git a/plugin-mapping.properties b/plugin-mapping.properties index 4172161f5e4..74b84a47582 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -150,6 +150,7 @@ seatunnel.sink.GraphQL = connector-graphql seatunnel.sink.Aerospike = connector-aerospike seatunnel.sink.SensorsData = connector-sensorsdata seatunnel.sink.HugeGraph = connector-hugegraph +seatunnel.sink.Fluss = connector-fluss # For custom transforms, make sure to use the seatunnel.transform.[PluginIdentifier]=[JarPerfix] naming convention. For example: # seatunnel.transform.Sql = seatunnel-transforms-v2 diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index 8f79459a541..f4196c4918c 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -651,6 +651,13 @@ provided
+ + org.apache.seatunnel + connector-fluss + ${project.version} + provided + + com.aliyun.phoenix diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index ffcfc5f0a79..3d826457139 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -90,6 +90,7 @@ connector-aerospike-e2e connector-sensorsdata-e2e connector-hugegraph-e2e + connector-fluss-e2e From e4a0fc5ea0b64068dc380ad1894b6456106840a7 Mon Sep 17 00:00:00 2001 From: dyp12 Date: Fri, 28 Nov 2025 17:36:15 +0800 Subject: [PATCH 35/37] fead:add createWriter --- .../connectors/jdbc/AbstractSchemaChangeBaseIT.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java index a467428f51c..b4b57f10e92 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java @@ -292,7 +292,7 @@ public void testMysqlCdcWithSchemaEvolutionCaseExactlyOnce(TestContainer contain } private void assertSchemaEvolution(String sourceTable, String sinkTable) { - await().atMost(30000, TimeUnit.MILLISECONDS) + await().atMost(60000, TimeUnit.MILLISECONDS) .untilAsserted( () -> Assertions.assertIterableEquals( @@ -307,7 +307,7 @@ private void assertSchemaEvolution(String sourceTable, String sinkTable) { // case1 add columns with cdc data at same time sourceDatabase.setTemplateName("add_columns").createAndInitialize(); - await().atMost(30000, TimeUnit.MILLISECONDS) + await().atMost(60000, TimeUnit.MILLISECONDS) .untilAsserted( () -> Assertions.assertIterableEquals( @@ -321,7 +321,7 @@ private void assertSchemaEvolution(String sourceTable, String sinkTable) { schemaChangeCase.getSinkQueryColumns(), schemaChangeCase.getSchemaName(), sinkTable)))); - await().atMost(30000, TimeUnit.MILLISECONDS) + await().atMost(60000, TimeUnit.MILLISECONDS) .untilAsserted( () -> { Assertions.assertIterableEquals( From 644c1799f8d8259102bf91963c15074e15d8953b Mon Sep 17 00:00:00 2001 From: dyp12 Date: Fri, 28 Nov 2025 19:41:04 +0800 Subject: [PATCH 36/37] fead:add createWriter --- .../seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java index b4b57f10e92..4a9b56f300d 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java @@ -307,7 +307,7 @@ private void assertSchemaEvolution(String sourceTable, String sinkTable) { // case1 add columns with cdc data at same time sourceDatabase.setTemplateName("add_columns").createAndInitialize(); - await().atMost(60000, TimeUnit.MILLISECONDS) + await().atMost(120000, TimeUnit.MILLISECONDS) .untilAsserted( () -> Assertions.assertIterableEquals( From 570e69fa9885d5c6f4c9f595ad39114518eee508 Mon Sep 17 00:00:00 2001 From: dyp12 <49448072+dyp12@users.noreply.github.com> Date: Sat, 29 Nov 2025 15:58:40 +0800 Subject: [PATCH 37/37] Update AbstractSchemaChangeBaseIT.java --- .../connectors/jdbc/AbstractSchemaChangeBaseIT.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java index 4a9b56f300d..a467428f51c 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-ddl/src/test/java/org/apache/seatunnel/connectors/jdbc/AbstractSchemaChangeBaseIT.java @@ -292,7 +292,7 @@ public void testMysqlCdcWithSchemaEvolutionCaseExactlyOnce(TestContainer contain } private void assertSchemaEvolution(String sourceTable, String sinkTable) { - await().atMost(60000, TimeUnit.MILLISECONDS) + await().atMost(30000, TimeUnit.MILLISECONDS) .untilAsserted( () -> Assertions.assertIterableEquals( @@ -307,7 +307,7 @@ private void assertSchemaEvolution(String sourceTable, String sinkTable) { // case1 add columns with cdc data at same time sourceDatabase.setTemplateName("add_columns").createAndInitialize(); - await().atMost(120000, TimeUnit.MILLISECONDS) + await().atMost(30000, TimeUnit.MILLISECONDS) .untilAsserted( () -> Assertions.assertIterableEquals( @@ -321,7 +321,7 @@ private void assertSchemaEvolution(String sourceTable, String sinkTable) { schemaChangeCase.getSinkQueryColumns(), schemaChangeCase.getSchemaName(), sinkTable)))); - await().atMost(60000, TimeUnit.MILLISECONDS) + await().atMost(30000, TimeUnit.MILLISECONDS) .untilAsserted( () -> { Assertions.assertIterableEquals(