diff --git a/docs/content.zh/docs/connectors/flink-sources/db2-cdc.md b/docs/content.zh/docs/connectors/flink-sources/db2-cdc.md index cc6576f7849..dfab602be89 100644 --- a/docs/content.zh/docs/connectors/flink-sources/db2-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/db2-cdc.md @@ -274,6 +274,16 @@ Db2 server. This might help reduce the risk of the TaskManager experiencing an out-of-memory (OOM) error when taking a snapshot of the largest unbounded chunk.
+ + records.per.second + optional + true + Double + + The maximum number of records emitted per second, the default value: -1, means no rate limiting.(only for flink2.x)
+ WARNING: Incremental/binlog phase: Rate limiting can cause the connector to fall behind the upstream change stream, risking binlog/WAL purging before the connector catches up (data loss for MySQL, replication slot issues for PostgreSQL). + + diff --git a/docs/content.zh/docs/connectors/flink-sources/mongodb-cdc.md b/docs/content.zh/docs/connectors/flink-sources/mongodb-cdc.md index 7ea19f2ae07..6f4fa11dd34 100644 --- a/docs/content.zh/docs/connectors/flink-sources/mongodb-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/mongodb-cdc.md @@ -347,6 +347,16 @@ MongoDB 的更改事件记录在消息之前没有更新。因此,我们只能 For example updating an already updated value in snapshot, or deleting an already deleted entry in snapshot. These replayed change log events should be handled specially. + + records.per.second + optional + true + Double + + 每秒发出的最大记录数,默认值为 -1,表示不进行速率限制。(仅适用于 flink 2.x)
+ 警告:增量/二进制日志阶段:速率限制可能导致连接器落后于上游变更流,在连接器赶上之前,二进制日志/WAL 可能会被清除(MySQL 数据丢失,PostgreSQL 复制槽问题)。 + + diff --git a/docs/content.zh/docs/connectors/flink-sources/mysql-cdc.md b/docs/content.zh/docs/connectors/flink-sources/mysql-cdc.md index 85b14141bef..049b88ee6b5 100644 --- a/docs/content.zh/docs/connectors/flink-sources/mysql-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/mysql-cdc.md @@ -454,6 +454,16 @@ Flink SQL> SELECT * FROM orders; 如果设置 'use.legacy.json.format' = 'false', 这条数据会被转换为 {"key1": "value1", "key2": "value2"}, 也就是 key 和 value 前的空格都会被保留。 + + records.per.second + optional + true + Double + + 每秒发出的最大记录数,默认值为 -1,表示不进行速率限制。(仅适用于 flink 2.x)
+ 警告:增量/二进制日志阶段:速率限制可能导致连接器落后于上游变更流,在连接器赶上之前,二进制日志/WAL 可能会被清除(MySQL 数据丢失,PostgreSQL 复制槽问题)。 + + diff --git a/docs/content.zh/docs/connectors/flink-sources/oceanbase-cdc.md b/docs/content.zh/docs/connectors/flink-sources/oceanbase-cdc.md index b077c8c2d69..37209fc9ae9 100644 --- a/docs/content.zh/docs/connectors/flink-sources/oceanbase-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/oceanbase-cdc.md @@ -468,6 +468,16 @@ Flink SQL> SELECT * FROM orders; 如果设置 'use.legacy.json.format' = 'false', 这条数据会被转换为 {"key1": "value1", "key2": "value2"}, 也就是 key 和 value 前的空格都会被保留。 + + records.per.second + optional + true + Double + + 每秒发出的最大记录数,默认值为 -1,表示不进行速率限制。(仅适用于 flink 2.x)
+ 警告:增量/二进制日志阶段:速率限制可能导致连接器落后于上游变更流,在连接器赶上之前,二进制日志/WAL 可能会被清除(MySQL 数据丢失,PostgreSQL 复制槽问题)。 + + diff --git a/docs/content.zh/docs/connectors/flink-sources/oracle-cdc.md b/docs/content.zh/docs/connectors/flink-sources/oracle-cdc.md index bae42ff03b6..e9a14829a0d 100644 --- a/docs/content.zh/docs/connectors/flink-sources/oracle-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/oracle-cdc.md @@ -454,6 +454,16 @@ Connector Options For example updating an already updated value in snapshot, or deleting an already deleted entry in snapshot. These replayed change log events should be handled specially. + + records.per.second + optional + true + Double + + The maximum number of records emitted per second, the default value: -1, means no rate limiting.(only for flink2.x)
+ WARNING: Incremental/binlog phase: Rate limiting can cause the connector to fall behind the upstream change stream, risking binlog/WAL purging before the connector catches up (data loss for MySQL, replication slot issues for PostgreSQL). + + diff --git a/docs/content.zh/docs/connectors/flink-sources/postgres-cdc.md b/docs/content.zh/docs/connectors/flink-sources/postgres-cdc.md index 24c6e67cd12..047b60d3d08 100644 --- a/docs/content.zh/docs/connectors/flink-sources/postgres-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/postgres-cdc.md @@ -292,6 +292,16 @@ Connector Options (2) Table list (regex or predefined list) should only match the parent table name, if table list matches both parent and child tables, snapshot data will be read twice. + + records.per.second + optional + true + Double + + 每秒发出的最大记录数,默认值为 -1,表示不进行速率限制。(仅适用于 flink 2.x)
+ 警告:增量/二进制日志阶段:速率限制可能导致连接器落后于上游变更流,在连接器赶上之前,二进制日志/WAL 可能会被清除(MySQL 数据丢失,PostgreSQL 复制槽问题)。 + + diff --git a/docs/content.zh/docs/connectors/flink-sources/sqlserver-cdc.md b/docs/content.zh/docs/connectors/flink-sources/sqlserver-cdc.md index 52f45cb68cf..e015c1088d9 100644 --- a/docs/content.zh/docs/connectors/flink-sources/sqlserver-cdc.md +++ b/docs/content.zh/docs/connectors/flink-sources/sqlserver-cdc.md @@ -263,6 +263,16 @@ Connector Options For example updating an already updated value in snapshot, or deleting an already deleted entry in snapshot. These replayed change log events should be handled specially. + + records.per.second + optional + true + Double + + The maximum number of records emitted per second, the default value: -1, means no rate limiting.(only for flink2.x)
+ WARNING: Incremental/binlog phase: Rate limiting can cause the connector to fall behind the upstream change stream, risking binlog/WAL purging before the connector catches up (data loss for MySQL, replication slot issues for PostgreSQL). + + diff --git a/docs/content.zh/docs/connectors/pipeline-connectors/mysql.md b/docs/content.zh/docs/connectors/pipeline-connectors/mysql.md index 04ec2844551..2be1325c592 100644 --- a/docs/content.zh/docs/connectors/pipeline-connectors/mysql.md +++ b/docs/content.zh/docs/connectors/pipeline-connectors/mysql.md @@ -362,6 +362,16 @@ pipeline: 如果设置 'use.legacy.json.format' = 'false', 这条数据会被转换为 {"key1": "value1", "key2": "value2"}, 也就是 key 和 value 前的空格都会被保留。 + + records.per.second + optional + true + Double + + 每秒发出的最大记录数,默认值为 -1,表示不进行速率限制。(仅适用于 flink 2.x)
+ 警告:增量/二进制日志阶段:速率限制可能导致连接器落后于上游变更流,在连接器赶上之前,二进制日志/WAL 可能会被清除(MySQL 数据丢失,PostgreSQL 复制槽问题)。 + + diff --git a/docs/content.zh/docs/connectors/pipeline-connectors/oracle.md b/docs/content.zh/docs/connectors/pipeline-connectors/oracle.md index 3a19a88fa89..e5c624b5153 100644 --- a/docs/content.zh/docs/connectors/pipeline-connectors/oracle.md +++ b/docs/content.zh/docs/connectors/pipeline-connectors/oracle.md @@ -269,6 +269,16 @@ pipeline: 是否在快照读取阶段跳过回填。如果跳过回填,则快照阶段捕获的表上的更改将在稍后的增量读取阶段被使用,而不是合并到快照中。警告:跳过回填可能会导致数据不一致,因为在快照阶段发生的一些binlog事件可能会被重放(至少只有一次语义承诺)。例如,更新快照中已更新的值,或删除快照中已删除的条目。这些重放的增量事件应该特别处理。 + + records.per.second + optional + true + Double + + 每秒发出的最大记录数,默认值为 -1,表示不进行速率限制。(仅适用于 flink 2.x)
+ 警告:增量/二进制日志阶段:速率限制可能导致连接器落后于上游变更流,在连接器赶上之前,二进制日志/WAL 可能会被清除(MySQL 数据丢失,PostgreSQL 复制槽问题)。 + + diff --git a/docs/content.zh/docs/connectors/pipeline-connectors/postgres.md b/docs/content.zh/docs/connectors/pipeline-connectors/postgres.md index 02fb272cf43..5dfffc667ad 100644 --- a/docs/content.zh/docs/connectors/pipeline-connectors/postgres.md +++ b/docs/content.zh/docs/connectors/pipeline-connectors/postgres.md @@ -294,6 +294,16 @@ pipeline: 默认值为 false。 + + records.per.second + optional + true + Double + + 每秒发出的最大记录数,默认值为 -1,表示不进行速率限制。(仅适用于 flink 2.x)
+ 警告:增量/二进制日志阶段:速率限制可能导致连接器落后于上游变更流,在连接器赶上之前,二进制日志/WAL 可能会被清除(MySQL 数据丢失,PostgreSQL 复制槽问题)。 + + diff --git a/docs/content/docs/connectors/flink-sources/db2-cdc.md b/docs/content/docs/connectors/flink-sources/db2-cdc.md index d2e6dcef5e6..972bedc6a30 100644 --- a/docs/content/docs/connectors/flink-sources/db2-cdc.md +++ b/docs/content/docs/connectors/flink-sources/db2-cdc.md @@ -274,6 +274,16 @@ Db2 server. This might help reduce the risk of the TaskManager experiencing an out-of-memory (OOM) error when taking a snapshot of the largest unbounded chunk.
+ + records.per.second + optional + true + Double + + The maximum number of records emitted per second, the default value: -1, means no rate limiting.(only for flink2.x)
+ WARNING: Incremental/binlog phase: Rate limiting can cause the connector to fall behind the upstream change stream, risking binlog/WAL purging before the connector catches up (data loss for MySQL, replication slot issues for PostgreSQL). + + diff --git a/docs/content/docs/connectors/flink-sources/mongodb-cdc.md b/docs/content/docs/connectors/flink-sources/mongodb-cdc.md index 45dc8716b4d..b793a0c1dca 100644 --- a/docs/content/docs/connectors/flink-sources/mongodb-cdc.md +++ b/docs/content/docs/connectors/flink-sources/mongodb-cdc.md @@ -372,6 +372,16 @@ Connector Options For example updating an already updated value in snapshot, or deleting an already deleted entry in snapshot. These replayed change log events should be handled specially. + + records.per.second + optional + true + Double + + The maximum number of records emitted per second, the default value: -1, means no rate limiting.(only for flink2.x)
+ WARNING: Incremental/binlog phase: Rate limiting can cause the connector to fall behind the upstream change stream, risking binlog/WAL purging before the connector catches up (data loss for MySQL, replication slot issues for PostgreSQL). + + diff --git a/docs/content/docs/connectors/flink-sources/mysql-cdc.md b/docs/content/docs/connectors/flink-sources/mysql-cdc.md index 2d47d9b7d71..51459c7b8b3 100644 --- a/docs/content/docs/connectors/flink-sources/mysql-cdc.md +++ b/docs/content/docs/connectors/flink-sources/mysql-cdc.md @@ -468,6 +468,16 @@ Only valid for cdc 1.x version. During a snapshot operation, the connector will For example updating an already updated value in snapshot, or deleting an already deleted entry in snapshot. These replayed change log events should be handled specially. + + records.per.second + optional + true + Double + + The maximum number of records emitted per second, the default value: -1, means no rate limiting.(only for flink2.x)
+ WARNING: Incremental/binlog phase: Rate limiting can cause the connector to fall behind the upstream change stream, risking binlog/WAL purging before the connector catches up (data loss for MySQL, replication slot issues for PostgreSQL). + + diff --git a/docs/content/docs/connectors/flink-sources/oceanbase-cdc.md b/docs/content/docs/connectors/flink-sources/oceanbase-cdc.md index 700a83c4555..586ba0d12ca 100644 --- a/docs/content/docs/connectors/flink-sources/oceanbase-cdc.md +++ b/docs/content/docs/connectors/flink-sources/oceanbase-cdc.md @@ -486,6 +486,16 @@ During a snapshot operation, the connector will query each included table to pro For example updating an already updated value in snapshot, or deleting an already deleted entry in snapshot. These replayed change log events should be handled specially. + + records.per.second + optional + true + Double + + The maximum number of records emitted per second, the default value: -1, means no rate limiting.(only for flink2.x)
+ WARNING: Incremental/binlog phase: Rate limiting can cause the connector to fall behind the upstream change stream, risking binlog/WAL purging before the connector catches up (data loss for MySQL, replication slot issues for PostgreSQL). + + diff --git a/docs/content/docs/connectors/flink-sources/oracle-cdc.md b/docs/content/docs/connectors/flink-sources/oracle-cdc.md index 3c720683b28..fe98e14f964 100644 --- a/docs/content/docs/connectors/flink-sources/oracle-cdc.md +++ b/docs/content/docs/connectors/flink-sources/oracle-cdc.md @@ -455,6 +455,16 @@ Connector Options For example updating an already updated value in snapshot, or deleting an already deleted entry in snapshot. These replayed change log events should be handled specially. + + records.per.second + optional + true + Double + + The maximum number of records emitted per second, the default value: -1, means no rate limiting.(only for flink2.x)
+ WARNING: Incremental/binlog phase: Rate limiting can cause the connector to fall behind the upstream change stream, risking binlog/WAL purging before the connector catches up (data loss for MySQL, replication slot issues for PostgreSQL). + + diff --git a/docs/content/docs/connectors/flink-sources/postgres-cdc.md b/docs/content/docs/connectors/flink-sources/postgres-cdc.md index d3c999ce4f7..50cbbcb8369 100644 --- a/docs/content/docs/connectors/flink-sources/postgres-cdc.md +++ b/docs/content/docs/connectors/flink-sources/postgres-cdc.md @@ -289,6 +289,16 @@ SELECT * FROM shipments; (2) Table list (regex or predefined list) should only match the parent table name, if table list matches both parent and child tables, snapshot data will be read twice. + + records.per.second + optional + true + Double + + The maximum number of records emitted per second, the default value: -1, means no rate limiting.(only for flink2.x)
+ WARNING: Incremental/binlog phase: Rate limiting can cause the connector to fall behind the upstream change stream, risking binlog/WAL purging before the connector catches up (data loss for MySQL, replication slot issues for PostgreSQL). + + diff --git a/docs/content/docs/connectors/flink-sources/sqlserver-cdc.md b/docs/content/docs/connectors/flink-sources/sqlserver-cdc.md index f2c274f6be5..2844515f3ed 100644 --- a/docs/content/docs/connectors/flink-sources/sqlserver-cdc.md +++ b/docs/content/docs/connectors/flink-sources/sqlserver-cdc.md @@ -263,6 +263,16 @@ Connector Options For example updating an already updated value in snapshot, or deleting an already deleted entry in snapshot. These replayed change log events should be handled specially. + + records.per.second + optional + true + Double + + The maximum number of records emitted per second, the default value: -1, means no rate limiting.(only for flink2.x)
+ WARNING: Incremental/binlog phase: Rate limiting can cause the connector to fall behind the upstream change stream, risking binlog/WAL purging before the connector catches up (data loss for MySQL, replication slot issues for PostgreSQL). + + diff --git a/docs/content/docs/connectors/pipeline-connectors/mysql.md b/docs/content/docs/connectors/pipeline-connectors/mysql.md index e6e61e20720..1790a2b066b 100644 --- a/docs/content/docs/connectors/pipeline-connectors/mysql.md +++ b/docs/content/docs/connectors/pipeline-connectors/mysql.md @@ -371,6 +371,16 @@ pipeline: List of readable metadata from SourceRecord to be passed to downstream and could be used in transform module, split by `,`. Available readable metadata are: op_ts. + + records.per.second + optional + true + Double + + The maximum number of records emitted per second, the default value: -1, means no rate limiting.(only for flink2.x)
+ WARNING: Incremental/binlog phase: Rate limiting can cause the connector to fall behind the upstream change stream, risking binlog/WAL purging before the connector catches up (data loss for MySQL, replication slot issues for PostgreSQL). + + diff --git a/docs/content/docs/connectors/pipeline-connectors/oracle.md b/docs/content/docs/connectors/pipeline-connectors/oracle.md index 5a791ae3632..ac9e1233eed 100644 --- a/docs/content/docs/connectors/pipeline-connectors/oracle.md +++ b/docs/content/docs/connectors/pipeline-connectors/oracle.md @@ -276,6 +276,16 @@ The max retry times that the connector should retry to build oracle database ser Whether to skip backfill in snapshot reading phase. If backfill is skipped, changes on captured tables during snapshot phase will be consumed later in incremental reading phase instead of being merged into the snapshot.WARNING: Skipping backfill might lead to data inconsistency because some incremental events happened within the snapshot phase might be replayed (only at-least-once semantic is promised). For example updating an already updated value in snapshot, or deleting an already deleted entry in snapshot. These replayed binlog events should be handled specially. + + records.per.second + optional + true + Double + + The maximum number of records emitted per second, the default value: -1, means no rate limiting.(only for flink2.x)
+ WARNING: Incremental/binlog phase: Rate limiting can cause the connector to fall behind the upstream change stream, risking binlog/WAL purging before the connector catches up (data loss for MySQL, replication slot issues for PostgreSQL). + + diff --git a/docs/content/docs/connectors/pipeline-connectors/postgres.md b/docs/content/docs/connectors/pipeline-connectors/postgres.md index 1deb246f50a..2fc8bbe124e 100644 --- a/docs/content/docs/connectors/pipeline-connectors/postgres.md +++ b/docs/content/docs/connectors/pipeline-connectors/postgres.md @@ -286,6 +286,16 @@ pipeline: Defaults to false. + + records.per.second + optional + true + Double + + The maximum number of records emitted per second, the default value: -1, means no rate limiting.(only for flink2.x)
+ WARNING: Incremental/binlog phase: Rate limiting can cause the connector to fall behind the upstream change stream, risking binlog/WAL purging before the connector catches up (data loss for MySQL, replication slot issues for PostgreSQL). + + diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java index 1b3540da0bb..a3555239a23 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java @@ -75,6 +75,7 @@ import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.PARSE_ONLINE_SCHEMA_CHANGES; import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.PASSWORD; import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.PORT; +import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.RECORDS_PER_SECOND; import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SCAN_BINLOG_NEWLY_ADDED_TABLE_ENABLED; import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED; import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP; @@ -167,6 +168,7 @@ public DataSource createDataSource(Context context) { boolean useLegacyJsonFormat = config.get(USE_LEGACY_JSON_FORMAT); boolean isAssignUnboundedChunkFirst = config.get(SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED); + double recordsPerSecond = config.get(RECORDS_PER_SECOND); validateIntegerOption(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE, splitSize, 1); validateIntegerOption(CHUNK_META_GROUP_SIZE, splitMetaGroupSize, 1); @@ -220,7 +222,8 @@ public DataSource createDataSource(Context context) { .treatTinyInt1AsBoolean(treatTinyInt1AsBoolean) .useLegacyJsonFormat(useLegacyJsonFormat) .assignUnboundedChunkFirst(isAssignUnboundedChunkFirst) - .skipSnapshotBackfill(skipSnapshotBackfill); + .skipSnapshotBackfill(skipSnapshotBackfill) + .recordsPerSecond(recordsPerSecond); List tableIds = MySqlSchemaUtils.listTables(configFactory.createConfig(0), null); @@ -358,6 +361,7 @@ public Set> optionalOptions() { options.add(PARSE_ONLINE_SCHEMA_CHANGES); options.add(SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED); options.add(SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP); + options.add(RECORDS_PER_SECOND); return options; } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/MySqlDataSourceOptions.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/MySqlDataSourceOptions.java index 6aff556e7fa..ea8ab5844e2 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/MySqlDataSourceOptions.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/MySqlDataSourceOptions.java @@ -330,4 +330,11 @@ public class MySqlDataSourceOptions { .defaultValue(false) .withDescription( "Whether to skip backfill in snapshot reading phase. If backfill is skipped, changes on captured tables during snapshot phase will be consumed later in change log reading phase instead of being merged into the snapshot.WARNING: Skipping backfill might lead to data inconsistency because some change log events happened within the snapshot phase might be replayed (only at-least-once semantic is promised). For example updating an already updated value in snapshot, or deleting an already deleted entry in snapshot. These replayed change log events should be handled specially."); + + public static final ConfigOption RECORDS_PER_SECOND = + ConfigOptions.key("records.per.second") + .doubleType() + .defaultValue(-1d) + .withDescription( + "The maximum number of records emitted per second, the default value: -1, means no rate limiting."); } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java index a2b4647fc72..3a796f84e13 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java @@ -117,6 +117,7 @@ public DataSource createDataSource(Context context) { config.get(OracleDataSourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED); boolean skipSnapshotBackfill = config.get(OracleDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP); + double recordsPerSecond = config.get(OracleDataSourceOptions.RECORDS_PER_SECOND); OracleSourceConfigFactory configFactory = new OracleSourceConfigFactory(); configFactory.url(url); configFactory.hostname(hostname); @@ -138,6 +139,7 @@ public DataSource createDataSource(Context context) { configFactory.skipSnapshotBackfill(skipSnapshotBackfill); configFactory.includeSchemaChanges(true); configFactory.serverTimeZone(serverTimeZone); + configFactory.recordsPerSecond(recordsPerSecond); Selectors selectors = new Selectors.SelectorsBuilder().includeTables(tables).build(); String[] capturedTables = getTableList(configFactory.create(0), selectors); @@ -204,6 +206,7 @@ public Set> optionalOptions() { options.add(OracleDataSourceOptions.LOG_MINING_STRATEGY); options.add(OracleDataSourceOptions.DATABASE_CONNECTION_ADAPTER); options.add(OracleDataSourceOptions.SCAN_STARTUP_MODE); + options.add(OracleDataSourceOptions.RECORDS_PER_SECOND); return options; } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSourceOptions.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSourceOptions.java index b0f1296250e..d7fe9a84639 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSourceOptions.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleDataSourceOptions.java @@ -208,6 +208,13 @@ public class OracleDataSourceOptions { "List of readable metadata from SourceRecord to be passed to downstream, split by `,`. " + "Available readable metadata are: op_ts."); + public static final ConfigOption RECORDS_PER_SECOND = + ConfigOptions.key("records.per.second") + .doubleType() + .defaultValue(-1d) + .withDescription( + "The maximum number of records emitted per second, the default value: -1, means no rate limiting."); + public static Map getPropertiesByPrefix( Configuration tableOptions, String prefix) { final Map props = new HashMap<>(); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/factory/PostgresDataSourceFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/factory/PostgresDataSourceFactory.java index 93d8657cf56..361e7ff8f7f 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/factory/PostgresDataSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/factory/PostgresDataSourceFactory.java @@ -66,6 +66,7 @@ import static org.apache.flink.cdc.connectors.postgres.source.PostgresDataSourceOptions.METADATA_LIST; import static org.apache.flink.cdc.connectors.postgres.source.PostgresDataSourceOptions.PASSWORD; import static org.apache.flink.cdc.connectors.postgres.source.PostgresDataSourceOptions.PG_PORT; +import static org.apache.flink.cdc.connectors.postgres.source.PostgresDataSourceOptions.RECORDS_PER_SECOND; import static org.apache.flink.cdc.connectors.postgres.source.PostgresDataSourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED; import static org.apache.flink.cdc.connectors.postgres.source.PostgresDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP; import static org.apache.flink.cdc.connectors.postgres.source.PostgresDataSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN; @@ -133,6 +134,7 @@ public DataSource createDataSource(Context context) { int lsnCommitCheckpointsDelay = config.get(SCAN_LSN_COMMIT_CHECKPOINTS_DELAY); boolean tableIdIncludeDatabase = config.get(TABLE_ID_INCLUDE_DATABASE); boolean includeSchemaChanges = config.get(SCHEMA_CHANGE_ENABLED); + double recordsPerSecond = config.get(RECORDS_PER_SECOND); validateIntegerOption(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE, splitSize, 1); validateIntegerOption(CHUNK_META_GROUP_SIZE, splitMetaGroupSize, 1); @@ -175,6 +177,7 @@ public DataSource createDataSource(Context context) { .assignUnboundedChunkFirst(isAssignUnboundedChunkFirst) .includeDatabaseInTableId(tableIdIncludeDatabase) .includeSchemaChanges(includeSchemaChanges) + .recordsPerSecond(recordsPerSecond) .getConfigFactory(); List tableIds = PostgresSchemaUtils.listTables(configFactory.create(0), null); @@ -266,6 +269,7 @@ public Set> optionalOptions() { options.add(SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED); options.add(TABLE_ID_INCLUDE_DATABASE); options.add(SCHEMA_CHANGE_ENABLED); + options.add(RECORDS_PER_SECOND); return options; } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/source/PostgresDataSourceOptions.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/source/PostgresDataSourceOptions.java index 95cc823d211..394da3d62c2 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/source/PostgresDataSourceOptions.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/source/PostgresDataSourceOptions.java @@ -281,4 +281,11 @@ public class PostgresDataSourceOptions { .defaultValue(false) .withDescription( "Whether to infer CDC column types when processing pgoutput Relation messages."); + + public static final ConfigOption RECORDS_PER_SECOND = + ConfigOptions.key("records.per.second") + .doubleType() + .defaultValue(-1d) + .withDescription( + "The maximum number of records emitted per second, the default value: -1, means no rate limiting."); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/config/BaseSourceConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/config/BaseSourceConfig.java index b8ce8dd4448..c2ceffc14bc 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/config/BaseSourceConfig.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/config/BaseSourceConfig.java @@ -39,6 +39,7 @@ public abstract class BaseSourceConfig implements SourceConfig { protected final boolean skipSnapshotBackfill; protected final boolean isScanNewlyAddedTableEnabled; protected final boolean assignUnboundedChunkFirst; + protected final double recordsPerSecond; // -------------------------------------------------------------------------------------------- // Debezium Configurations @@ -58,7 +59,8 @@ public BaseSourceConfig( boolean isScanNewlyAddedTableEnabled, Properties dbzProperties, Configuration dbzConfiguration, - boolean assignUnboundedChunkFirst) { + boolean assignUnboundedChunkFirst, + double recordsPerSecond) { this.startupOptions = startupOptions; this.splitSize = splitSize; this.splitMetaGroupSize = splitMetaGroupSize; @@ -71,6 +73,7 @@ public BaseSourceConfig( this.dbzProperties = dbzProperties; this.dbzConfiguration = dbzConfiguration; this.assignUnboundedChunkFirst = assignUnboundedChunkFirst; + this.recordsPerSecond = recordsPerSecond; } @Override @@ -123,4 +126,9 @@ public boolean isSkipSnapshotBackfill() { public boolean isAssignUnboundedChunkFirst() { return assignUnboundedChunkFirst; } + + @Override + public double getRecordsPerSecond() { + return recordsPerSecond; + } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/config/JdbcSourceConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/config/JdbcSourceConfig.java index afa730667e0..bcc7f07a5f4 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/config/JdbcSourceConfig.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/config/JdbcSourceConfig.java @@ -74,7 +74,8 @@ public JdbcSourceConfig( String chunkKeyColumn, boolean skipSnapshotBackfill, boolean isScanNewlyAddedTableEnabled, - boolean assignUnboundedChunkFirst) { + boolean assignUnboundedChunkFirst, + double recordsPerSecond) { super( startupOptions, splitSize, @@ -87,7 +88,8 @@ public JdbcSourceConfig( isScanNewlyAddedTableEnabled, dbzProperties, dbzConfiguration, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); this.driverClassName = driverClassName; this.hostname = hostname; this.port = port; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/config/JdbcSourceConfigFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/config/JdbcSourceConfigFactory.java index 1c21a8796d0..f1baea25dd5 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/config/JdbcSourceConfigFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/config/JdbcSourceConfigFactory.java @@ -63,6 +63,7 @@ public abstract class JdbcSourceConfigFactory implements Factory extends Serializable { diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/options/JdbcSourceOptions.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/options/JdbcSourceOptions.java index 42247636460..3bb3957d8cd 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/options/JdbcSourceOptions.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/options/JdbcSourceOptions.java @@ -110,4 +110,11 @@ public class JdbcSourceOptions extends SourceOptions { .withDescription( "The chunk key of table snapshot, captured tables are split into multiple chunks by a chunk key when read the snapshot of table." + "By default, the chunk key is the first column of the primary key and the chunk key is the RowId in oracle."); + + public static final ConfigOption RECORDS_PER_SECOND = + ConfigOptions.key("records.per.second") + .doubleType() + .defaultValue(-1d) + .withDescription( + "The maximum number of records emitted per second, the default value: -1, means no rate limiting."); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceReader.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceReader.java index 402d5107414..cc87e14d0d9 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceReader.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceReader.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.connectors.base.source.reader; import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; import org.apache.flink.cdc.common.annotation.Experimental; import org.apache.flink.cdc.common.annotation.VisibleForTesting; import org.apache.flink.cdc.connectors.base.config.SourceConfig; @@ -45,7 +46,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordEmitter; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; +import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBaseAdapter; import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.util.FlinkRuntimeException; @@ -78,7 +79,7 @@ */ @Experimental public class IncrementalSourceReader - extends SingleThreadMultiplexSourceReaderBase< + extends SingleThreadMultiplexSourceReaderBaseAdapter< SourceRecords, T, SourceSplitBase, SourceSplitState> { private static final Logger LOG = LoggerFactory.getLogger(IncrementalSourceReader.class); @@ -121,8 +122,12 @@ public IncrementalSourceReader( super( new SingleThreadFetcherManager<>(splitReaderSupplier::get), recordEmitter, + null, config, - incrementalSourceReaderContext.getSourceReaderContext()); + incrementalSourceReaderContext.getSourceReaderContext(), + sourceConfig.getRecordsPerSecond() == -1 + ? null + : RateLimiterStrategy.perSecond(sourceConfig.getRecordsPerSecond())); this.sourceConfig = sourceConfig; this.finishedUnackedSplits = new HashMap<>(); this.uncompletedStreamSplits = new HashMap<>(); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/GenericConnectionPoolTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/GenericConnectionPoolTest.java index 754a936fcb7..2cf5898e1c4 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/GenericConnectionPoolTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/GenericConnectionPoolTest.java @@ -126,7 +126,8 @@ private static MockedSourceConfig getMockMySqlSourceConfig( Duration.ofSeconds(10), 2, 3, - false); + false, + -1); } private static class MockConnectionPoolFactory extends JdbcConnectionPoolFactory { diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/mocked/MockedSourceConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/mocked/MockedSourceConfig.java index 801f2ea9837..38eca7a643e 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/mocked/MockedSourceConfig.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/mocked/MockedSourceConfig.java @@ -59,7 +59,8 @@ public MockedSourceConfig( Duration connectTimeout, int connectMaxRetries, int connectionPoolSize, - boolean isScanNewlyAddedTableEnabled) { + boolean isScanNewlyAddedTableEnabled, + double recordsPerSecond) { super( startupOptions, databaseList, @@ -86,7 +87,8 @@ public MockedSourceConfig( null, true, isScanNewlyAddedTableEnabled, - false); + false, + recordsPerSecond); } @Override diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/Db2SourceBuilder.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/Db2SourceBuilder.java index 8048adfbdfd..434d6633bc9 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/Db2SourceBuilder.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/Db2SourceBuilder.java @@ -241,6 +241,11 @@ public Db2SourceBuilder assignUnboundedChunkFirst(boolean assignUnboundedChun return this; } + public Db2SourceBuilder recordsPerSecond(double recordsPerSecond) { + this.configFactory.recordsPerSecond(recordsPerSecond); + return this; + } + /** * Build the {@link Db2IncrementalSource}. * diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/config/Db2SourceConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/config/Db2SourceConfig.java index ab35aa409ea..2c512c5af16 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/config/Db2SourceConfig.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/config/Db2SourceConfig.java @@ -57,7 +57,8 @@ public Db2SourceConfig( int connectionPoolSize, String chunkKeyColumn, boolean skipSnapshotBackfill, - boolean assignUnboundedChunkFirst) { + boolean assignUnboundedChunkFirst, + double recordsPerSecond) { super( startupOptions, databaseList, @@ -84,7 +85,8 @@ public Db2SourceConfig( chunkKeyColumn, skipSnapshotBackfill, false, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } @Override diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/config/Db2SourceConfigFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/config/Db2SourceConfigFactory.java index 365e7871099..42659de6bbd 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/config/Db2SourceConfigFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/config/Db2SourceConfigFactory.java @@ -104,6 +104,7 @@ public Db2SourceConfig create(int subtask) { connectionPoolSize, chunkKeyColumn, skipSnapshotBackfill, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSource.java index 12c8d27e04b..98f7c7c370c 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSource.java @@ -79,6 +79,7 @@ public class Db2TableSource implements ScanTableSource, SupportsReadingMetadata private final boolean closeIdleReaders; private final boolean skipSnapshotBackfill; private final boolean assignUnboundedChunkFirst; + private final double recordsPerSecond; /** Metadata that is appended at the end of a physical source row. */ protected List metadataKeys; @@ -106,7 +107,8 @@ public Db2TableSource( @Nullable String chunkKeyColumn, boolean closeIdleReaders, boolean skipSnapshotBackfill, - boolean assignUnboundedChunkFirst) { + boolean assignUnboundedChunkFirst, + double recordsPerSecond) { this.physicalSchema = physicalSchema; this.port = port; this.hostname = hostname; @@ -132,6 +134,7 @@ public Db2TableSource( this.closeIdleReaders = closeIdleReaders; this.skipSnapshotBackfill = skipSnapshotBackfill; this.assignUnboundedChunkFirst = assignUnboundedChunkFirst; + this.recordsPerSecond = recordsPerSecond; } @Override @@ -179,6 +182,7 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { .closeIdleReaders(closeIdleReaders) .skipSnapshotBackfill(skipSnapshotBackfill) .assignUnboundedChunkFirst(assignUnboundedChunkFirst) + .recordsPerSecond(recordsPerSecond) .build(); return SourceProvider.of(db2ChangeEventSource); } else { @@ -240,7 +244,8 @@ public DynamicTableSource copy() { chunkKeyColumn, closeIdleReaders, skipSnapshotBackfill, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); source.metadataKeys = metadataKeys; source.producedDataType = producedDataType; return source; @@ -277,7 +282,8 @@ public boolean equals(Object o) { && Objects.equals(chunkKeyColumn, that.chunkKeyColumn) && Objects.equals(closeIdleReaders, that.closeIdleReaders) && Objects.equals(skipSnapshotBackfill, that.skipSnapshotBackfill) - && Objects.equals(assignUnboundedChunkFirst, that.assignUnboundedChunkFirst); + && Objects.equals(assignUnboundedChunkFirst, that.assignUnboundedChunkFirst) + && Objects.equals(recordsPerSecond, that.recordsPerSecond); } @Override @@ -305,7 +311,8 @@ public int hashCode() { chunkKeyColumn, closeIdleReaders, skipSnapshotBackfill, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } @Override diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSourceFactory.java index 529a8f0bc48..fdf989f1c06 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSourceFactory.java @@ -37,6 +37,7 @@ import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECTION_POOL_SIZE; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_MAX_RETRIES; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_TIMEOUT; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.RECORDS_PER_SECOND; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN; import static org.apache.flink.cdc.connectors.base.options.SourceOptions.CHUNK_META_GROUP_SIZE; import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED; @@ -141,6 +142,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { boolean skipSnapshotBackfill = config.get(SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP); boolean assignUnboundedChunkFirst = config.get(SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED); + double recordsPerSecond = config.get(RECORDS_PER_SECOND); if (enableParallelRead) { validateIntegerOption(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE, splitSize, 1); @@ -177,7 +179,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { chunkKeyColumn, closeIdleReaders, skipSnapshotBackfill, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } @Override @@ -215,6 +218,7 @@ public Set> optionalOptions() { options.add(SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED); options.add(SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP); options.add(SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED); + options.add(RECORDS_PER_SECOND); return options; } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/test/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSourceFactoryTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/test/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSourceFactoryTest.java index a8a453174a0..edfbc9a6b3b 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/test/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSourceFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/test/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSourceFactoryTest.java @@ -44,6 +44,7 @@ import java.util.Map; import java.util.Properties; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.RECORDS_PER_SECOND; import static org.apache.flink.cdc.debezium.utils.ResolvedSchemaUtils.getPhysicalSchema; import static org.apache.flink.table.api.TableSchema.fromResolvedSchema; @@ -84,6 +85,8 @@ class Db2TableSourceFactoryTest { private static final String MY_TABLE = "flinkuser.myTable"; private static final Properties PROPERTIES = new Properties(); + private static final double RECORDS_PER_SECOND_DEFAULT = RECORDS_PER_SECOND.defaultValue(); + @Test void testCommonProperties() { Map properties = getAllOptions(); @@ -117,7 +120,8 @@ void testCommonProperties() { false, JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP.defaultValue(), JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED - .defaultValue()); + .defaultValue(), + RECORDS_PER_SECOND_DEFAULT); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -158,7 +162,8 @@ void testOptionalProperties() { null, false, JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP.defaultValue(), - true); + true, + RECORDS_PER_SECOND_DEFAULT); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -233,7 +238,8 @@ void testMetadataColumns() { false, JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP.defaultValue(), JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED - .defaultValue()); + .defaultValue(), + RECORDS_PER_SECOND_DEFAULT); expectedSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType(); expectedSource.metadataKeys = Arrays.asList("op_ts", "database_name", "table_name", "schema_name"); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBSourceBuilder.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBSourceBuilder.java index c61a5d0ce99..32c2e2ac5b5 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBSourceBuilder.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBSourceBuilder.java @@ -277,6 +277,11 @@ public MongoDBSourceBuilder assignUnboundedChunkFirst(boolean assignUnbounded return this; } + public MongoDBSourceBuilder recordsPerSecond(double recordsPerSecond) { + this.configFactory.recordsPerSecond(recordsPerSecond); + return this; + } + /** * Build the {@link MongoDBSource}. * diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/config/MongoDBSourceConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/config/MongoDBSourceConfig.java index 03562618822..6e3bdd1dc22 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/config/MongoDBSourceConfig.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/config/MongoDBSourceConfig.java @@ -56,6 +56,7 @@ public class MongoDBSourceConfig implements SourceConfig { private final boolean skipSnapshotBackfill; private final boolean isScanNewlyAddedTableEnabled; private final boolean assignUnboundedChunkFirst; + private final double recordsPerSecond; MongoDBSourceConfig( String scheme, @@ -79,7 +80,8 @@ public class MongoDBSourceConfig implements SourceConfig { boolean disableCursorTimeout, boolean skipSnapshotBackfill, boolean isScanNewlyAddedTableEnabled, - boolean assignUnboundedChunkFirst) { + boolean assignUnboundedChunkFirst, + double recordsPerSecond) { this.scheme = checkNotNull(scheme); this.hosts = checkNotNull(hosts); this.username = username; @@ -103,6 +105,7 @@ public class MongoDBSourceConfig implements SourceConfig { this.skipSnapshotBackfill = skipSnapshotBackfill; this.isScanNewlyAddedTableEnabled = isScanNewlyAddedTableEnabled; this.assignUnboundedChunkFirst = assignUnboundedChunkFirst; + this.recordsPerSecond = recordsPerSecond; } public String getScheme() { @@ -207,6 +210,11 @@ public boolean isAssignUnboundedChunkFirst() { return assignUnboundedChunkFirst; } + @Override + public double getRecordsPerSecond() { + return recordsPerSecond; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -234,7 +242,8 @@ public boolean equals(Object o) { && Objects.equals(collectionList, that.collectionList) && Objects.equals(connectionString, that.connectionString) && Objects.equals(skipSnapshotBackfill, that.skipSnapshotBackfill) - && Objects.equals(isScanNewlyAddedTableEnabled, that.isScanNewlyAddedTableEnabled); + && Objects.equals(isScanNewlyAddedTableEnabled, that.isScanNewlyAddedTableEnabled) + && recordsPerSecond == that.recordsPerSecond; } @Override @@ -258,6 +267,7 @@ public int hashCode() { samplesPerChunk, closeIdleReaders, skipSnapshotBackfill, - isScanNewlyAddedTableEnabled); + isScanNewlyAddedTableEnabled, + recordsPerSecond); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/config/MongoDBSourceConfigFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/config/MongoDBSourceConfigFactory.java index 691f3712a1a..0879ca09d25 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/config/MongoDBSourceConfigFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/config/MongoDBSourceConfigFactory.java @@ -63,6 +63,7 @@ public class MongoDBSourceConfigFactory implements Factory protected boolean skipSnapshotBackfill = false; protected boolean scanNewlyAddedTableEnabled = false; protected boolean assignUnboundedChunkFirst = false; + private double recordsPerSecond = MongoDBSourceOptions.RECORDS_PER_SECOND.defaultValue(); /** The protocol connected to MongoDB. For example mongodb or mongodb+srv. */ public MongoDBSourceConfigFactory scheme(String scheme) { @@ -280,6 +281,11 @@ public MongoDBSourceConfigFactory assignUnboundedChunkFirst(boolean assignUnboun return this; } + public MongoDBSourceConfigFactory recordsPerSecond(double recordsPerSecond) { + this.recordsPerSecond = recordsPerSecond; + return this; + } + /** Creates a new {@link MongoDBSourceConfig} for the given subtask {@code subtaskId}. */ @Override public MongoDBSourceConfig create(int subtaskId) { @@ -306,6 +312,7 @@ public MongoDBSourceConfig create(int subtaskId) { disableCursorTimeout, skipSnapshotBackfill, scanNewlyAddedTableEnabled, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/config/MongoDBSourceOptions.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/config/MongoDBSourceOptions.java index 3e4cf609667..d13a4d53606 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/config/MongoDBSourceOptions.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/source/config/MongoDBSourceOptions.java @@ -178,4 +178,11 @@ public class MongoDBSourceOptions { .defaultValue(true) .withDescription( "MongoDB server normally times out idle cursors after an inactivity period (10 minutes) to prevent excess memory use. Set this option to true to prevent that."); + + public static final ConfigOption RECORDS_PER_SECOND = + ConfigOptions.key("records.per.second") + .doubleType() + .defaultValue(-1d) + .withDescription( + "The maximum number of records emitted per second, the default value: -1, means no rate limiting."); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableSource.java index 03d00626473..78453e67772 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableSource.java @@ -89,6 +89,7 @@ public class MongoDBTableSource implements ScanTableSource, SupportsReadingMetad private final boolean skipSnapshotBackfill; private final boolean scanNewlyAddedTableEnabled; private final boolean assignUnboundedChunkFirst; + private final double recordsPerSecond; // -------------------------------------------------------------------------------------------- // Mutable attributes @@ -127,7 +128,8 @@ public MongoDBTableSource( boolean noCursorTimeout, boolean skipSnapshotBackfill, boolean scanNewlyAddedTableEnabled, - boolean assignUnboundedChunkFirst) { + boolean assignUnboundedChunkFirst, + double recordsPerSecond) { this.physicalSchema = physicalSchema; this.scheme = checkNotNull(scheme); this.hosts = checkNotNull(hosts); @@ -157,6 +159,7 @@ public MongoDBTableSource( this.skipSnapshotBackfill = skipSnapshotBackfill; this.scanNewlyAddedTableEnabled = scanNewlyAddedTableEnabled; this.assignUnboundedChunkFirst = assignUnboundedChunkFirst; + this.recordsPerSecond = recordsPerSecond; } @Override @@ -217,7 +220,8 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { .scanNewlyAddedTableEnabled(scanNewlyAddedTableEnabled) .deserializer(deserializer) .disableCursorTimeout(noCursorTimeout) - .assignUnboundedChunkFirst(assignUnboundedChunkFirst); + .assignUnboundedChunkFirst(assignUnboundedChunkFirst) + .recordsPerSecond(recordsPerSecond); Optional.ofNullable(databaseList).ifPresent(builder::databaseList); Optional.ofNullable(collectionList).ifPresent(builder::collectionList); @@ -324,7 +328,8 @@ public DynamicTableSource copy() { noCursorTimeout, skipSnapshotBackfill, scanNewlyAddedTableEnabled, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); source.metadataKeys = metadataKeys; source.producedDataType = producedDataType; return source; @@ -367,7 +372,8 @@ public boolean equals(Object o) { && Objects.equals(noCursorTimeout, that.noCursorTimeout) && Objects.equals(skipSnapshotBackfill, that.skipSnapshotBackfill) && Objects.equals(scanNewlyAddedTableEnabled, that.scanNewlyAddedTableEnabled) - && Objects.equals(assignUnboundedChunkFirst, that.assignUnboundedChunkFirst); + && Objects.equals(assignUnboundedChunkFirst, that.assignUnboundedChunkFirst) + && Objects.equals(recordsPerSecond, that.recordsPerSecond); } @Override @@ -401,7 +407,8 @@ public int hashCode() { noCursorTimeout, skipSnapshotBackfill, scanNewlyAddedTableEnabled, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } @Override diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableSourceFactory.java index 71770cf1e38..3bc576fcd08 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableSourceFactory.java @@ -54,6 +54,7 @@ import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.PASSWORD; import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.POLL_AWAIT_TIME_MILLIS; import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.POLL_MAX_BATCH_SIZE; +import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.RECORDS_PER_SECOND; import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SAMPLES; import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE_MB; import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED; @@ -139,6 +140,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { config.getOptional(FULL_DOCUMENT_PRE_POST_IMAGE).orElse(false); boolean noCursorTimeout = config.getOptional(SCAN_NO_CURSOR_TIMEOUT).orElse(true); + + double recordsPerSecond = config.get(RECORDS_PER_SECOND); ResolvedSchema physicalSchema = getPhysicalSchema(context.getCatalogTable().getResolvedSchema()); checkArgument(physicalSchema.getPrimaryKey().isPresent(), "Primary key must be present"); @@ -173,7 +176,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { noCursorTimeout, skipSnapshotBackfill, scanNewlyAddedTableEnabled, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } private void checkPrimaryKey(UniqueConstraint pk, String message) { @@ -258,6 +262,7 @@ public Set> optionalOptions() { options.add(SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP); options.add(SCAN_NEWLY_ADDED_TABLE_ENABLED); options.add(SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED); + options.add(RECORDS_PER_SECOND); return options; } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableFactoryTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableFactoryTest.java index 41f53c8e824..90dc9fff354 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableFactoryTest.java @@ -57,6 +57,7 @@ import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.HEARTBEAT_INTERVAL_MILLIS; import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.POLL_AWAIT_TIME_MILLIS; import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.POLL_MAX_BATCH_SIZE; +import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.RECORDS_PER_SECOND; import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SAMPLES; import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE_MB; import static org.apache.flink.cdc.connectors.mongodb.source.config.MongoDBSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED; @@ -125,6 +126,8 @@ class MongoDBTableFactoryTest { private static final boolean SCAN_NEWLY_ADDED_TABLE_ENABLED_DEFAULT = SCAN_NEWLY_ADDED_TABLE_ENABLED.defaultValue(); + private static final double RECORDS_PER_SECOND_DEFAULT = RECORDS_PER_SECOND.defaultValue(); + @Test void testCommonProperties() { Map properties = getAllOptions(); @@ -159,7 +162,8 @@ void testCommonProperties() { SCAN_NO_CURSOR_TIMEOUT_DEFAULT, SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP_DEFAULT, SCAN_NEWLY_ADDED_TABLE_ENABLED_DEFAULT, - SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED.defaultValue()); + SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED.defaultValue(), + RECORDS_PER_SECOND_DEFAULT); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -215,7 +219,8 @@ void testOptionalProperties() { false, true, true, - true); + true, + RECORDS_PER_SECOND_DEFAULT); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -259,7 +264,8 @@ void testMetadataColumns() { SCAN_NO_CURSOR_TIMEOUT_DEFAULT, SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP_DEFAULT, SCAN_NEWLY_ADDED_TABLE_ENABLED_DEFAULT, - SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED.defaultValue()); + SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED.defaultValue(), + RECORDS_PER_SECOND_DEFAULT); expectedSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType(); expectedSource.metadataKeys = Arrays.asList("op_ts", "database_name", "row_kind"); @@ -370,7 +376,8 @@ public void testCopyExistingPipelineInDebeziumMode() { SCAN_NO_CURSOR_TIMEOUT_DEFAULT, SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP_DEFAULT, SCAN_NEWLY_ADDED_TABLE_ENABLED_DEFAULT, - SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED.defaultValue()); + SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED.defaultValue(), + RECORDS_PER_SECOND_DEFAULT); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/MySqlSourceBuilder.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/MySqlSourceBuilder.java index 93fa2a0d36a..72d34c358c0 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/MySqlSourceBuilder.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/MySqlSourceBuilder.java @@ -312,6 +312,11 @@ public MySqlSourceBuilder assignUnboundedChunkFirst(boolean assignUnboundedCh return this; } + public MySqlSourceBuilder recordsPerSecond(double recordsPerSecond) { + this.configFactory.recordsPerSecond(recordsPerSecond); + return this; + } + /** * Build the {@link MySqlSource}. * diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/config/MySqlSourceConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/config/MySqlSourceConfig.java index cf456fcaed0..db5c2c4518a 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/config/MySqlSourceConfig.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/config/MySqlSourceConfig.java @@ -81,6 +81,8 @@ public class MySqlSourceConfig implements Serializable { private final MySqlConnectorConfig dbzMySqlConfig; private final boolean treatTinyInt1AsBoolean; + private final double recordsPerSecond; + MySqlSourceConfig( String hostname, int port, @@ -112,7 +114,8 @@ public class MySqlSourceConfig implements Serializable { boolean parseOnLineSchemaChanges, boolean treatTinyInt1AsBoolean, boolean useLegacyJsonFormat, - boolean assignUnboundedChunkFirst) { + boolean assignUnboundedChunkFirst, + double recordsPerSecond) { this.hostname = checkNotNull(hostname); this.port = port; this.username = checkNotNull(username); @@ -158,6 +161,7 @@ public class MySqlSourceConfig implements Serializable { this.treatTinyInt1AsBoolean = treatTinyInt1AsBoolean; this.useLegacyJsonFormat = useLegacyJsonFormat; this.assignUnboundedChunkFirst = assignUnboundedChunkFirst; + this.recordsPerSecond = recordsPerSecond; } public String getHostname() { @@ -299,4 +303,8 @@ public boolean isSkipSnapshotBackfill() { public boolean isTreatTinyInt1AsBoolean() { return treatTinyInt1AsBoolean; } + + public double getRecordsPerSecond() { + return recordsPerSecond; + } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/config/MySqlSourceConfigFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/config/MySqlSourceConfigFactory.java index 569b62232db..75bc46ef61f 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/config/MySqlSourceConfigFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/config/MySqlSourceConfigFactory.java @@ -78,6 +78,7 @@ public class MySqlSourceConfigFactory implements Serializable { private boolean treatTinyInt1AsBoolean = true; private boolean useLegacyJsonFormat = true; private boolean assignUnboundedChunkFirst = false; + private double recordsPerSecond = MySqlSourceOptions.RECORDS_PER_SECOND.defaultValue(); public MySqlSourceConfigFactory hostname(String hostname) { this.hostname = hostname; @@ -341,6 +342,11 @@ public MySqlSourceConfigFactory assignUnboundedChunkFirst(boolean assignUnbounde return this; } + public MySqlSourceConfigFactory recordsPerSecond(double recordsPerSecond) { + this.recordsPerSecond = recordsPerSecond; + return this; + } + /** Creates a new {@link MySqlSourceConfig} for the given subtask {@code subtaskId}. */ public MySqlSourceConfig createConfig(int subtaskId) { // hard code server name, because we don't need to distinguish it, docs: @@ -444,6 +450,7 @@ public MySqlSourceConfig createConfig(int subtaskId, String serverName) { parseOnLineSchemaChanges, treatTinyInt1AsBoolean, useLegacyJsonFormat, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/config/MySqlSourceOptions.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/config/MySqlSourceOptions.java index a8e143f5fc5..b28a2761347 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/config/MySqlSourceOptions.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/config/MySqlSourceOptions.java @@ -292,4 +292,11 @@ public class MySqlSourceOptions { .defaultValue(true) .withDescription( "Whether to assign the unbounded chunks first during snapshot reading phase. This might help reduce the risk of the TaskManager experiencing an out-of-memory (OOM) error when taking a snapshot of the largest unbounded chunk."); + + public static final ConfigOption RECORDS_PER_SECOND = + ConfigOptions.key("records.per.second") + .doubleType() + .defaultValue(-1d) + .withDescription( + "The maximum number of records emitted per second, the default value: -1, means no rate limiting."); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/reader/MySqlSourceReader.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/reader/MySqlSourceReader.java index b3f421ceecc..d07f1a79381 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/reader/MySqlSourceReader.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/reader/MySqlSourceReader.java @@ -18,9 +18,11 @@ package org.apache.flink.cdc.connectors.mysql.source.reader; import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; import org.apache.flink.cdc.common.annotation.VisibleForTesting; import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; +import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; import org.apache.flink.cdc.connectors.mysql.source.events.BinlogSplitAssignedEvent; import org.apache.flink.cdc.connectors.mysql.source.events.BinlogSplitMetaEvent; import org.apache.flink.cdc.connectors.mysql.source.events.BinlogSplitMetaRequestEvent; @@ -43,7 +45,7 @@ import org.apache.flink.cdc.connectors.mysql.source.utils.ChunkUtils; import org.apache.flink.cdc.connectors.mysql.source.utils.TableDiscoveryUtils; import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; +import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBaseAdapter; import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; @@ -68,7 +70,7 @@ /** The source reader for MySQL source splits. */ public class MySqlSourceReader - extends SingleThreadMultiplexSourceReaderBase< + extends SingleThreadMultiplexSourceReaderBaseAdapter< SourceRecords, T, MySqlSplit, MySqlSplitState> { private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceReader.class); @@ -90,8 +92,13 @@ public MySqlSourceReader( super( new SingleThreadFetcherManager<>(splitReaderSupplier::get), recordEmitter, + null, config, - context.getSourceReaderContext()); + context.getSourceReaderContext(), + sourceConfig.getRecordsPerSecond() + == MySqlSourceOptions.RECORDS_PER_SECOND.defaultValue() + ? null + : RateLimiterStrategy.perSecond(sourceConfig.getRecordsPerSecond())); this.recordEmitter = recordEmitter; this.sourceConfig = sourceConfig; this.finishedUnackedSplits = new HashMap<>(); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSource.java index 2a1f0519435..ba6650f2949 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSource.java @@ -104,6 +104,8 @@ public class MySqlTableSource implements ScanTableSource, SupportsReadingMetadat private final boolean appendOnly; + private final double recordsPerSecond; + // -------------------------------------------------------------------------------------------- // Mutable attributes // -------------------------------------------------------------------------------------------- @@ -144,7 +146,8 @@ public MySqlTableSource( boolean parseOnlineSchemaChanges, boolean useLegacyJsonFormat, boolean assignUnboundedChunkFirst, - boolean appendOnly) { + boolean appendOnly, + double recordsPerSecond) { this.physicalSchema = physicalSchema; this.port = port; this.hostname = checkNotNull(hostname); @@ -178,6 +181,7 @@ public MySqlTableSource( this.useLegacyJsonFormat = useLegacyJsonFormat; this.assignUnboundedChunkFirst = assignUnboundedChunkFirst; this.appendOnly = appendOnly; + this.recordsPerSecond = recordsPerSecond; } @Override @@ -241,6 +245,7 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { .parseOnLineSchemaChanges(parseOnlineSchemaChanges) .useLegacyJsonFormat(useLegacyJsonFormat) .assignUnboundedChunkFirst(assignUnboundedChunkFirst) + .recordsPerSecond(recordsPerSecond) .build(); return SourceProvider.of(parallelSource); } else { @@ -330,7 +335,8 @@ public DynamicTableSource copy() { parseOnlineSchemaChanges, useLegacyJsonFormat, assignUnboundedChunkFirst, - appendOnly); + appendOnly, + recordsPerSecond); source.metadataKeys = metadataKeys; source.producedDataType = producedDataType; return source; @@ -376,7 +382,8 @@ public boolean equals(Object o) { && parseOnlineSchemaChanges == that.parseOnlineSchemaChanges && useLegacyJsonFormat == that.useLegacyJsonFormat && assignUnboundedChunkFirst == that.assignUnboundedChunkFirst - && Objects.equals(appendOnly, that.appendOnly); + && Objects.equals(appendOnly, that.appendOnly) + && recordsPerSecond == that.recordsPerSecond; } @Override @@ -413,7 +420,8 @@ public int hashCode() { parseOnlineSchemaChanges, useLegacyJsonFormat, assignUnboundedChunkFirst, - appendOnly); + appendOnly, + recordsPerSecond); } @Override diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSourceFactory.java index 5ea430d94e7..c352a106242 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSourceFactory.java @@ -110,6 +110,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { boolean appendOnly = config.get(MySqlSourceOptions.SCAN_READ_CHANGELOG_AS_APPEND_ONLY_ENABLED); + double recordsPerSecond = config.get(MySqlSourceOptions.RECORDS_PER_SECOND); + if (enableParallelRead) { validatePrimaryKeyIfEnableParallel(physicalSchema, chunkKeyColumn); validateIntegerOption( @@ -156,7 +158,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { parseOnLineSchemaChanges, useLegacyJsonFormat, assignUnboundedChunkFirst, - appendOnly); + appendOnly, + recordsPerSecond); } @Override @@ -206,6 +209,7 @@ public Set> optionalOptions() { options.add(MySqlSourceOptions.USE_LEGACY_JSON_FORMAT); options.add(MySqlSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST); options.add(MySqlSourceOptions.SCAN_READ_CHANGELOG_AS_APPEND_ONLY_ENABLED); + options.add(MySqlSourceOptions.RECORDS_PER_SECOND); return options; } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSourceFactoryTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSourceFactoryTest.java index 01c2dff84da..b4557ef99d9 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSourceFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSourceFactoryTest.java @@ -52,6 +52,7 @@ import static org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions.CONNECT_TIMEOUT; import static org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions.HEARTBEAT_INTERVAL; import static org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions.PARSE_ONLINE_SCHEMA_CHANGES; +import static org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions.RECORDS_PER_SECOND; import static org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP; import static org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; import static org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED; @@ -129,7 +130,8 @@ void testCommonProperties() { PARSE_ONLINE_SCHEMA_CHANGES.defaultValue(), USE_LEGACY_JSON_FORMAT.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST.defaultValue(), - false); + false, + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -179,7 +181,8 @@ void testEnableParallelReadSource() { PARSE_ONLINE_SCHEMA_CHANGES.defaultValue(), USE_LEGACY_JSON_FORMAT.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST.defaultValue(), - false); + false, + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -225,7 +228,8 @@ void testEnableParallelReadSourceWithSingleServerId() { PARSE_ONLINE_SCHEMA_CHANGES.defaultValue(), USE_LEGACY_JSON_FORMAT.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST.defaultValue(), - false); + false, + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -269,7 +273,8 @@ void testEnableParallelReadSourceLatestOffset() { PARSE_ONLINE_SCHEMA_CHANGES.defaultValue(), USE_LEGACY_JSON_FORMAT.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST.defaultValue(), - false); + false, + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -330,7 +335,8 @@ void testOptionalProperties() { PARSE_ONLINE_SCHEMA_CHANGES.defaultValue(), true, SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST.defaultValue(), - false); + false, + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource) .isEqualTo(expectedSource) .isInstanceOf(MySqlTableSource.class); @@ -389,7 +395,8 @@ void testStartupFromSpecificOffset() { PARSE_ONLINE_SCHEMA_CHANGES.defaultValue(), USE_LEGACY_JSON_FORMAT.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST.defaultValue(), - false); + false, + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -431,7 +438,8 @@ void testStartupFromInitial() { PARSE_ONLINE_SCHEMA_CHANGES.defaultValue(), USE_LEGACY_JSON_FORMAT.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST.defaultValue(), - false); + false, + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -474,7 +482,8 @@ void testStartupFromEarliestOffset() { PARSE_ONLINE_SCHEMA_CHANGES.defaultValue(), USE_LEGACY_JSON_FORMAT.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST.defaultValue(), - false); + false, + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -518,7 +527,8 @@ void testStartupFromSpecificTimestamp() { PARSE_ONLINE_SCHEMA_CHANGES.defaultValue(), USE_LEGACY_JSON_FORMAT.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST.defaultValue(), - false); + false, + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -560,7 +570,8 @@ void testStartupFromLatestOffset() { PARSE_ONLINE_SCHEMA_CHANGES.defaultValue(), USE_LEGACY_JSON_FORMAT.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST.defaultValue(), - false); + false, + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -607,7 +618,8 @@ void testMetadataColumns() { PARSE_ONLINE_SCHEMA_CHANGES.defaultValue(), USE_LEGACY_JSON_FORMAT.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST.defaultValue(), - false); + false, + RECORDS_PER_SECOND.defaultValue()); expectedSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType(); expectedSource.metadataKeys = Arrays.asList("op_ts", "database_name"); @@ -810,7 +822,8 @@ void testEnablingExperimentalOptions() { true, true, true, - false); + false, + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceBuilder.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceBuilder.java index 335958923f6..712f935859d 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceBuilder.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceBuilder.java @@ -260,6 +260,11 @@ public OracleSourceBuilder assignUnboundedChunkFirst(boolean assignUnboundedC return this; } + public OracleSourceBuilder recordsPerSecond(double recordsPerSecond) { + this.configFactory.recordsPerSecond(recordsPerSecond); + return this; + } + /** * Build the {@link OracleIncrementalSource}. * diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/config/OracleSourceConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/config/OracleSourceConfig.java index 26152d52c09..820010f6757 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/config/OracleSourceConfig.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/config/OracleSourceConfig.java @@ -65,7 +65,8 @@ public OracleSourceConfig( String chunkKeyColumn, boolean skipSnapshotBackfill, boolean scanNewlyAddedTableEnabled, - boolean assignUnboundedChunkFirst) { + boolean assignUnboundedChunkFirst, + double recordsPerSecond) { super( startupOptions, databaseList, @@ -92,7 +93,8 @@ public OracleSourceConfig( chunkKeyColumn, skipSnapshotBackfill, scanNewlyAddedTableEnabled, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); this.url = url; } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/config/OracleSourceConfigFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/config/OracleSourceConfigFactory.java index a255b380626..b9fe7367956 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/config/OracleSourceConfigFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/source/config/OracleSourceConfigFactory.java @@ -135,6 +135,7 @@ public OracleSourceConfig create(int subtaskId) { chunkKeyColumn, skipSnapshotBackfill, scanNewlyAddedTableEnabled, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSource.java index 2cb3e899b83..dab35264a00 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSource.java @@ -81,6 +81,7 @@ public class OracleTableSource implements ScanTableSource, SupportsReadingMetada private final boolean skipSnapshotBackfill; private final boolean scanNewlyAddedTableEnabled; private final boolean assignUnboundedChunkFirst; + private final double recordsPerSecond; // -------------------------------------------------------------------------------------------- // Mutable attributes @@ -117,7 +118,8 @@ public OracleTableSource( boolean closeIdleReaders, boolean skipSnapshotBackfill, boolean scanNewlyAddedTableEnabled, - boolean assignUnboundedChunkFirst) { + boolean assignUnboundedChunkFirst, + double recordsPerSecond) { this.physicalSchema = physicalSchema; this.url = url; this.port = port; @@ -145,6 +147,7 @@ public OracleTableSource( this.skipSnapshotBackfill = skipSnapshotBackfill; this.scanNewlyAddedTableEnabled = scanNewlyAddedTableEnabled; this.assignUnboundedChunkFirst = assignUnboundedChunkFirst; + this.recordsPerSecond = recordsPerSecond; } @Override @@ -195,6 +198,7 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { .chunkKeyColumn(chunkKeyColumn) .scanNewlyAddedTableEnabled(scanNewlyAddedTableEnabled) .assignUnboundedChunkFirst(assignUnboundedChunkFirst) + .recordsPerSecond(recordsPerSecond) .build(); return SourceProvider.of(oracleChangeEventSource); @@ -262,7 +266,8 @@ public DynamicTableSource copy() { closeIdleReaders, skipSnapshotBackfill, scanNewlyAddedTableEnabled, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); source.metadataKeys = metadataKeys; source.producedDataType = producedDataType; return source; @@ -303,7 +308,8 @@ public boolean equals(Object o) { && Objects.equals(closeIdleReaders, that.closeIdleReaders) && Objects.equals(skipSnapshotBackfill, that.skipSnapshotBackfill) && Objects.equals(scanNewlyAddedTableEnabled, that.scanNewlyAddedTableEnabled) - && Objects.equals(assignUnboundedChunkFirst, that.assignUnboundedChunkFirst); + && Objects.equals(assignUnboundedChunkFirst, that.assignUnboundedChunkFirst) + && Objects.equals(recordsPerSecond, that.recordsPerSecond); } @Override @@ -335,7 +341,8 @@ public int hashCode() { closeIdleReaders, skipSnapshotBackfill, scanNewlyAddedTableEnabled, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } @Override diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSourceFactory.java index 9f345574939..0c916bc6372 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSourceFactory.java @@ -41,11 +41,12 @@ import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.DATABASE_NAME; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.HOSTNAME; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.PASSWORD; -import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.RECORDS_PER_SECOND; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.TABLE_NAME; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.USERNAME; import static org.apache.flink.cdc.connectors.base.options.SourceOptions.CHUNK_META_GROUP_SIZE; +import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED; import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP; import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED; @@ -117,6 +118,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { boolean scanNewlyAddedTableEnabled = config.get(SCAN_NEWLY_ADDED_TABLE_ENABLED); boolean assignUnboundedChunkFirst = config.get(SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED); + double recordsPerSecond = config.get(RECORDS_PER_SECOND); if (enableParallelRead) { validateIntegerOption(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE, splitSize, 1); @@ -155,7 +157,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { closeIdlerReaders, skipSnapshotBackfill, scanNewlyAddedTableEnabled, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } @Override @@ -195,6 +198,7 @@ public Set> optionalOptions() { options.add(SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP); options.add(SCAN_NEWLY_ADDED_TABLE_ENABLED); options.add(SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED); + options.add(RECORDS_PER_SECOND); return options; } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSourceFactoryTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSourceFactoryTest.java index 82616bd9229..a52bfefd568 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSourceFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSourceFactoryTest.java @@ -117,7 +117,8 @@ void testRequiredProperties() { JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP.defaultValue(), JdbcSourceOptions.SCAN_NEWLY_ADDED_TABLE_ENABLED.defaultValue(), JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED - .defaultValue()); + .defaultValue(), + JdbcSourceOptions.RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -156,7 +157,8 @@ void testCommonProperties() { SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP.defaultValue(), SourceOptions.SCAN_NEWLY_ADDED_TABLE_ENABLED.defaultValue(), JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED - .defaultValue()); + .defaultValue(), + JdbcSourceOptions.RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -200,7 +202,8 @@ void testOptionalProperties() { SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP.defaultValue(), true, JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED - .defaultValue()); + .defaultValue(), + JdbcSourceOptions.RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -272,7 +275,8 @@ void testScanIncrementalProperties() { true, true, true, - true); + true, + JdbcSourceOptions.RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -312,7 +316,8 @@ void testStartupFromInitial() { SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP.defaultValue(), SourceOptions.SCAN_NEWLY_ADDED_TABLE_ENABLED.defaultValue(), JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED - .defaultValue()); + .defaultValue(), + JdbcSourceOptions.RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -352,7 +357,8 @@ void testStartupFromLatestOffset() { SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP.defaultValue(), SourceOptions.SCAN_NEWLY_ADDED_TABLE_ENABLED.defaultValue(), JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED - .defaultValue()); + .defaultValue(), + JdbcSourceOptions.RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -396,7 +402,8 @@ void testMetadataColumns() { SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP.defaultValue(), SourceOptions.SCAN_NEWLY_ADDED_TABLE_ENABLED.defaultValue(), JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED - .defaultValue()); + .defaultValue(), + JdbcSourceOptions.RECORDS_PER_SECOND.defaultValue()); expectedSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType(); expectedSource.metadataKeys = Arrays.asList("op_ts", "database_name", "table_name", "schema_name"); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/PostgresSourceBuilder.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/PostgresSourceBuilder.java index 53bb066b59d..08d0165d082 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/PostgresSourceBuilder.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/PostgresSourceBuilder.java @@ -321,6 +321,11 @@ public PostgresSourceBuilder includeSchemaChanges(boolean includeSchemaChange return this; } + public PostgresSourceBuilder recordsPerSecond(double recordsPerSecond) { + this.configFactory.recordsPerSecond(recordsPerSecond); + return this; + } + /** * Build the {@link PostgresIncrementalSource}. * diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/config/PostgresSourceConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/config/PostgresSourceConfig.java index 4ad4b3e7e9d..b7ed6085fb4 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/config/PostgresSourceConfig.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/config/PostgresSourceConfig.java @@ -71,7 +71,8 @@ public PostgresSourceConfig( int lsnCommitCheckpointsDelay, boolean assignUnboundedChunkFirst, boolean includePartitionedTables, - boolean includeDatabaseInTableId) { + boolean includeDatabaseInTableId, + double recordsPerSecond) { super( startupOptions, databaseList, @@ -98,7 +99,8 @@ public PostgresSourceConfig( chunkKeyColumn, skipSnapshotBackfill, isScanNewlyAddedTableEnabled, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); this.subtaskId = subtaskId; this.lsnCommitCheckpointsDelay = lsnCommitCheckpointsDelay; this.includePartitionedTables = includePartitionedTables; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/config/PostgresSourceConfigFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/config/PostgresSourceConfigFactory.java index 847b1547461..ddbbcddd2e3 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/config/PostgresSourceConfigFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/config/PostgresSourceConfigFactory.java @@ -140,7 +140,8 @@ public PostgresSourceConfig create(int subtaskId) { lsnCommitCheckpointsDelay, assignUnboundedChunkFirst, includePartitionedTables, - includeDatabaseInTableId); + includeDatabaseInTableId, + recordsPerSecond); } /** diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableFactory.java index 876f04a83bf..401fd46b0b4 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableFactory.java @@ -38,6 +38,7 @@ import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.DATABASE_NAME; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.HOSTNAME; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.PASSWORD; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.RECORDS_PER_SECOND; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.SCHEMA_NAME; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.TABLE_NAME; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.USERNAME; @@ -122,6 +123,7 @@ public DynamicTableSource createDynamicTableSource(DynamicTableFactory.Context c boolean assignUnboundedChunkFirst = config.get(SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED); boolean appendOnly = config.get(SCAN_READ_CHANGELOG_AS_APPEND_ONLY_ENABLED); + double recordsPerSecond = config.get(RECORDS_PER_SECOND); if (enableParallelRead) { validateIntegerOption(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE, splitSize, 1); @@ -170,7 +172,8 @@ public DynamicTableSource createDynamicTableSource(DynamicTableFactory.Context c lsnCommitCheckpointsDelay, assignUnboundedChunkFirst, appendOnly, - includePartitionedTables); + includePartitionedTables, + recordsPerSecond); } @Override @@ -216,6 +219,7 @@ public Set> optionalOptions() { options.add(SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED); options.add(SCAN_READ_CHANGELOG_AS_APPEND_ONLY_ENABLED); options.add(SCAN_INCLUDE_PARTITIONED_TABLES_ENABLED); + options.add(RECORDS_PER_SECOND); return options; } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableSource.java index 7ebb43f9b4e..bfb4608e6f0 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableSource.java @@ -88,6 +88,7 @@ public class PostgreSQLTableSource implements ScanTableSource, SupportsReadingMe private final boolean assignUnboundedChunkFirst; private final boolean appendOnly; private final boolean includePartitionedTables; + private final double recordsPerSecond; // -------------------------------------------------------------------------------------------- // Mutable attributes @@ -130,7 +131,8 @@ public PostgreSQLTableSource( int lsnCommitCheckpointsDelay, boolean assignUnboundedChunkFirst, boolean appendOnly, - boolean includePartitionedTables) { + boolean includePartitionedTables, + double recordsPerSecond) { this.physicalSchema = physicalSchema; this.port = port; this.hostname = checkNotNull(hostname); @@ -165,6 +167,7 @@ public PostgreSQLTableSource( this.assignUnboundedChunkFirst = assignUnboundedChunkFirst; this.appendOnly = appendOnly; this.includePartitionedTables = includePartitionedTables; + this.recordsPerSecond = recordsPerSecond; } @Override @@ -305,7 +308,8 @@ public DynamicTableSource copy() { lsnCommitCheckpointsDelay, assignUnboundedChunkFirst, appendOnly, - includePartitionedTables); + includePartitionedTables, + recordsPerSecond); source.metadataKeys = metadataKeys; source.producedDataType = producedDataType; return source; @@ -351,7 +355,8 @@ public boolean equals(Object o) { && Objects.equals(scanNewlyAddedTableEnabled, that.scanNewlyAddedTableEnabled) && Objects.equals(assignUnboundedChunkFirst, that.assignUnboundedChunkFirst) && Objects.equals(appendOnly, that.appendOnly) - && Objects.equals(includePartitionedTables, that.includePartitionedTables); + && Objects.equals(includePartitionedTables, that.includePartitionedTables) + && Objects.equals(recordsPerSecond, that.recordsPerSecond); } @Override @@ -388,7 +393,8 @@ public int hashCode() { scanNewlyAddedTableEnabled, assignUnboundedChunkFirst, appendOnly, - includePartitionedTables); + includePartitionedTables, + recordsPerSecond); } @Override diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/MockPostgreSQLTableSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/MockPostgreSQLTableSource.java index ecd049b6f19..d47efd748ec 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/MockPostgreSQLTableSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/MockPostgreSQLTableSource.java @@ -68,7 +68,8 @@ public MockPostgreSQLTableSource(PostgreSQLTableSource postgreSQLTableSource) { (int) get(postgreSQLTableSource, "lsnCommitCheckpointsDelay"), (boolean) get(postgreSQLTableSource, "assignUnboundedChunkFirst"), (boolean) get(postgreSQLTableSource, "appendOnly"), - (boolean) get(postgreSQLTableSource, "includePartitionedTables")); + (boolean) get(postgreSQLTableSource, "includePartitionedTables"), + (double) get(postgreSQLTableSource, "recordsPerSecond")); } @Override diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableFactoryTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableFactoryTest.java index d72f97eb87c..2ba51cc65d8 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableFactoryTest.java @@ -64,6 +64,7 @@ import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; import static org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceOptions.HEARTBEAT_INTERVAL; +import static org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceOptions.RECORDS_PER_SECOND; import static org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceOptions.SCAN_INCLUDE_PARTITIONED_TABLES_ENABLED; import static org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceOptions.SCAN_LSN_COMMIT_CHECKPOINTS_DELAY; import static org.apache.flink.cdc.connectors.utils.AssertUtils.assertProducedTypeOfSourceFunction; @@ -157,7 +158,8 @@ void testCommonProperties() { SCAN_LSN_COMMIT_CHECKPOINTS_DELAY.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED.defaultValue(), SCAN_READ_CHANGELOG_AS_APPEND_ONLY_ENABLED.defaultValue(), - SCAN_INCLUDE_PARTITIONED_TABLES_ENABLED.defaultValue()); + SCAN_INCLUDE_PARTITIONED_TABLES_ENABLED.defaultValue(), + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -207,7 +209,8 @@ void testOptionalProperties() { SCAN_LSN_COMMIT_CHECKPOINTS_DELAY.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED.defaultValue(), true, - SCAN_INCLUDE_PARTITIONED_TABLES_ENABLED.defaultValue()); + SCAN_INCLUDE_PARTITIONED_TABLES_ENABLED.defaultValue(), + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -254,7 +257,8 @@ void testMetadataColumns() { SCAN_LSN_COMMIT_CHECKPOINTS_DELAY.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED.defaultValue(), SCAN_READ_CHANGELOG_AS_APPEND_ONLY_ENABLED.defaultValue(), - SCAN_INCLUDE_PARTITIONED_TABLES_ENABLED.defaultValue()); + SCAN_INCLUDE_PARTITIONED_TABLES_ENABLED.defaultValue(), + RECORDS_PER_SECOND.defaultValue()); expectedSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType(); expectedSource.metadataKeys = Arrays.asList("row_kind", "op_ts", "database_name", "schema_name", "table_name"); @@ -311,7 +315,8 @@ void testEnableParallelReadSource() { SCAN_LSN_COMMIT_CHECKPOINTS_DELAY.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED.defaultValue(), SCAN_READ_CHANGELOG_AS_APPEND_ONLY_ENABLED.defaultValue(), - SCAN_INCLUDE_PARTITIONED_TABLES_ENABLED.defaultValue()); + SCAN_INCLUDE_PARTITIONED_TABLES_ENABLED.defaultValue(), + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -358,7 +363,8 @@ void testStartupFromLatestOffset() { SCAN_LSN_COMMIT_CHECKPOINTS_DELAY.defaultValue(), SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED.defaultValue(), SCAN_READ_CHANGELOG_AS_APPEND_ONLY_ENABLED.defaultValue(), - SCAN_INCLUDE_PARTITIONED_TABLES_ENABLED.defaultValue()); + SCAN_INCLUDE_PARTITIONED_TABLES_ENABLED.defaultValue(), + RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceBuilder.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceBuilder.java index 64141f05b51..25880316e82 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceBuilder.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceBuilder.java @@ -238,6 +238,11 @@ public SqlServerSourceBuilder assignUnboundedChunkFirst(boolean assignUnbound return this; } + public SqlServerSourceBuilder recordsPerSecond(double recordsPerSecond) { + this.configFactory.recordsPerSecond(recordsPerSecond); + return this; + } + /** * Build the {@link SqlServerIncrementalSource}. * diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java index 356b6a0135a..96f7190bcb1 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java @@ -57,7 +57,8 @@ public SqlServerSourceConfig( int connectionPoolSize, String chunkKeyColumn, boolean skipSnapshotBackfill, - boolean assignUnboundedChunkFirst) { + boolean assignUnboundedChunkFirst, + double recordsPerSecond) { super( startupOptions, databaseList, @@ -84,7 +85,8 @@ public SqlServerSourceConfig( chunkKeyColumn, skipSnapshotBackfill, false, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } @Override diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactory.java index 661094a8942..f4ab3092242 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/config/SqlServerSourceConfigFactory.java @@ -104,6 +104,7 @@ public SqlServerSourceConfig create(int subtask) { connectionPoolSize, chunkKeyColumn, skipSnapshotBackfill, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactory.java index a841366b20c..3bf2ce81612 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactory.java @@ -37,6 +37,7 @@ import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECTION_POOL_SIZE; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_MAX_RETRIES; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_TIMEOUT; +import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.RECORDS_PER_SECOND; import static org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN; import static org.apache.flink.cdc.connectors.base.options.SourceOptions.CHUNK_META_GROUP_SIZE; import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED; @@ -142,6 +143,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { boolean skipSnapshotBackfill = config.get(SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP); boolean assignUnboundedChunkFirst = config.get(SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED); + double recordsPerSecond = config.get(RECORDS_PER_SECOND); if (enableParallelRead) { validateIntegerOption(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE, splitSize, 1); @@ -178,7 +180,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { chunkKeyColumn, closeIdleReaders, skipSnapshotBackfill, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } @Override @@ -216,6 +219,7 @@ public Set> optionalOptions() { options.add(SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED); options.add(SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP); options.add(SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED); + options.add(RECORDS_PER_SECOND); return options; } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableSource.java index 377b0d57355..0b632ac32e6 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableSource.java @@ -80,6 +80,7 @@ public class SqlServerTableSource implements ScanTableSource, SupportsReadingMet private final boolean closeIdleReaders; private final boolean skipSnapshotBackfill; private final boolean assignUnboundedChunkFirst; + private final double recordsPerSecond; // -------------------------------------------------------------------------------------------- // Mutable attributes @@ -114,7 +115,8 @@ public SqlServerTableSource( @Nullable String chunkKeyColumn, boolean closeIdleReaders, boolean skipSnapshotBackfill, - boolean assignUnboundedChunkFirst) { + boolean assignUnboundedChunkFirst, + double recordsPerSecond) { this.physicalSchema = physicalSchema; this.port = port; this.hostname = checkNotNull(hostname); @@ -140,6 +142,7 @@ public SqlServerTableSource( this.closeIdleReaders = closeIdleReaders; this.skipSnapshotBackfill = skipSnapshotBackfill; this.assignUnboundedChunkFirst = assignUnboundedChunkFirst; + this.recordsPerSecond = recordsPerSecond; } @Override @@ -189,6 +192,7 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { .closeIdleReaders(closeIdleReaders) .skipSnapshotBackfill(skipSnapshotBackfill) .assignUnboundedChunkFirst(assignUnboundedChunkFirst) + .recordsPerSecond(recordsPerSecond) .build(); return SourceProvider.of(sqlServerChangeEventSource); } else { @@ -250,7 +254,8 @@ public DynamicTableSource copy() { chunkKeyColumn, closeIdleReaders, skipSnapshotBackfill, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); source.metadataKeys = metadataKeys; source.producedDataType = producedDataType; return source; @@ -289,7 +294,8 @@ public boolean equals(Object o) { && Objects.equals(chunkKeyColumn, that.chunkKeyColumn) && Objects.equals(closeIdleReaders, that.closeIdleReaders) && Objects.equals(skipSnapshotBackfill, that.skipSnapshotBackfill) - && Objects.equals(assignUnboundedChunkFirst, that.assignUnboundedChunkFirst); + && Objects.equals(assignUnboundedChunkFirst, that.assignUnboundedChunkFirst) + && Objects.equals(recordsPerSecond, that.recordsPerSecond); } @Override @@ -319,7 +325,8 @@ public int hashCode() { chunkKeyColumn, closeIdleReaders, skipSnapshotBackfill, - assignUnboundedChunkFirst); + assignUnboundedChunkFirst, + recordsPerSecond); } @Override diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactoryTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactoryTest.java index 492b2795bb9..98782374845 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactoryTest.java @@ -113,7 +113,8 @@ void testCommonProperties() { false, JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP.defaultValue(), JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED - .defaultValue()); + .defaultValue(), + JdbcSourceOptions.RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -158,7 +159,8 @@ void testEnableParallelReadSource() { true, true, JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED - .defaultValue()); + .defaultValue(), + JdbcSourceOptions.RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -201,7 +203,8 @@ void testOptionalProperties() { true, JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP.defaultValue(), JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED - .defaultValue()); + .defaultValue(), + JdbcSourceOptions.RECORDS_PER_SECOND.defaultValue()); Assertions.assertThat(actualSource).isEqualTo(expectedSource); } @@ -243,7 +246,8 @@ void testMetadataColumns() { false, JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_BACKFILL_SKIP.defaultValue(), JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_UNBOUNDED_CHUNK_FIRST_ENABLED - .defaultValue()); + .defaultValue(), + JdbcSourceOptions.RECORDS_PER_SECOND.defaultValue()); expectedSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType(); expectedSource.metadataKeys = Arrays.asList("op_ts", "database_name", "schema_name", "table_name"); diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlE2eITCase.java index 7b551db8333..93ae1c747d1 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlE2eITCase.java @@ -24,6 +24,8 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -535,4 +537,102 @@ void testDanglingDropTableEventInBinlog() throws Exception { "CreateTableEvent{tableId=%s.products, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`description` VARCHAR(512),`weight` FLOAT,`enum_c` STRING 'red',`json_c` STRING,`point_c` STRING}, primaryKeys=id, options=()}", "DataChangeEvent{tableId=%s.products, before=[106, hammer, 16oz carpenter's hammer, 1.0, null, null, null], after=[106, hammer, 18oz carpenter hammer, 1.0, null, null, null], op=UPDATE, meta=()}"); } + + @ParameterizedTest + @ValueSource(doubles = {-1d, 2d}) + void testRateLimit(double rateLimit) throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.\\.*\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + " records.per.second: %f\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "\n" + + "pipeline:\n" + + " parallelism: %d", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + mysqlInventoryDatabase.getDatabaseName(), + rateLimit, + parallelism); + + submitPipelineJob(pipelineJob); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.customers, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`address` VARCHAR(1024),`phone_number` VARCHAR(512)}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=%s.customers, before=[], after=[104, user_4, Shanghai, 123567891234], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.customers, before=[], after=[103, user_3, Shanghai, 123567891234], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.customers, before=[], after=[102, user_2, Shanghai, 123567891234], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.customers, before=[], after=[101, user_1, Shanghai, 123567891234], op=INSERT, meta=()}", + "CreateTableEvent{tableId=%s.products, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`description` VARCHAR(512),`weight` FLOAT,`enum_c` STRING 'red',`json_c` STRING,`point_c` STRING}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[109, spare tire, 24 inch spare tire, 22.2, null, null, null], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[107, rocks, box of assorted rocks, 5.3, null, null, null], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[108, jacket, water resistent black wind breaker, 0.1, null, null, null], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[105, hammer, 14oz carpenter's hammer, 0.875, red, {\"k1\": \"v1\", \"k2\": \"v2\"}, {\"coordinates\":[5,5],\"type\":\"Point\",\"srid\":0}], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[106, hammer, 16oz carpenter's hammer, 1.0, null, null, null], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, red, {\"key3\": \"value3\"}, {\"coordinates\":[3,3],\"type\":\"Point\",\"srid\":0}], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[104, hammer, 12oz carpenter's hammer, 0.75, white, {\"key4\": \"value4\"}, {\"coordinates\":[4,4],\"type\":\"Point\",\"srid\":0}], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[101, scooter, Small 2-wheel scooter, 3.14, red, {\"key1\": \"value1\"}, {\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0}], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[102, car battery, 12V car battery, 8.1, white, {\"key2\": \"value2\"}, {\"coordinates\":[2,2],\"type\":\"Point\",\"srid\":0}], op=INSERT, meta=()}"); + + LOG.info("Begin incremental reading stage."); + // generate binlogs + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + mysqlInventoryDatabase.getDatabaseName()); + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stat = conn.createStatement()) { + stat.execute("UPDATE products SET description='18oz carpenter hammer' WHERE id=106;"); + stat.execute("UPDATE products SET weight='5.1' WHERE id=107;"); + + // Perform DDL changes after the binlog is generated + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.products, before=[106, hammer, 16oz carpenter's hammer, 1.0, null, null, null], after=[106, hammer, 18oz carpenter hammer, 1.0, null, null, null], op=UPDATE, meta=()}", + mysqlInventoryDatabase.getDatabaseName())); + + // modify table schema + stat.execute("ALTER TABLE products ADD COLUMN new_col INT;"); + stat.execute( + "INSERT INTO products VALUES (default,'jacket','water resistent white wind breaker',0.2, null, null, null, 1);"); // 110 + stat.execute( + "INSERT INTO products VALUES (default,'scooter','Big 2-wheel scooter ',5.18, null, null, null, 1);"); // 111 + stat.execute( + "UPDATE products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;"); + stat.execute("UPDATE products SET weight='5.17' WHERE id=111;"); + stat.execute("DELETE FROM products WHERE id=111;"); + } catch (SQLException e) { + LOG.error("Update table for CDC failed.", e); + throw e; + } + + validateResult( + dbNameFormatter, + "DataChangeEvent{tableId=%s.products, before=[106, hammer, 16oz carpenter's hammer, 1.0, null, null, null], after=[106, hammer, 18oz carpenter hammer, 1.0, null, null, null], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[107, rocks, box of assorted rocks, 5.3, null, null, null], after=[107, rocks, box of assorted rocks, 5.1, null, null, null], op=UPDATE, meta=()}", + "AddColumnEvent{tableId=%s.products, addedColumns=[ColumnWithPosition{column=`new_col` INT, position=LAST, existedColumnName=null}]}", + "DataChangeEvent{tableId=%s.products, before=[], after=[110, jacket, water resistent white wind breaker, 0.2, null, null, null, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[111, scooter, Big 2-wheel scooter , 5.18, null, null, null, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[110, jacket, water resistent white wind breaker, 0.2, null, null, null, 1], after=[110, jacket, new water resistent white wind breaker, 0.5, null, null, null, 1], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[111, scooter, Big 2-wheel scooter , 5.18, null, null, null, 1], after=[111, scooter, Big 2-wheel scooter , 5.17, null, null, null, 1], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[111, scooter, Big 2-wheel scooter , 5.17, null, null, null, 1], after=[], op=DELETE, meta=()}"); + } } diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OracleE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OracleE2eITCase.java index e6b5387a224..48fbc0e2c60 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OracleE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OracleE2eITCase.java @@ -27,6 +27,8 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.DockerClientFactory; @@ -274,6 +276,166 @@ void testSyncWholeDatabase() throws Exception { } } + @ParameterizedTest + @ValueSource(doubles = {-1d, 2d}) + void testRateLimit(double rateLimit) throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: oracle\n" + + " hostname: %s\n" + + " port: %d\n" + + " username: %s\n" + + " password: %s\n" + + " tables: DEBEZIUM.PRODUCTS,DEBEZIUM.CUSTOMERS,DEBEZIUM.CUSTOMERS_1\n" + + " database: %s\n" + + " scan.startup.mode: initial\n" + + " records.per.second: %f\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "\n" + + "pipeline:\n" + + " parallelism: %d", + oracle.getNetworkAliases().get(0), + oracle.getExposedPorts().get(0), + TOP_USER, + TOP_SECRET, + ORACLE_DATABASE, + rateLimit, + 1); + submitPipelineJob(pipelineJob, postgresCdcJar, oracleOjdbcJar); + LOG.info("Containers are started."); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + try (Connection conn = getOracleJdbcConnection(); + Statement stat = conn.createStatement()) { + + waitUntilSpecificEvent( + "CreateTableEvent{tableId=DEBEZIUM.PRODUCTS, schema=columns={`ID` BIGINT NOT NULL,`NAME` VARCHAR(255) NOT NULL,`DESCRIPTION` VARCHAR(512),`WEIGHT` FLOAT}, primaryKeys=ID, options=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.PRODUCTS, before=[], after=[109, spare tire, 24 inch spare tire, 22.2], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.PRODUCTS, before=[], after=[104, hammer, 12oz carpenters hammer, 0.75], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.PRODUCTS, before=[], after=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 1.8], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.PRODUCTS, before=[], after=[102, car battery, 12V car battery, 8.1], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.PRODUCTS, before=[], after=[101, scooter, Small 2-wheel scooter, 3.14], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.PRODUCTS, before=[], after=[108, jacket, water resistent black wind breaker, 0.1], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.PRODUCTS, before=[], after=[107, rocks, box of assorted rocks, 5.3], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.PRODUCTS, before=[], after=[106, hammer, 16oz carpenters hammer, 1.0], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.PRODUCTS, before=[], after=[105, hammer, 14oz carpenters hammer, 0.875], op=INSERT, meta=()}"); + + waitUntilSpecificEvent( + "CreateTableEvent{tableId=DEBEZIUM.CUSTOMERS, schema=columns={`ID` BIGINT NOT NULL,`NAME` VARCHAR(255) NOT NULL,`ADDRESS` VARCHAR(1024),`PHONE_NUMBER` VARCHAR(512)}, primaryKeys=ID, options=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691841, user_1, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691841, user_4, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691841, user_3, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691841, user_2, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691842, user_13, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691842, user_14, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691842, user_11, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691842, user_12, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691842, user_21, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691842, user_10, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691841, user_6, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691841, user_5, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691841, user_9, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691842, user_19, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691842, user_20, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691841, user_8, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691842, user_17, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691842, user_18, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691842, user_15, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691841, user_7, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS, before=[], after=[171798691842, user_16, Shanghai, 123567891234], op=INSERT, meta=()}"); + + stat.execute( + "UPDATE DEBEZIUM.PRODUCTS SET DESCRIPTION='18oz carpenter hammer' WHERE ID=106 "); + stat.execute("UPDATE DEBEZIUM.PRODUCTS SET WEIGHT='5.1' WHERE ID=107 "); + stat.execute("commit "); + // Perform DML changes after the redo log is generated + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.PRODUCTS, before=[106, hammer, 16oz carpenters hammer, 1.0], after=[106, hammer, 18oz carpenter hammer, 1.0], op=UPDATE, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.PRODUCTS, before=[107, rocks, box of assorted rocks, 5.3], after=[107, rocks, box of assorted rocks, 5.1], op=UPDATE, meta=()}"); + waitUntilSpecificEvent( + "CreateTableEvent{tableId=DEBEZIUM.CUSTOMERS_1, schema=columns={`ID` BIGINT NOT NULL,`NAME` VARCHAR(255) NOT NULL,`ADDRESS` VARCHAR(1024),`PHONE_NUMBER` VARCHAR(512)}, primaryKeys=ID, options=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691842, user_10, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691842, user_21, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691841, user_5, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691842, user_14, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691841, user_6, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691842, user_13, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691842, user_12, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691842, user_11, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691841, user_4, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691841, user_2, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691841, user_3, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691841, user_1, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691842, user_18, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691841, user_9, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691842, user_17, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691842, user_16, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691842, user_15, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691841, user_7, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691842, user_20, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691841, user_8, Shanghai, 123567891234], op=INSERT, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=DEBEZIUM.CUSTOMERS_1, before=[], after=[171798691842, user_19, Shanghai, 123567891234], op=INSERT, meta=()}"); + } catch (Exception e) { + LOG.error("Update table for CDC failed.", e); + throw new RuntimeException(e); + } + } + private void initializeOracleTable(String sqlFile) { final String ddlFile = String.format("ddl/%s.sql", sqlFile); final URL ddlTestFile = OracleSourceTestBase.class.getClassLoader().getResource(ddlFile); diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/PostgresE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/PostgresE2eITCase.java index bcc70aff22a..51b8bc26c4b 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/PostgresE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/PostgresE2eITCase.java @@ -24,6 +24,8 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.PostgreSQLContainer; @@ -217,4 +219,124 @@ void testSyncWholeDatabase() throws Exception { waitUntilSpecificEvent( "DataChangeEvent{tableId=inventory.products, before=[], after=[112, gizmo, A fancy gizmo, gadgets], op=INSERT, meta=()}"); } + + @ParameterizedTest + @ValueSource(doubles = {-1d, 2d}) + void testRateLimit(double rateLimit) throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: postgres\n" + + " hostname: %s\n" + + " port: %d\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.inventory.products,%s.inventory.customers\n" + + " slot.name: %s\n" + + " scan.startup.mode: initial\n" + + " server-time-zone: UTC\n" + + " connect.timeout: 120s\n" + + " schema-change.enabled: true\n" + + " records.per.second: %f\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "\n" + + "pipeline:\n" + + " parallelism: %d\n" + + " schema.change.behavior: evolve", + INTER_CONTAINER_POSTGRES_ALIAS, + 5432, + POSTGRES_TEST_USER, + POSTGRES_TEST_PASSWORD, + postgresInventoryDatabase.getDatabaseName(), + postgresInventoryDatabase.getDatabaseName(), + slotName, + rateLimit, + parallelism); + Path postgresCdcJar = TestUtils.getResource("postgres-cdc-pipeline-connector.jar"); + submitPipelineJob(pipelineJob, postgresCdcJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.customers, schema=columns={`id` INT NOT NULL 'nextval('inventory.customers_id_seq'::regclass)',`first_name` VARCHAR(255) NOT NULL,`last_name` VARCHAR(255) NOT NULL,`email` VARCHAR(255) NOT NULL}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=%s.customers, before=[], after=[104, Anne, Kretchmar, annek@noanswer.org], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.customers, before=[], after=[103, Edward, Walker, ed@walker.com], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.customers, before=[], after=[102, George, Bailey, gbailey@foobar.com], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.customers, before=[], after=[101, Sally, Thomas, sally.thomas@acme.com], op=INSERT, meta=()}", + "CreateTableEvent{tableId=%s.products, schema=columns={`id` INT NOT NULL 'nextval('inventory.products_id_seq'::regclass)',`name` VARCHAR(255) NOT NULL,`description` VARCHAR(512),`weight` FLOAT}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[109, spare tire, 24 inch spare tire, 22.2], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[107, rocks, box of assorted rocks, 5.3], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[108, jacket, water resistent black wind breaker, 0.1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[105, hammer, 14oz carpenter's hammer, 0.875], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[106, hammer, 16oz carpenter's hammer, 1.0], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[104, hammer, 12oz carpenter's hammer, 0.75], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[101, scooter, Small 2-wheel scooter, 3.14], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[102, car battery, 12V car battery, 8.1], op=INSERT, meta=()}"); + + LOG.info("Begin incremental reading stage."); + + try (Connection conn = + getJdbcConnection( + POSTGRES_CONTAINER, postgresInventoryDatabase.getDatabaseName()); + Statement stat = conn.createStatement()) { + stat.execute( + "UPDATE inventory.products SET description='18oz carpenter hammer' WHERE id=106;"); + stat.execute("UPDATE inventory.products SET weight='5.1' WHERE id=107;"); + + // Perform DML changes after the wal log is generated + waitUntilSpecificEvent( + "DataChangeEvent{tableId=inventory.products, before=[106, hammer, 16oz carpenter's hammer, 1.0], after=[106, hammer, 18oz carpenter hammer, 1.0], op=UPDATE, meta=()}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=inventory.products, before=[107, rocks, box of assorted rocks, 5.3], after=[107, rocks, box of assorted rocks, 5.1], op=UPDATE, meta=()}"); + } catch (Exception e) { + LOG.error("Update table for CDC failed.", e); + throw new RuntimeException(e); + } + + LOG.info("Begin schema change stage."); + + try (Connection conn = + getJdbcConnection( + POSTGRES_CONTAINER, postgresInventoryDatabase.getDatabaseName()); + Statement stat = conn.createStatement()) { + // Test ADD COLUMN + stat.execute("ALTER TABLE inventory.products ADD COLUMN category VARCHAR(255);"); + stat.execute( + "INSERT INTO inventory.products VALUES (default, 'widget', 'A small widget', 1.5, 'tools');"); + + // Test DROP COLUMN + stat.execute("ALTER TABLE inventory.products DROP COLUMN weight;"); + stat.execute( + "INSERT INTO inventory.products VALUES (default, 'gadget', 'A useful gadget', 'electronics');"); + + // Test RENAME COLUMN + stat.execute( + "ALTER TABLE inventory.products RENAME COLUMN category TO product_category;"); + stat.execute( + "INSERT INTO inventory.products VALUES (default, 'gizmo', 'A fancy gizmo', 'gadgets');"); + } catch (Exception e) { + LOG.error("Schema change test failed.", e); + throw new RuntimeException(e); + } + + // Validate schema change events and corresponding data + waitUntilSpecificEvent( + "AddColumnEvent{tableId=inventory.products, addedColumns=[ColumnWithPosition{column=`category` VARCHAR(255), position=LAST, existedColumnName=null}]}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=inventory.products, before=[], after=[110, widget, A small widget, 1.5, tools], op=INSERT, meta=()}"); + + waitUntilSpecificEvent( + "DropColumnEvent{tableId=inventory.products, droppedColumnNames=[weight]}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=inventory.products, before=[], after=[111, gadget, A useful gadget, electronics], op=INSERT, meta=()}"); + + waitUntilSpecificEvent( + "RenameColumnEvent{tableId=inventory.products, nameMapping={category=product_category}}"); + waitUntilSpecificEvent( + "DataChangeEvent{tableId=inventory.products, before=[], after=[112, gizmo, A fancy gizmo, gadgets], op=INSERT, meta=()}"); + } } diff --git a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/MySqlE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/MySqlE2eITCase.java index 3ef0110ba5c..26ea9ebc56f 100644 --- a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/MySqlE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/MySqlE2eITCase.java @@ -96,7 +96,6 @@ void testMySqlCDC() throws Exception { ");", "INSERT INTO products_sink", "SELECT * FROM products_source;"); - submitSQLJob(sqlLines, mysqlCdcJar, jdbcJar, mySqlConnectorJar); waitUntilJobRunning(Duration.ofSeconds(30)); diff --git a/flink-cdc-flink1-compat/src/main/java/org/apache/flink/connector/base/source/reader/SingleThreadMultiplexSourceReaderBaseAdapter.java b/flink-cdc-flink1-compat/src/main/java/org/apache/flink/connector/base/source/reader/SingleThreadMultiplexSourceReaderBaseAdapter.java new file mode 100644 index 00000000000..5319fa85367 --- /dev/null +++ b/flink-cdc-flink1-compat/src/main/java/org/apache/flink/connector/base/source/reader/SingleThreadMultiplexSourceReaderBaseAdapter.java @@ -0,0 +1,57 @@ +/* + * 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.flink.connector.base.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +/** + * Compatibility adapter for Flink 1.20. This class is part of the multi-version compatibility layer + * that allows Flink CDC to work across different Flink versions. + */ +@Internal +public abstract class SingleThreadMultiplexSourceReaderBaseAdapter< + E, T, SplitT extends SourceSplit, SplitStateT> + extends SingleThreadMultiplexSourceReaderBase { + + private static final Logger LOG = + LoggerFactory.getLogger(SingleThreadMultiplexSourceReaderBaseAdapter.class); + + public SingleThreadMultiplexSourceReaderBaseAdapter( + SingleThreadFetcherManager splitFetcherManager, + RecordEmitter recordEmitter, + @Nullable RecordEvaluator eofRecordEvaluator, + Configuration config, + SourceReaderContext context, + @Nullable RateLimiterStrategy rateLimiterStrategy) { + super(splitFetcherManager, recordEmitter, eofRecordEvaluator, config, context); + if (null != rateLimiterStrategy) { + LOG.warn( + "Because the runtime environment is Flink 1.x, the connector options `records.per.second` is ignored."); + } + } +} diff --git a/flink-cdc-flink2-compat/src/main/java/org/apache/flink/connector/base/source/reader/SingleThreadMultiplexSourceReaderBaseAdapter.java b/flink-cdc-flink2-compat/src/main/java/org/apache/flink/connector/base/source/reader/SingleThreadMultiplexSourceReaderBaseAdapter.java new file mode 100644 index 00000000000..e29891acbb8 --- /dev/null +++ b/flink-cdc-flink2-compat/src/main/java/org/apache/flink/connector/base/source/reader/SingleThreadMultiplexSourceReaderBaseAdapter.java @@ -0,0 +1,53 @@ +/* + * 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.flink.connector.base.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager; + +import javax.annotation.Nullable; + +/** + * Compatibility adapter for Flink 2.2. This class is part of the multi-version compatibility layer + * that allows Flink CDC to work across different Flink versions. + */ +@Internal +public abstract class SingleThreadMultiplexSourceReaderBaseAdapter< + E, T, SplitT extends SourceSplit, SplitStateT> + extends SingleThreadMultiplexSourceReaderBase { + + public SingleThreadMultiplexSourceReaderBaseAdapter( + SingleThreadFetcherManager splitFetcherManager, + RecordEmitter recordEmitter, + @Nullable RecordEvaluator eofRecordEvaluator, + Configuration config, + SourceReaderContext context, + @Nullable RateLimiterStrategy rateLimiterStrategy) { + super( + splitFetcherManager, + recordEmitter, + eofRecordEvaluator, + config, + context, + rateLimiterStrategy); + } +}