Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[mysql] Fix snapshot fetch size conf does not take effect #2766

Merged
merged 1 commit into from
Dec 2, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,7 @@ public void submitSplit(MySqlSplit mySqlSplit) {
this.reachEnd.set(false);
this.splitSnapshotReadTask =
new MySqlSnapshotSplitReadTask(
statefulTaskContext.getSourceConfig(),
statefulTaskContext.getConnectorConfig(),
statefulTaskContext.getSnapshotChangeEventSourceMetrics(),
statefulTaskContext.getDatabaseSchema(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import com.ververica.cdc.connectors.mysql.debezium.dispatcher.EventDispatcherImpl;
import com.ververica.cdc.connectors.mysql.debezium.dispatcher.SignalEventDispatcher;
import com.ververica.cdc.connectors.mysql.debezium.reader.SnapshotSplitReader;
import com.ververica.cdc.connectors.mysql.source.config.MySqlSourceConfig;
import com.ververica.cdc.connectors.mysql.source.offset.BinlogOffset;
import com.ververica.cdc.connectors.mysql.source.split.MySqlSnapshotSplit;
import com.ververica.cdc.connectors.mysql.source.utils.StatementUtils;
Expand Down Expand Up @@ -68,7 +69,7 @@ public class MySqlSnapshotSplitReadTask
/** Interval for showing a log statement with the progress while scanning a single table. */
private static final Duration LOG_INTERVAL = Duration.ofMillis(10_000);

private final MySqlConnectorConfig connectorConfig;
private final MySqlSourceConfig sourceConfig;
private final MySqlDatabaseSchema databaseSchema;
private final MySqlConnection jdbcConnection;
private final EventDispatcherImpl<TableId> dispatcher;
Expand All @@ -79,6 +80,7 @@ public class MySqlSnapshotSplitReadTask
private final SnapshotChangeEventSourceMetrics<MySqlPartition> snapshotChangeEventSourceMetrics;

public MySqlSnapshotSplitReadTask(
MySqlSourceConfig sourceConfig,
MySqlConnectorConfig connectorConfig,
SnapshotChangeEventSourceMetrics<MySqlPartition> snapshotChangeEventSourceMetrics,
MySqlDatabaseSchema databaseSchema,
Expand All @@ -89,7 +91,7 @@ public MySqlSnapshotSplitReadTask(
Clock clock,
MySqlSnapshotSplit snapshotSplit) {
super(connectorConfig, snapshotChangeEventSourceMetrics);
this.connectorConfig = connectorConfig;
this.sourceConfig = sourceConfig;
this.databaseSchema = databaseSchema;
this.jdbcConnection = jdbcConnection;
this.dispatcher = dispatcher;
Expand Down Expand Up @@ -224,7 +226,7 @@ private void createDataEventsForTable(
snapshotSplit.getSplitStart(),
snapshotSplit.getSplitEnd(),
snapshotSplit.getSplitKeyType().getFieldCount(),
connectorConfig.getQueryFetchSize());
sourceConfig.getFetchSize());
ResultSet rs = selectStatement.executeQuery()) {

ColumnUtils.ColumnArray columnArray = ColumnUtils.toArray(rs, table);
Expand Down