|
|
@ -24,10 +24,12 @@ import org.apache.flink.util.FlinkRuntimeException;
|
|
|
|
|
|
|
|
|
|
|
|
import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
|
|
|
|
import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.assigners.MySqlHybridSplitAssigner;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.assigners.MySqlHybridSplitAssigner;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.assigners.MySqlSplitAssigner;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.assigners.MySqlSplitAssigner;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.assigners.state.PendingSplitsState;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.assigners.state.PendingSplitsState;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.config.MySqlSourceConfig;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.config.MySqlSourceConfig;
|
|
|
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.BinlogSplitAssignedEvent;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.BinlogSplitMetaEvent;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.BinlogSplitMetaEvent;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.BinlogSplitMetaRequestEvent;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.BinlogSplitMetaRequestEvent;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.BinlogSplitUpdateAckEvent;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.BinlogSplitUpdateAckEvent;
|
|
|
@ -39,6 +41,7 @@ import com.ververica.cdc.connectors.mysql.source.events.LatestFinishedSplitsNumb
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.LatestFinishedSplitsNumberRequestEvent;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.LatestFinishedSplitsNumberRequestEvent;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.offset.BinlogOffset;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.offset.BinlogOffset;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.split.FinishedSnapshotSplitInfo;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.split.FinishedSnapshotSplitInfo;
|
|
|
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.split.MySqlBinlogSplit;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.split.MySqlSplit;
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.split.MySqlSplit;
|
|
|
|
import org.slf4j.Logger;
|
|
|
|
import org.slf4j.Logger;
|
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
@ -72,6 +75,8 @@ public class MySqlSourceEnumerator implements SplitEnumerator<MySqlSplit, Pendin
|
|
|
|
private final TreeSet<Integer> readersAwaitingSplit;
|
|
|
|
private final TreeSet<Integer> readersAwaitingSplit;
|
|
|
|
private List<List<FinishedSnapshotSplitInfo>> binlogSplitMeta;
|
|
|
|
private List<List<FinishedSnapshotSplitInfo>> binlogSplitMeta;
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
@Nullable private Integer binlogSplitTaskId;
|
|
|
|
|
|
|
|
|
|
|
|
public MySqlSourceEnumerator(
|
|
|
|
public MySqlSourceEnumerator(
|
|
|
|
SplitEnumeratorContext<MySqlSplit> context,
|
|
|
|
SplitEnumeratorContext<MySqlSplit> context,
|
|
|
|
MySqlSourceConfig sourceConfig,
|
|
|
|
MySqlSourceConfig sourceConfig,
|
|
|
@ -107,6 +112,12 @@ public class MySqlSourceEnumerator implements SplitEnumerator<MySqlSplit, Pendin
|
|
|
|
@Override
|
|
|
|
@Override
|
|
|
|
public void addSplitsBack(List<MySqlSplit> splits, int subtaskId) {
|
|
|
|
public void addSplitsBack(List<MySqlSplit> splits, int subtaskId) {
|
|
|
|
LOG.debug("The enumerator adds splits back: {}", splits);
|
|
|
|
LOG.debug("The enumerator adds splits back: {}", splits);
|
|
|
|
|
|
|
|
Optional<MySqlSplit> binlogSplit =
|
|
|
|
|
|
|
|
splits.stream().filter(MySqlSplit::isBinlogSplit).findAny();
|
|
|
|
|
|
|
|
if (binlogSplit.isPresent()) {
|
|
|
|
|
|
|
|
LOG.info("The enumerator adds add binlog split back: {}", binlogSplit);
|
|
|
|
|
|
|
|
this.binlogSplitTaskId = null;
|
|
|
|
|
|
|
|
}
|
|
|
|
splitAssigner.addSplits(splits);
|
|
|
|
splitAssigner.addSplits(splits);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
@ -153,6 +164,11 @@ public class MySqlSourceEnumerator implements SplitEnumerator<MySqlSplit, Pendin
|
|
|
|
"The enumerator receives request from subtask {} for the latest finished splits number after added newly tables. ",
|
|
|
|
"The enumerator receives request from subtask {} for the latest finished splits number after added newly tables. ",
|
|
|
|
subtaskId);
|
|
|
|
subtaskId);
|
|
|
|
handleLatestFinishedSplitNumberRequest(subtaskId);
|
|
|
|
handleLatestFinishedSplitNumberRequest(subtaskId);
|
|
|
|
|
|
|
|
} else if (sourceEvent instanceof BinlogSplitAssignedEvent) {
|
|
|
|
|
|
|
|
LOG.info(
|
|
|
|
|
|
|
|
"The enumerator receives notice from subtask {} for the binlog split assignment. ",
|
|
|
|
|
|
|
|
subtaskId);
|
|
|
|
|
|
|
|
binlogSplitTaskId = subtaskId;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
@ -188,7 +204,10 @@ public class MySqlSourceEnumerator implements SplitEnumerator<MySqlSplit, Pendin
|
|
|
|
continue;
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
if (splitAssigner.isStreamSplitAssigned() && sourceConfig.isCloseIdleReaders()) {
|
|
|
|
if (splitAssigner.isStreamSplitAssigned()
|
|
|
|
|
|
|
|
&& sourceConfig.isCloseIdleReaders()
|
|
|
|
|
|
|
|
&& noMoreSnapshotSplits()
|
|
|
|
|
|
|
|
&& (binlogSplitTaskId != null && !binlogSplitTaskId.equals(nextAwaiting))) {
|
|
|
|
// close idle readers when snapshot phase finished.
|
|
|
|
// close idle readers when snapshot phase finished.
|
|
|
|
context.signalNoMoreSplits(nextAwaiting);
|
|
|
|
context.signalNoMoreSplits(nextAwaiting);
|
|
|
|
awaitingReader.remove();
|
|
|
|
awaitingReader.remove();
|
|
|
@ -200,6 +219,9 @@ public class MySqlSourceEnumerator implements SplitEnumerator<MySqlSplit, Pendin
|
|
|
|
if (split.isPresent()) {
|
|
|
|
if (split.isPresent()) {
|
|
|
|
final MySqlSplit mySqlSplit = split.get();
|
|
|
|
final MySqlSplit mySqlSplit = split.get();
|
|
|
|
context.assignSplit(mySqlSplit, nextAwaiting);
|
|
|
|
context.assignSplit(mySqlSplit, nextAwaiting);
|
|
|
|
|
|
|
|
if (mySqlSplit instanceof MySqlBinlogSplit) {
|
|
|
|
|
|
|
|
this.binlogSplitTaskId = nextAwaiting;
|
|
|
|
|
|
|
|
}
|
|
|
|
awaitingReader.remove();
|
|
|
|
awaitingReader.remove();
|
|
|
|
LOG.info("The enumerator assigns split {} to subtask {}", mySqlSplit, nextAwaiting);
|
|
|
|
LOG.info("The enumerator assigns split {} to subtask {}", mySqlSplit, nextAwaiting);
|
|
|
|
} else {
|
|
|
|
} else {
|
|
|
@ -210,6 +232,16 @@ public class MySqlSourceEnumerator implements SplitEnumerator<MySqlSplit, Pendin
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
private boolean noMoreSnapshotSplits() {
|
|
|
|
|
|
|
|
if (splitAssigner instanceof MySqlHybridSplitAssigner) {
|
|
|
|
|
|
|
|
return ((MySqlHybridSplitAssigner) splitAssigner).noMoreSnapshotSplits();
|
|
|
|
|
|
|
|
} else if (splitAssigner instanceof MySqlBinlogSplitAssigner) {
|
|
|
|
|
|
|
|
return true;
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
throw new IllegalStateException(
|
|
|
|
|
|
|
|
"Unexpected subtype of MySqlSplitAssigner class when invoking noMoreSnapshotSplits.");
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
private int[] getRegisteredReader() {
|
|
|
|
private int[] getRegisteredReader() {
|
|
|
|
return this.context.registeredReaders().keySet().stream()
|
|
|
|
return this.context.registeredReaders().keySet().stream()
|
|
|
|
.mapToInt(Integer::intValue)
|
|
|
|
.mapToInt(Integer::intValue)
|
|
|
|