Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
68 commits
Select commit Hold shift + click to select a range
2785bf2
[FLINK-38911][mysql] Support binlog-only newly added table capture fo…
Jan 28, 2026
aa9dfb0
datastream api support scan.binlog.newly-added-table ability
Jan 29, 2026
ea64d7e
remove the some desc
Jan 30, 2026
b548110
Supplement the missing binlog_sql test's file
Feb 10, 2026
723f1a3
fix test class
Feb 26, 2026
bf93c21
fix 2
Feb 26, 2026
d2c10e4
fix mvn spotless:apply
Mar 9, 2026
ccb7994
[FLINK-38887] Transform supports handling nested types (ARRAY, MAP, R…
yuxiqian Jan 30, 2026
44801d8
[FLINK-38836][docs] update section headings and improve pipeline conf…
Hisoka-X Jan 30, 2026
5776a3d
[FLINK-38996][transform] Enhance error messages for projection and fi…
Hisoka-X Jan 30, 2026
abdab7c
[FLINK-38994][cli] Remove log4j1 dependency (#4242)
zml1206 Jan 30, 2026
c67e9f6
[FLINK-38985][docs] Add documentation for VARIANT type and PARSE_JSON…
suhwan-cheon Jan 31, 2026
166d70a
[FLINK-38965][postgres] Fix LIKE wildcard matching issue for similar …
Hisoka-X Jan 31, 2026
4bdd913
[FLINK-38520][postgres] Postgres YAML CDC support array with null ele…
Hisoka-X Feb 2, 2026
8e5620f
[FLINK-38906] Pass transform parser in context and polish built-in fu…
yuxiqian Feb 2, 2026
1fd7549
[FLINK-38846][mysql] Optimize LinkedList processing in handleChange m…
Hisoka-X Feb 3, 2026
7830b39
[chore][improve][fluss] Improve flaky test case FlussSinkITCase (#4256)
Hisoka-X Feb 3, 2026
ce829fc
[hotfix][kafka] Fix Debezium schema generation for complex types (ARR…
linguoxuan Feb 3, 2026
5b3c060
[FLINK-38888][transform] YAML Pipeline supports item subscription of …
Hisoka-X Feb 4, 2026
c631abd
[ci] Fix master branch documentation build failure (#4260)
yuxiqian Feb 5, 2026
4692f05
[FLINK-39010][transform] YAML Pipeline supports item subscription of …
Hisoka-X Feb 6, 2026
fef3efb
[FLINK-38514][postgres] Add support for UUID array type in PostgreSQL…
Hisoka-X Feb 6, 2026
4b89523
[FLINK-39038][infra] Limit GitHub actions max parallelism and workflo…
yuxiqian Feb 9, 2026
8de2a5c
[FLINK-39047][docs] Translate Pipeline Transform documentations to Ch…
Hisoka-X Feb 9, 2026
5e1b3f4
[FLINK-39048][iceberg] Bump iceberg-flink-runtime from 1.19 to 1.20 (…
suhwan-cheon Feb 9, 2026
99d1ff5
[FLINK-38512][postgres] Update database name validation to allow hyph…
Hisoka-X Feb 9, 2026
bde51ad
[FLINK-36796][pipeline-connector][oracle]add oracle pipeline connecto…
linjianchang Feb 10, 2026
d8111aa
[FLINK-39056][pipeline-connector][iceberg] Fix Duplicate Data Issue i…
fcfangcc Feb 11, 2026
af0f857
[FLINK-39063][connect/fluss] Fix some spelling mistakes in fluss pipe…
ThorneANN Feb 11, 2026
696e056
[minor][hotfix] Update configOption name from `sink.job.id.prefix` to…
lvyanquan Feb 12, 2026
368ee6c
[FLINK-38601][connect/mongodb] Catch throwable errors when consuming …
linguoxuan Feb 13, 2026
b572d6c
[FLINK-39066][pipeline-connector][Fluss] Fix the metric histogram alw…
ThorneANN Feb 24, 2026
7e943c6
[test] Improve pipeline connector test coverage i.e. doris/postgres/f…
leonardBang Feb 24, 2026
894134c
[FLINK-39144][paimon] Fix ioManager close.(#4283)
sd4324530 Feb 24, 2026
64ff69d
[FLINK-39143][pipeline-connecotr][Fluss]Optimize the connection of fl…
ThorneANN Feb 25, 2026
984ce8c
[FLINK-38455][elasticsearch][fix] Fix Elasticsearch Missing required …
Hisoka-X Feb 25, 2026
7d311ce
[FLINK-39001][doc][Flink-source]supple NewlyAddTable's doc with mongo…
ThorneANN Feb 25, 2026
368565d
[pipeline] Use GHA workflow triggers to run labeler workflow (#4274)
yuxiqian Feb 26, 2026
6adda1f
[FLINK-39071] Select fields rather * to read snapshot data in case sc…
loserwang1024 Feb 27, 2026
e0e1fe0
[Minor][FLINK-39168][cdc-connector-mysql] Cache split.splitId() resul…
ThorneANN Feb 27, 2026
2385d7e
[ci] Fix docker environment related test failure (#4275)
yuxiqian Feb 27, 2026
486a794
[FLINK-37485][starrocks] Add support for TIME type (#4253)
Hisoka-X Mar 3, 2026
711dfb9
[chore][test] Fix flaky postgres pipeline test case (#4293)
Hisoka-X Mar 4, 2026
b3c7c30
[FLINK-38833][paimon] Shuffle record to different subtasks by table, …
sd4324530 Mar 4, 2026
a333735
[FLINK-38334][mysql] Fix MySQL CDC source stuck in INITIAL_ASSIGNING …
morozov Mar 5, 2026
c750824
[FLINK-37586][udf] Add support for options in user-defined functions …
Hisoka-X Mar 5, 2026
8b29134
[FLINK-38726][fluss] Bump Fluss version to 0.9.0-incubating
leonardBang Mar 6, 2026
b858421
[FLINK-39204][pipeline-connector/fluss] Fluss yaml sink support add c…
loserwang1024 Mar 6, 2026
86a3ea5
[FLINK-39152][pipeline-connector][Fluss]Fix FlussConfig Utils class …
ThorneANN Mar 9, 2026
40ec18d
[FLINK-38160][starrocks] Add support for BINARY and VARBINARY types i…
Hisoka-X Mar 9, 2026
9c2af28
[DOC][MINOR] Update starrocks.md (#4304)
zml1206 Mar 9, 2026
2e40f04
[FLINK-37292] [cdc] Remove the flink-kubernetes dependencies during t…
aiwenmo Mar 9, 2026
ffb8b14
[minor][docs][style] Update documentation link and improve code style…
jinkachy Mar 9, 2026
1de9849
[FLINK-38247][MySQL] Handle BIGINT UNSIGNED overflow correctly in Pre…
suhwan-cheon Mar 9, 2026
2ff5681
[FLINK-39055][Iceberg] Support default column values in Iceberg sink …
suhwan-cheon Mar 10, 2026
ffdbf69
[FLINK-38911][mysql] Support binlog-only newly added table capture fo…
Jan 28, 2026
3660060
fix some desc and replace use tableid to explain
Mar 19, 2026
c149a8b
[FLINK-38911][mysql] Support binlog-only newly added table capture fo…
Jan 28, 2026
6107bc8
datastream api support scan.binlog.newly-added-table ability
Jan 29, 2026
46f8a71
remove the some desc
Jan 30, 2026
ddd6897
Supplement the missing binlog_sql test's file
Feb 10, 2026
8e52db9
fix test class
Feb 26, 2026
c81d99f
fix 2
Feb 26, 2026
fb2ee3c
fix mvn spotless:apply
Mar 9, 2026
ffb2cd0
[FLINK-38911][mysql] Support binlog-only newly added table capture fo…
Jan 28, 2026
40278d6
fix some desc and replace use tableid to explain
Mar 19, 2026
4e83d27
Merge remote-tracking branch 'origin/FLINK-38911-binlog-added-new-tab…
Mar 19, 2026
9f85a5e
Merge branch 'apache:master' into FLINK-38911-binlog-added-new-table
ThorneANN Mar 24, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 15 additions & 0 deletions docs/content.zh/docs/connectors/flink-sources/mysql-cdc.md
Original file line number Diff line number Diff line change
Expand Up @@ -398,6 +398,21 @@ Flink SQL> SELECT * FROM orders;
<td>Boolean</td>
<td>是否在快照结束后关闭空闲的 Reader。 此特性需要 flink 版本大于等于 1.14 并且 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' 需要设置为 true。</td>
</tr>
<tr>
<td>scan.binlog.newly-added-table.enabled</td>
<td>optional</td>
<td style="word-wrap: break-word;">false</td>
<td>Boolean</td>
<td>不能与 <code>scan.newly-added-table.enabled</code> 同时启用。<br>
<b><code>table-name</code>参数 匹配模式示例:</b>
<ul>
<li><code>db\.*</code> - 捕获数据库 'db' 下的所有表</li>
<li><code>db\.user_\.*</code> - 捕获数据库 'db' 下类似 'user_orders'、'user_profiles' 的表</li>
<li><code>db\.order_[0-9]+</code> - 捕获数据库 'db' 下类似 'order_1'、'order_2' 的表</li>
<li><code>db1\.*,db2\.user_\.*</code> - 捕获 'db1' 下所有表和 'db2' 下 'user_*' 开头的表</li>
</ul>
</td>
</tr>
<tr>
<td>scan.parse.online.schema.changes.enabled</td>
<td>optional</td>
Expand Down
15 changes: 15 additions & 0 deletions docs/content/docs/connectors/flink-sources/mysql-cdc.md
Original file line number Diff line number Diff line change
Expand Up @@ -400,6 +400,21 @@ Only valid for cdc 1.x version. During a snapshot operation, the connector will
so it does not need to be explicitly configured 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' = 'true'
</td>
</tr>
<tr>
<td>scan.binlog.newly-added-table.enabled</td>
<td>optional</td>
<td style="word-wrap: break-word;">false</td>
<td>Boolean</td>
<td>Cannot be enabled together with <code>scan.newly-added-table.enabled</code>. <br>
<b><code>table-name</code> config Examples:</b>
<ul>
<li><code>db\.*</code> - captures all tables in database 'db'</li>
<li><code>db\.user_\.*</code> - captures tables like 'user_orders', 'user_profiles' in database 'db'</li>
<li><code>db\.order_[0-9]+</code> - captures tables like 'order_1', 'order_2' in database 'db'</li>
<li><code>db1\.*,db2\.user_\.*</code> - captures all tables in 'db1' and 'user_*' tables in 'db2'</li>
</ul>
</td>
</tr>
<tr>
<td>debezium.binary.handling.mode</td>
<td>optional</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -335,14 +335,25 @@ private boolean shouldEmit(SourceRecord sourceRecord) {
}

private boolean hasEnterPureBinlogPhase(TableId tableId, BinlogOffset position) {
// the existed tables those have finished snapshot reading
// Case 1: the existed tables those have finished snapshot reading
if (maxSplitHighWatermarkMap.containsKey(tableId)
&& position.isAfter(maxSplitHighWatermarkMap.get(tableId))) {
pureBinlogPhaseTables.add(tableId);
return true;
}

// Use still need to capture new sharding table if user disable scan new added table,
// Case 2: binlog-only mode for newly added tables
// Capture new tables that match the filter without snapshot
if (statefulTaskContext.getSourceConfig().isScanBinlogNewlyAddedTableEnabled()) {
if (!maxSplitHighWatermarkMap.containsKey(tableId)
&& capturedTableFilter.test(tableId)) {
LOG.info("Auto-capturing newly added table in binlog-only mode: {}", tableId);
pureBinlogPhaseTables.add(tableId);
return true;
}
}

// Case 3: Use still need to capture new sharding table if user disable scan new added table,
// The history records for all new added tables(including sharding table and normal table)
// will be capture after restore from a savepoint if user enable scan new added table
if (!statefulTaskContext.getSourceConfig().isScanNewlyAddedTableEnabled()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.flink.cdc.connectors.mysql.source;

import org.apache.flink.cdc.common.annotation.Experimental;
import org.apache.flink.cdc.common.annotation.PublicEvolving;
import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory;
import org.apache.flink.cdc.connectors.mysql.table.StartupOptions;
Expand Down Expand Up @@ -211,6 +212,18 @@ public MySqlSourceBuilder<T> scanNewlyAddedTableEnabled(boolean scanNewlyAddedTa
return this;
}

/**
* Whether to capture newly added tables in binlog reading phase without snapshot. This option
* can only be used with stream-only startup modes. Cannot be enabled together with {@link
* #scanNewlyAddedTableEnabled(boolean)}.
*/
@Experimental
public MySqlSourceBuilder<T> scanBinlogNewlyAddedTableEnabled(
boolean scanBinlogNewlyAddedTableEnabled) {
this.configFactory.scanBinlogNewlyAddedTableEnabled(scanBinlogNewlyAddedTableEnabled);
return this;
}

/** Specifies the startup options. */
public MySqlSourceBuilder<T> startupOptions(StartupOptions startupOptions) {
this.configFactory.startupOptions(startupOptions);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ public class MySqlSourceConfig implements Serializable {
private final boolean includeHeartbeatEvents;
private final boolean includeTransactionMetadataEvents;
private final boolean scanNewlyAddedTableEnabled;
private final boolean scanBinlogNewlyAddedTableEnabled;
private final boolean closeIdleReaders;
private final Properties jdbcProperties;
private final Map<ObjectPath, String> chunkKeyColumns;
Expand Down Expand Up @@ -104,6 +105,7 @@ public class MySqlSourceConfig implements Serializable {
boolean includeHeartbeatEvents,
boolean includeTransactionMetadataEvents,
boolean scanNewlyAddedTableEnabled,
boolean scanBinlogNewlyAddedTableEnabled,
boolean closeIdleReaders,
Properties dbzProperties,
Properties jdbcProperties,
Expand Down Expand Up @@ -135,6 +137,7 @@ public class MySqlSourceConfig implements Serializable {
this.includeHeartbeatEvents = includeHeartbeatEvents;
this.includeTransactionMetadataEvents = includeTransactionMetadataEvents;
this.scanNewlyAddedTableEnabled = scanNewlyAddedTableEnabled;
this.scanBinlogNewlyAddedTableEnabled = scanBinlogNewlyAddedTableEnabled;
this.closeIdleReaders = closeIdleReaders;
this.dbzProperties = checkNotNull(dbzProperties);
this.dbzConfiguration = Configuration.from(dbzProperties);
Expand Down Expand Up @@ -245,6 +248,10 @@ public boolean isScanNewlyAddedTableEnabled() {
return scanNewlyAddedTableEnabled;
}

public boolean isScanBinlogNewlyAddedTableEnabled() {
return scanBinlogNewlyAddedTableEnabled;
}

public boolean isCloseIdleReaders() {
return closeIdleReaders;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.flink.cdc.connectors.mysql.source.config;

import org.apache.flink.cdc.common.annotation.Internal;
import org.apache.flink.cdc.common.route.TableIdRouter;
import org.apache.flink.cdc.connectors.mysql.debezium.EmbeddedFlinkDatabaseHistory;
import org.apache.flink.cdc.connectors.mysql.source.MySqlSource;
import org.apache.flink.cdc.connectors.mysql.table.StartupOptions;
Expand Down Expand Up @@ -68,6 +69,7 @@ public class MySqlSourceConfigFactory implements Serializable {
private boolean includeHeartbeatEvents = false;
private boolean includeTransactionMetadataEvents = false;
private boolean scanNewlyAddedTableEnabled = false;
private boolean scanBinlogNewlyAddedTableEnabled = false;
private boolean closeIdleReaders = false;
private Properties jdbcProperties;
private Duration heartbeatInterval = MySqlSourceOptions.HEARTBEAT_INTERVAL.defaultValue();
Expand Down Expand Up @@ -258,6 +260,17 @@ public MySqlSourceConfigFactory scanNewlyAddedTableEnabled(boolean scanNewlyAdde
return this;
}

/**
* Whether to capture newly added tables in binlog reading phase without snapshot. This option
* can only be used with stream-only startup modes. Cannot be enabled together with {@link
* #scanNewlyAddedTableEnabled(boolean)}.
*/
public MySqlSourceConfigFactory scanBinlogNewlyAddedTableEnabled(
boolean scanBinlogNewlyAddedTableEnabled) {
this.scanBinlogNewlyAddedTableEnabled = scanBinlogNewlyAddedTableEnabled;
return this;
}

/** Custom properties that will overwrite the default JDBC connection URL. */
public MySqlSourceConfigFactory jdbcProperties(Properties jdbcProperties) {
this.jdbcProperties = jdbcProperties;
Expand Down Expand Up @@ -397,8 +410,21 @@ public MySqlSourceConfig createConfig(int subtaskId, String serverName) {
if (databaseList != null) {
props.setProperty("database.include.list", String.join(",", databaseList));
}
// Validate: Two modes are mutually exclusive
if (scanBinlogNewlyAddedTableEnabled && scanNewlyAddedTableEnabled) {
throw new IllegalArgumentException(
"Cannot enable both 'scan.binlog.newly-added-table.enabled' and "
+ "'scan.newly-added-table.enabled' as they may cause duplicate data");
}
Copy link

Copilot AI Feb 11, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The PR description mentions that validation logic should ensure binlog-only mode works only with stream-only startup modes (latest, earliest, specific offset, timestamp). However, this validation is missing. When scanBinlogNewlyAddedTableEnabled is true but the startup mode is not stream-only (e.g., initial or snapshot), the behavior may be incorrect. Add validation to check if startupOptions.isStreamOnly() when scanBinlogNewlyAddedTableEnabled is enabled.

Suggested change
}
}
// Validate: binlog-only newly-added-table scan requires a stream-only startup mode
if (scanBinlogNewlyAddedTableEnabled && !startupOptions.isStreamOnly()) {
throw new IllegalArgumentException(
"'scan.binlog.newly-added-table.enabled' can only be enabled when using a "
+ "stream-only startup mode (latest-offset, earliest-offset, specific-offset, or timestamp).");
}

Copilot uses AI. Check for mistakes.

if (tableList != null) {
props.setProperty("table.include.list", String.join(",", tableList));
if (scanBinlogNewlyAddedTableEnabled) {
props.setProperty(
"table.include.list",
TableIdRouter.convertTableListToRegExpPattern(String.join(",", tableList)));
} else {
props.setProperty("table.include.list", String.join(",", tableList));
}
}
if (serverTimeZone != null) {
props.setProperty("database.serverTimezone", serverTimeZone);
Expand Down Expand Up @@ -436,6 +462,7 @@ public MySqlSourceConfig createConfig(int subtaskId, String serverName) {
includeHeartbeatEvents,
includeTransactionMetadataEvents,
scanNewlyAddedTableEnabled,
scanBinlogNewlyAddedTableEnabled,
closeIdleReaders,
props,
jdbcProperties,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -243,6 +243,22 @@ public class MySqlSourceOptions {
.withDescription(
"Whether capture the scan the newly added tables or not, by default is false. This option is only useful when we start the job from a savepoint/checkpoint.");

@Experimental
public static final ConfigOption<Boolean> SCAN_BINLOG_NEWLY_ADDED_TABLE_ENABLED =
ConfigOptions.key("scan.binlog.newly-added-table.enabled")
.booleanType()
.defaultValue(false)
.withDescription(
"In binlog reading stage, whether to capture newly added tables "
+ "that match the table patterns. When enabled, new tables will be "
+ "captured without snapshot, only binlog events will be emitted. "
+ "Cannot be enabled together with 'scan.newly-added-table.enabled'. "
+ "table-name pattern examples: "
+ "'db.\\.*' (all tables in database 'db'), "
+ "'db.user_\\.*' (tables like 'user_orders', 'user_profiles'), "
+ "'db.order_[0-9]+' (tables like 'order_1', 'order_2'), "
+ "'db1.\\.*,db2.user_\\.*' (all tables in 'db1' and 'user_*' tables in 'db2').");

@Experimental
public static final ConfigOption<String> SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN =
ConfigOptions.key("scan.incremental.snapshot.chunk.key-column")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,7 @@ public class MySqlTableSource implements ScanTableSource, SupportsReadingMetadat
private final double distributionFactorLower;
private final StartupOptions startupOptions;
private final boolean scanNewlyAddedTableEnabled;
private final boolean scanBinlogNewlyAddedTableEnabled;
private final boolean closeIdleReaders;
private final Properties jdbcProperties;
private final Duration heartbeatInterval;
Expand Down Expand Up @@ -136,6 +137,7 @@ public MySqlTableSource(
double distributionFactorLower,
StartupOptions startupOptions,
boolean scanNewlyAddedTableEnabled,
boolean scanBinlogNewlyAddedTableEnabled,
boolean closeIdleReaders,
Properties jdbcProperties,
Duration heartbeatInterval,
Expand Down Expand Up @@ -166,6 +168,7 @@ public MySqlTableSource(
this.distributionFactorLower = distributionFactorLower;
this.startupOptions = startupOptions;
this.scanNewlyAddedTableEnabled = scanNewlyAddedTableEnabled;
this.scanBinlogNewlyAddedTableEnabled = scanBinlogNewlyAddedTableEnabled;
this.closeIdleReaders = closeIdleReaders;
this.jdbcProperties = jdbcProperties;
this.parseOnlineSchemaChanges = parseOnlineSchemaChanges;
Expand Down Expand Up @@ -233,6 +236,7 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
.startupOptions(startupOptions)
.deserializer(deserializer)
.scanNewlyAddedTableEnabled(scanNewlyAddedTableEnabled)
.scanBinlogNewlyAddedTableEnabled(scanBinlogNewlyAddedTableEnabled)
.closeIdleReaders(closeIdleReaders)
.jdbcProperties(jdbcProperties)
.heartbeatInterval(heartbeatInterval)
Expand Down Expand Up @@ -322,6 +326,7 @@ public DynamicTableSource copy() {
distributionFactorLower,
startupOptions,
scanNewlyAddedTableEnabled,
scanBinlogNewlyAddedTableEnabled,
closeIdleReaders,
jdbcProperties,
heartbeatInterval,
Expand Down Expand Up @@ -353,6 +358,7 @@ public boolean equals(Object o) {
&& distributionFactorUpper == that.distributionFactorUpper
&& distributionFactorLower == that.distributionFactorLower
&& scanNewlyAddedTableEnabled == that.scanNewlyAddedTableEnabled
&& scanBinlogNewlyAddedTableEnabled == that.scanBinlogNewlyAddedTableEnabled
&& closeIdleReaders == that.closeIdleReaders
&& Objects.equals(physicalSchema, that.physicalSchema)
&& Objects.equals(hostname, that.hostname)
Expand Down Expand Up @@ -405,6 +411,7 @@ public int hashCode() {
producedDataType,
metadataKeys,
scanNewlyAddedTableEnabled,
scanBinlogNewlyAddedTableEnabled,
closeIdleReaders,
jdbcProperties,
heartbeatInterval,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,8 @@ public DynamicTableSource createDynamicTableSource(Context context) {
config.get(MySqlSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND);
boolean scanNewlyAddedTableEnabled =
config.get(MySqlSourceOptions.SCAN_NEWLY_ADDED_TABLE_ENABLED);
boolean scanBinlogNewlyAddedTableEnabled =
config.get(MySqlSourceOptions.SCAN_BINLOG_NEWLY_ADDED_TABLE_ENABLED);
Duration heartbeatInterval = config.get(MySqlSourceOptions.HEARTBEAT_INTERVAL);
String chunkKeyColumn =
config.getOptional(MySqlSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN)
Expand Down Expand Up @@ -148,6 +150,7 @@ public DynamicTableSource createDynamicTableSource(Context context) {
distributionFactorLower,
startupOptions,
scanNewlyAddedTableEnabled,
scanBinlogNewlyAddedTableEnabled,
closeIdleReaders,
JdbcUrlUtils.getJdbcProperties(context.getCatalogTable().getOptions()),
heartbeatInterval,
Expand Down Expand Up @@ -198,6 +201,7 @@ public Set<ConfigOption<?>> optionalOptions() {
options.add(MySqlSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND);
options.add(MySqlSourceOptions.CONNECT_MAX_RETRIES);
options.add(MySqlSourceOptions.SCAN_NEWLY_ADDED_TABLE_ENABLED);
options.add(MySqlSourceOptions.SCAN_BINLOG_NEWLY_ADDED_TABLE_ENABLED);
options.add(MySqlSourceOptions.SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED);
options.add(MySqlSourceOptions.HEARTBEAT_INTERVAL);
options.add(MySqlSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN);
Expand Down
Loading