|
|
|
@ -40,6 +40,7 @@ import javax.annotation.Nullable;
|
|
|
|
|
|
|
|
|
|
import java.io.IOException;
|
|
|
|
|
import java.util.ArrayList;
|
|
|
|
|
import java.util.Collection;
|
|
|
|
|
import java.util.Comparator;
|
|
|
|
|
import java.util.HashMap;
|
|
|
|
|
import java.util.List;
|
|
|
|
@ -77,7 +78,7 @@ public class MySQLSourceEnumerator implements SplitEnumerator<MySQLSplit, MySQLS
|
|
|
|
|
// when the MySQLSourceEnumerator restore, it may missed some report information from reader
|
|
|
|
|
// tell all readers what we have received and request readers report their finished splits
|
|
|
|
|
notifyReaderReceivedFinishedSplits(assignedSplits.keySet().toArray(new Integer[0]));
|
|
|
|
|
notifyReaderReportFinishedSplits(assignedSplits.keySet().toArray(new Integer[0]));
|
|
|
|
|
notifyReaderReportFinishedSplitsIfNeed(assignedSplits.keySet().toArray(new Integer[0]));
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Override
|
|
|
|
@ -86,34 +87,46 @@ public class MySQLSourceEnumerator implements SplitEnumerator<MySQLSplit, MySQLS
|
|
|
|
|
// assign snapshot split firstly
|
|
|
|
|
if (split.isPresent()) {
|
|
|
|
|
context.assignSplit(split.get(), subtaskId);
|
|
|
|
|
|
|
|
|
|
// record assigned splits
|
|
|
|
|
recordAssignedSplits(split.get(), subtaskId);
|
|
|
|
|
LOGGER.info("Assign snapshot split {} for subtask {}", split.get(), subtaskId);
|
|
|
|
|
}
|
|
|
|
|
// no more snapshot split, try notify no more splits
|
|
|
|
|
else if (couldNotifyNoMoreSplits(subtaskId)) {
|
|
|
|
|
context.signalNoMoreSplits(subtaskId);
|
|
|
|
|
}
|
|
|
|
|
// no more snapshot split, try assign binlog split
|
|
|
|
|
else if (couldAssignBinlogSplit(subtaskId)) {
|
|
|
|
|
assignBinlogSplit(subtaskId);
|
|
|
|
|
LOGGER.info("Assign binlog split for subtask {}", subtaskId);
|
|
|
|
|
} else if (hasAssignedBinlogSplit(subtaskId)) {
|
|
|
|
|
context.signalNoMoreSplits(subtaskId);
|
|
|
|
|
LOGGER.info("No available split for subtask {}", subtaskId);
|
|
|
|
|
return;
|
|
|
|
|
} else {
|
|
|
|
|
// the binlog split can not assign due to snapshot splits report is incomplete
|
|
|
|
|
// tell reader report finished snapshot splits
|
|
|
|
|
notifyReaderReportFinishedSplits(new Integer[] {subtaskId});
|
|
|
|
|
// no more snapshot split, try assign binlog split
|
|
|
|
|
if (couldAssignBinlogSplit()) {
|
|
|
|
|
assignBinlogSplit(subtaskId);
|
|
|
|
|
LOGGER.info("Assign binlog split for subtask {}", subtaskId);
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
// no more snapshot split, try notify no more splits
|
|
|
|
|
else if (couldNotifyNoMoreSplits(subtaskId)) {
|
|
|
|
|
context.signalNoMoreSplits(subtaskId);
|
|
|
|
|
LOGGER.info("No available split for subtask {}", subtaskId);
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
// the binlog split may can not assign due to snapshot splits report is
|
|
|
|
|
// incomplete, tell reader report finished snapshot splits
|
|
|
|
|
notifyReaderReportFinishedSplitsIfNeed(new Integer[] {subtaskId});
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private void notifyReaderReportFinishedSplits(Integer[] subtaskIds) {
|
|
|
|
|
private void notifyReaderReportFinishedSplitsIfNeed(Integer[] subtaskIds) {
|
|
|
|
|
// call reader report finished snapshot
|
|
|
|
|
for (int subtaskId : subtaskIds) {
|
|
|
|
|
context.sendEventToSourceReader(subtaskId, new EnumeratorRequestReportEvent());
|
|
|
|
|
LOGGER.info("The enumerator call subtask {} to report its finished splits.", subtaskId);
|
|
|
|
|
final List<MySQLSplit> assignedSplit =
|
|
|
|
|
assignedSplits.getOrDefault(subtaskId, new ArrayList<>());
|
|
|
|
|
final List<Tuple2<String, BinlogPosition>> ackSpitsForReader =
|
|
|
|
|
receiveFinishedSnapshotSplits.getOrDefault(subtaskId, new ArrayList<>());
|
|
|
|
|
int assignedSnapshotSplitSize =
|
|
|
|
|
assignedSplit.stream()
|
|
|
|
|
.filter(sqlSplit -> sqlSplit.getSplitKind() == MySQLSplitKind.SNAPSHOT)
|
|
|
|
|
.collect(Collectors.toList())
|
|
|
|
|
.size();
|
|
|
|
|
if (assignedSnapshotSplitSize > ackSpitsForReader.size()) {
|
|
|
|
|
context.sendEventToSourceReader(subtaskId, new EnumeratorRequestReportEvent());
|
|
|
|
|
LOGGER.info(
|
|
|
|
|
"The enumerator call subtask {} to report its finished splits.", subtaskId);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
@ -172,43 +185,49 @@ public class MySQLSourceEnumerator implements SplitEnumerator<MySQLSplit, MySQLS
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private boolean couldNotifyNoMoreSplits(int subtaskId) {
|
|
|
|
|
// the task may never be assigned split
|
|
|
|
|
final List<MySQLSplit> assignedSplit = assignedSplits.get(subtaskId);
|
|
|
|
|
if (assignedSplit == null) {
|
|
|
|
|
if (assignedSplit == null || hasAssignedBinlogSplit()) {
|
|
|
|
|
return true;
|
|
|
|
|
} else {
|
|
|
|
|
return false;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private boolean hasAssignedBinlogSplit(int subtaskId) {
|
|
|
|
|
final List<MySQLSplit> assignedSplit = assignedSplits.get(subtaskId);
|
|
|
|
|
if (assignedSplit != null) {
|
|
|
|
|
return assignedSplit.stream()
|
|
|
|
|
.anyMatch(r -> r.getSplitKind().equals(MySQLSplitKind.BINLOG));
|
|
|
|
|
} else {
|
|
|
|
|
return false;
|
|
|
|
|
private boolean hasAssignedBinlogSplit() {
|
|
|
|
|
for (List<MySQLSplit> assignedSplit : assignedSplits.values()) {
|
|
|
|
|
if (assignedSplit != null) {
|
|
|
|
|
return assignedSplit.stream()
|
|
|
|
|
.anyMatch(r -> r.getSplitKind().equals(MySQLSplitKind.BINLOG));
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
return false;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private boolean couldAssignBinlogSplit(int subtaskId) {
|
|
|
|
|
final List<MySQLSplit> assignedSplit = assignedSplits.get(subtaskId);
|
|
|
|
|
final List<Tuple2<String, BinlogPosition>> receiveSnapshotSplits =
|
|
|
|
|
receiveFinishedSnapshotSplits.get(subtaskId);
|
|
|
|
|
if (assignedSplit != null
|
|
|
|
|
&& receiveSnapshotSplits != null
|
|
|
|
|
&& assignedSplit.size() == receiveSnapshotSplits.size()) {
|
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
return false;
|
|
|
|
|
private boolean couldAssignBinlogSplit() {
|
|
|
|
|
final int assignedSnapshotSplit =
|
|
|
|
|
assignedSplits.values().stream()
|
|
|
|
|
.flatMap(Collection::stream)
|
|
|
|
|
.collect(Collectors.toList())
|
|
|
|
|
.size();
|
|
|
|
|
final int receiveSnapshotSplits =
|
|
|
|
|
receiveFinishedSnapshotSplits.values().stream()
|
|
|
|
|
.flatMap(Collection::stream)
|
|
|
|
|
.collect(Collectors.toList())
|
|
|
|
|
.size();
|
|
|
|
|
// All assigned snapshot splits have finished
|
|
|
|
|
return assignedSnapshotSplit == receiveSnapshotSplits;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private void assignBinlogSplit(int subtaskId) {
|
|
|
|
|
private void assignBinlogSplit(int requestTaskId) {
|
|
|
|
|
final List<MySQLSplit> assignedSnapshotSplit =
|
|
|
|
|
assignedSplits.get(subtaskId).stream()
|
|
|
|
|
assignedSplits.values().stream()
|
|
|
|
|
.flatMap(Collection::stream)
|
|
|
|
|
.sorted(Comparator.comparing(MySQLSplit::splitId))
|
|
|
|
|
.collect(Collectors.toList());
|
|
|
|
|
final List<Tuple2<String, BinlogPosition>> receiveSnapshotSplits =
|
|
|
|
|
receiveFinishedSnapshotSplits.get(subtaskId).stream()
|
|
|
|
|
receiveFinishedSnapshotSplits.values().stream()
|
|
|
|
|
.flatMap(Collection::stream)
|
|
|
|
|
.sorted(Comparator.comparing(o -> o.f0))
|
|
|
|
|
.collect(Collectors.toList());
|
|
|
|
|
|
|
|
|
@ -240,7 +259,7 @@ public class MySQLSourceEnumerator implements SplitEnumerator<MySQLSplit, MySQLS
|
|
|
|
|
new MySQLSplit(
|
|
|
|
|
MySQLSplitKind.BINLOG,
|
|
|
|
|
lastSnapshotSplit.getTableId(),
|
|
|
|
|
"binlog-split-" + subtaskId,
|
|
|
|
|
"binlog-split-" + requestTaskId,
|
|
|
|
|
lastSnapshotSplit.getSplitBoundaryType(),
|
|
|
|
|
null,
|
|
|
|
|
null,
|
|
|
|
@ -251,9 +270,9 @@ public class MySQLSourceEnumerator implements SplitEnumerator<MySQLSplit, MySQLS
|
|
|
|
|
snapshotSplits,
|
|
|
|
|
databaseHistory);
|
|
|
|
|
// assign
|
|
|
|
|
context.assignSplit(binlogSplit, subtaskId);
|
|
|
|
|
context.assignSplit(binlogSplit, requestTaskId);
|
|
|
|
|
// record assigned splits
|
|
|
|
|
recordAssignedSplits(binlogSplit, subtaskId);
|
|
|
|
|
recordAssignedSplits(binlogSplit, requestTaskId);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private void recordAssignedSplits(MySQLSplit split, int subtaskId) {
|
|
|
|
|