Skip to content

[FLINK-38265] Stream Split shouldn't finish when exception occors but met END Watermark. #4101

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

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 @@ -319,23 +319,15 @@ public void exitAlterByChangeColumn(MySqlParser.AlterByChangeColumnContext ctx)
parser.runIfNotNull(
() -> {
Column column = columnDefinitionListener.getColumn();
String oldColumnName =
isTableIdCaseInsensitive
? column.name().toLowerCase(Locale.ROOT)
: column.name();
String newColumnName = parser.parseName(ctx.newColumn);
if (isTableIdCaseInsensitive && newColumnName != null) {
newColumnName = newColumnName.toLowerCase(Locale.ROOT);
}

Map<String, DataType> typeMapping = new HashMap<>();

typeMapping.put(oldColumnName, fromDbzColumn(column, tinyInt1isBit));
typeMapping.put(column.name(), fromDbzColumn(column, tinyInt1isBit));
changes.add(new AlterColumnTypeEvent(currentTable, typeMapping));

if (newColumnName != null && !oldColumnName.equalsIgnoreCase(newColumnName)) {
if (newColumnName != null && !column.name().equalsIgnoreCase(newColumnName)) {
Map<String, String> renameMap = new HashMap<>();
renameMap.put(oldColumnName, newColumnName);
renameMap.put(column.name(), newColumnName);
changes.add(new RenameColumnEvent(currentTable, renameMap));
}
listeners.remove(columnDefinitionListener);
Expand All @@ -347,9 +339,6 @@ public void exitAlterByChangeColumn(MySqlParser.AlterByChangeColumnContext ctx)
@Override
public void enterAlterByDropColumn(MySqlParser.AlterByDropColumnContext ctx) {
String removedColName = parser.parseName(ctx.uid());
if (isTableIdCaseInsensitive && removedColName != null) {
removedColName = removedColName.toLowerCase(Locale.ROOT);
}
changes.add(new DropColumnEvent(currentTable, Collections.singletonList(removedColName)));
super.enterAlterByDropColumn(ctx);
}
Expand Down Expand Up @@ -384,11 +373,7 @@ public void exitAlterByModifyColumn(MySqlParser.AlterByModifyColumnContext ctx)
() -> {
Column column = columnDefinitionListener.getColumn();
Map<String, DataType> typeMapping = new HashMap<>();
typeMapping.put(
isTableIdCaseInsensitive
? column.name().toLowerCase(Locale.ROOT)
: column.name(),
fromDbzColumn(column, tinyInt1isBit));
typeMapping.put(column.name(), fromDbzColumn(column, tinyInt1isBit));
changes.add(new AlterColumnTypeEvent(currentTable, typeMapping));
listeners.remove(columnDefinitionListener);
},
Expand All @@ -401,17 +386,10 @@ public void exitAlterByRenameColumn(MySqlParser.AlterByRenameColumnContext ctx)
parser.runIfNotNull(
() -> {
Column column = columnDefinitionListener.getColumn();
String oldColumnName =
isTableIdCaseInsensitive
? column.name().toLowerCase(Locale.ROOT)
: column.name();
String newColumnName = parser.parseName(ctx.newColumn);
if (isTableIdCaseInsensitive && newColumnName != null) {
newColumnName = newColumnName.toLowerCase(Locale.ROOT);
}
if (newColumnName != null && !column.name().equalsIgnoreCase(newColumnName)) {
Map<String, String> renameMap = new HashMap<>();
renameMap.put(oldColumnName, newColumnName);
renameMap.put(column.name(), newColumnName);
changes.add(new RenameColumnEvent(currentTable, renameMap));
}
listeners.remove(columnDefinitionListener);
Expand Down Expand Up @@ -456,9 +434,7 @@ public void enterTableOptionComment(MySqlParser.TableOptionCommentContext ctx) {

private org.apache.flink.cdc.common.schema.Column toCdcColumn(Column dbzColumn) {
return org.apache.flink.cdc.common.schema.Column.physicalColumn(
isTableIdCaseInsensitive
? dbzColumn.name().toLowerCase(Locale.ROOT)
: dbzColumn.name(),
dbzColumn.name(),
fromDbzColumn(dbzColumn, tinyInt1isBit),
dbzColumn.comment(),
dbzColumn.defaultValueExpression().orElse(null));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -196,7 +196,7 @@ private List<Event> executeAlterAndProvideExpected(TableId tableId, Statement st

statement.execute(
String.format(
"ALTER TABLE `%s`.`products` ADD COLUMN `DESC1` VARCHAR(45) NULL AFTER `weight`;",
"ALTER TABLE `%s`.`products` ADD COLUMN `DESC1` VARCHAR(45) NULL AFTER `WEIGHT`;",
inventoryDatabase.getDatabaseName()));
expected.add(
new AddColumnEvent(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,7 @@ public void submitTask(FetchTask<SourceSplitBase> fetchTask) {
try {
snapshotSplitReadTask.execute(taskContext);
} catch (Exception e) {

setReadException(e);
}
});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,8 @@
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;

import static org.apache.flink.cdc.connectors.base.source.meta.wartermark.WatermarkEvent.isEndWatermarkEvent;

/** Fetcher to fetch data from table split, the split is the stream split {@link StreamSplit}. */
public class IncrementalSourceStreamFetcher implements Fetcher<SourceRecords, SourceSplitBase> {
private static final Logger LOG = LoggerFactory.getLogger(IncrementalSourceStreamFetcher.class);
Expand Down Expand Up @@ -116,10 +118,19 @@ public boolean isFinished() {
public Iterator<SourceRecords> pollSplitRecords() throws InterruptedException {
checkReadException();
final List<SourceRecord> sourceRecords = new ArrayList<>();
// what happens if currentTaskRunning
if (currentTaskRunning) {
List<DataChangeEvent> batch = queue.poll();
for (DataChangeEvent event : batch) {
if (shouldEmit(event.getRecord())) {
if (isEndWatermarkEvent(event.getRecord())) {
LOG.info("Read split {} end watermark event", currentStreamSplit);
try {
stopReadTask();
} catch (Exception e) {
throw new RuntimeException(e);
}
break;
} else if (shouldEmit(event.getRecord())) {
sourceRecords.add(event.getRecord());
} else {
LOG.debug("{} data change event should not emit", event);
Expand Down Expand Up @@ -263,6 +274,7 @@ private void configureFilter() {
}

public void stopReadTask() throws Exception {
// todo: ji
this.currentTaskRunning = false;

if (taskContext != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@

import static org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils.createBinaryClient;
import static org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils.createMySqlConnection;
import static org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils.isEndWatermarkEvent;

/**
* A Debezium binlog reader implementation that also support reads binlog and filter overlapping
Expand Down Expand Up @@ -148,8 +149,6 @@ public void submitSplit(MySqlSplit mySqlSplit) {
currentBinlogSplit),
t);
readException = t;
} finally {
stopBinlogReadTask();
}
});
}
Expand All @@ -167,6 +166,16 @@ public Iterator<SourceRecords> pollSplitRecords() throws InterruptedException {
if (currentTaskRunning) {
List<DataChangeEvent> batch = queue.poll();
for (DataChangeEvent event : batch) {
if (isEndWatermarkEvent(event.getRecord())) {
LOG.info("Read split {} end watermark event", currentBinlogSplit);
try {
stopBinlogReadTask();
} catch (Exception e) {
throw new RuntimeException(e);
}
break;
}

if (isParsingOnLineSchemaChanges) {
Optional<SourceRecord> oscRecord =
parseOnLineSchemaChangeEvent(event.getRecord());
Expand Down
Loading