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

[FLINK-36813][cdc-connectors][mysql] support mysql sync part columns #3767

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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 @@ -40,10 +40,12 @@
import io.debezium.pipeline.spi.ChangeRecordEmitter;
import io.debezium.pipeline.spi.SnapshotResult;
import io.debezium.relational.Column;
import io.debezium.relational.ColumnFilterMode;
import io.debezium.relational.RelationalSnapshotChangeEventSource;
import io.debezium.relational.SnapshotChangeRecordEmitter;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.relational.Tables;
import io.debezium.schema.TopicSelector;
import io.debezium.util.Clock;
import io.debezium.util.ColumnUtils;
Expand All @@ -61,6 +63,11 @@
import java.sql.Types;
import java.time.Duration;
import java.util.Calendar;
import java.util.List;
import java.util.stream.Collectors;

import static io.debezium.relational.RelationalDatabaseConnectorConfig.COLUMN_EXCLUDE_LIST;
import static io.debezium.relational.RelationalDatabaseConnectorConfig.COLUMN_INCLUDE_LIST;

/** Task to read snapshot split of table. */
public class MySqlSnapshotSplitReadTask
Expand All @@ -83,6 +90,7 @@ public class MySqlSnapshotSplitReadTask

private final SnapshotPhaseHooks hooks;
private final boolean isBackfillSkipped;
private final Tables.ColumnNameFilter columnFilter;

public MySqlSnapshotSplitReadTask(
MySqlSourceConfig sourceConfig,
Expand All @@ -109,6 +117,21 @@ public MySqlSnapshotSplitReadTask(
this.snapshotChangeEventSourceMetrics = snapshotChangeEventSourceMetrics;
this.hooks = hooks;
this.isBackfillSkipped = isBackfillSkipped;

// setting column filter
String columnIncludeList =
sourceConfig.getDbzConfiguration().getString(COLUMN_INCLUDE_LIST);
String columnExcludeList =
sourceConfig.getDbzConfiguration().getString(COLUMN_EXCLUDE_LIST);
if (columnIncludeList != null) {
this.columnFilter =
Tables.ColumnNameFilterFactory.createIncludeListFilter(
columnIncludeList, ColumnFilterMode.CATALOG);
} else {
this.columnFilter =
Tables.ColumnNameFilterFactory.createExcludeListFilter(
columnExcludeList, ColumnFilterMode.CATALOG);
}
}

@Override
Expand Down Expand Up @@ -247,7 +270,8 @@ private void createDataEventsForTable(
snapshotSplit.getTableId(),
snapshotSplit.getSplitKeyType(),
snapshotSplit.getSplitStart() == null,
snapshotSplit.getSplitEnd() == null);
snapshotSplit.getSplitEnd() == null,
getScanColumns(table));
LOG.info(
"For split '{}' of table {} using select statement: '{}'",
snapshotSplit.splitId(),
Expand Down Expand Up @@ -305,6 +329,24 @@ private void createDataEventsForTable(
}
}

private String getScanColumns(Table table) {
List<String> columnNames =
table.retrieveColumnNames().stream()
.filter(
columnName ->
columnFilter.matches(
table.id().catalog(),
table.id().schema(),
table.id().table(),
columnName))
.map(columnName -> jdbcConnection.quotedColumnIdString(columnName))
.collect(Collectors.toList());
if (columnNames.isEmpty()) {
columnNames.add("*");
}
return String.join(", ", columnNames);
}

protected ChangeRecordEmitter<MySqlPartition> getChangeRecordEmitter(
MySqlSnapshotContext snapshotContext, TableId tableId, Object[] row) {
snapshotContext.offset.event(tableId, clock.currentTime());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,8 +131,12 @@ public static Object queryNextChunkMax(
}

public static String buildSplitScanQuery(
TableId tableId, RowType pkRowType, boolean isFirstSplit, boolean isLastSplit) {
return buildSplitQuery(tableId, pkRowType, isFirstSplit, isLastSplit, -1, true);
TableId tableId,
RowType pkRowType,
boolean isFirstSplit,
boolean isLastSplit,
String projection) {
return buildSplitQuery(tableId, pkRowType, isFirstSplit, isLastSplit, -1, true, projection);
}

private static String buildSplitQuery(
Expand All @@ -141,7 +145,8 @@ private static String buildSplitQuery(
boolean isFirstSplit,
boolean isLastSplit,
int limitSize,
boolean isScanningData) {
boolean isScanningData,
String projection) {
final String condition;

if (isFirstSplit && isLastSplit) {
Expand Down Expand Up @@ -174,7 +179,11 @@ private static String buildSplitQuery(

if (isScanningData) {
return buildSelectWithRowLimits(
tableId, limitSize, "*", Optional.ofNullable(condition), Optional.empty());
tableId,
limitSize,
projection,
Optional.ofNullable(condition),
Optional.empty());
} else {
final String orderBy =
pkRowType.getFieldNames().stream().collect(Collectors.joining(", "));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2351,4 +2351,70 @@ public void testBinaryHandlingModeWithBase64() throws Exception {
assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
result.getJobClient().get().cancel().get();
}

@Test
public void testColumnFilters() throws Exception {
customerDatabase.createAndInitialize();
String sourceDDL =
String.format(
"CREATE TABLE address ("
+ " `id` DECIMAL(20, 0) NOT NULL,"
+ " country STRING,"
+ " city STRING,"
+ " detail_address STRING,"
+ " primary key (`id`) not enforced"
+ ") WITH ("
+ " 'connector' = 'mysql-cdc',"
+ " 'hostname' = '%s',"
+ " 'port' = '%s',"
+ " 'username' = '%s',"
+ " 'password' = '%s',"
+ " 'database-name' = '%s',"
+ " 'table-name' = '%s',"
+ " 'debezium.column.include.list' = '%s.%s.id,%s.%s.country', "
+ " 'server-time-zone' = 'UTC',"
+ " 'server-id' = '%s'"
+ ")",
MYSQL_CONTAINER.getHost(),
MYSQL_CONTAINER.getDatabasePort(),
customerDatabase.getUsername(),
customerDatabase.getPassword(),
customerDatabase.getDatabaseName(),
"address",
customerDatabase.getDatabaseName(),
"address",
customerDatabase.getDatabaseName(),
"address",
getServerId());
tEnv.executeSql(sourceDDL);
// async submit job
TableResult result = tEnv.executeSql("SELECT * from address");

CloseableIterator<Row> iterator = result.collect();
waitForSnapshotStarted(iterator);

try (Connection connection = customerDatabase.getJdbcConnection();
Statement statement = connection.createStatement()) {
statement.execute("UPDATE address SET city = 'Hangzhou' WHERE id=416927583791428523;");
statement.execute(
"INSERT INTO address VALUES(418257940021724075, 'Germany', 'Berlin', 'West Town address 3')");
}

String[] expected =
new String[] {
"+I[417271541558096811, America, null, null]",
"+I[417272886855938987, America, null, null]",
"+I[417111867899200427, America, null, null]",
"+I[417420106184475563, Germany, null, null]",
"+I[418161258277847979, Germany, null, null]",
"+I[416874195632735147, China, null, null]",
"+I[416927583791428523, China, null, null]",
"+I[417022095255614379, China, null, null]",
"-U[416927583791428523, China, null, null]",
"+U[416927583791428523, China, null, null]",
"+I[418257940021724075, Germany, null, null]"
};
assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
result.getJobClient().get().cancel().get();
}
}
Loading