|
|
|
@ -24,19 +24,19 @@ import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSource
|
|
|
|
|
import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager;
|
|
|
|
|
import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
|
|
|
|
|
import org.apache.flink.util.FlinkRuntimeException;
|
|
|
|
|
import org.apache.flink.util.Preconditions;
|
|
|
|
|
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.debezium.DebeziumUtils;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.config.MySqlSourceConfig;
|
|
|
|
|
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.BinlogSplitUpdateAckEvent;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.BinlogSplitUpdateRequestEvent;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.FinishedSnapshotSplitsAckEvent;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.FinishedSnapshotSplitsReportEvent;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.FinishedSnapshotSplitsRequestEvent;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.LatestFinishedSplitsSizeEvent;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.LatestFinishedSplitsSizeRequestEvent;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.SuspendBinlogReaderAckEvent;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.SuspendBinlogReaderEvent;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.WakeupReaderEvent;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.events.LatestFinishedSplitsNumberEvent;
|
|
|
|
|
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.split.FinishedSnapshotSplitInfo;
|
|
|
|
|
import com.ververica.cdc.connectors.mysql.source.split.MySqlBinlogSplit;
|
|
|
|
@ -57,15 +57,18 @@ import java.sql.SQLException;
|
|
|
|
|
import java.util.ArrayList;
|
|
|
|
|
import java.util.Collections;
|
|
|
|
|
import java.util.HashMap;
|
|
|
|
|
import java.util.HashSet;
|
|
|
|
|
import java.util.List;
|
|
|
|
|
import java.util.Map;
|
|
|
|
|
import java.util.Set;
|
|
|
|
|
import java.util.function.Supplier;
|
|
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
|
|
|
|
|
import static com.ververica.cdc.connectors.mysql.source.events.WakeupReaderEvent.WakeUpTarget.SNAPSHOT_READER;
|
|
|
|
|
import static com.ververica.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID;
|
|
|
|
|
import static com.ververica.cdc.connectors.mysql.source.split.MySqlBinlogSplit.toNormalBinlogSplit;
|
|
|
|
|
import static com.ververica.cdc.connectors.mysql.source.split.MySqlBinlogSplit.toSuspendedBinlogSplit;
|
|
|
|
|
import static com.ververica.cdc.connectors.mysql.source.utils.ChunkUtils.getNextMetaGroupId;
|
|
|
|
|
import static com.ververica.cdc.connectors.mysql.source.utils.TableDiscoveryUtils.discoverSchemaForNewAddedTables;
|
|
|
|
|
|
|
|
|
|
/** The source reader for MySQL source splits. */
|
|
|
|
|
public class MySqlSourceReader<T>
|
|
|
|
@ -73,13 +76,12 @@ public class MySqlSourceReader<T>
|
|
|
|
|
SourceRecords, T, MySqlSplit, MySqlSplitState> {
|
|
|
|
|
|
|
|
|
|
private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceReader.class);
|
|
|
|
|
|
|
|
|
|
private final MySqlSourceConfig sourceConfig;
|
|
|
|
|
private final Map<String, MySqlSnapshotSplit> finishedUnackedSplits;
|
|
|
|
|
private final Map<String, MySqlBinlogSplit> uncompletedBinlogSplits;
|
|
|
|
|
private final int subtaskId;
|
|
|
|
|
private final MySqlSourceReaderContext mySqlSourceReaderContext;
|
|
|
|
|
private MySqlBinlogSplit suspendedBinlogSplit;
|
|
|
|
|
private volatile MySqlBinlogSplit suspendedBinlogSplit;
|
|
|
|
|
|
|
|
|
|
public MySqlSourceReader(
|
|
|
|
|
FutureCompletingBlockingQueue<RecordsWithSplitIds<SourceRecords>> elementQueue,
|
|
|
|
@ -104,7 +106,7 @@ public class MySqlSourceReader<T>
|
|
|
|
|
|
|
|
|
|
@Override
|
|
|
|
|
public void start() {
|
|
|
|
|
if (getNumberOfCurrentlyAssignedSplits() == 0) {
|
|
|
|
|
if (getNumberOfCurrentlyAssignedSplits() <= 1) {
|
|
|
|
|
context.sendSplitRequest();
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
@ -128,9 +130,6 @@ public class MySqlSourceReader<T>
|
|
|
|
|
.filter(split -> !finishedUnackedSplits.containsKey(split.splitId()))
|
|
|
|
|
.collect(Collectors.toList());
|
|
|
|
|
|
|
|
|
|
// add finished snapshot splits that didn't receive ack yet
|
|
|
|
|
unfinishedSplits.addAll(finishedUnackedSplits.values());
|
|
|
|
|
|
|
|
|
|
// add binlog splits who are uncompleted
|
|
|
|
|
unfinishedSplits.addAll(uncompletedBinlogSplits.values());
|
|
|
|
|
|
|
|
|
@ -147,29 +146,51 @@ public class MySqlSourceReader<T>
|
|
|
|
|
@Override
|
|
|
|
|
protected void onSplitFinished(Map<String, MySqlSplitState> finishedSplitIds) {
|
|
|
|
|
boolean requestNextSplit = true;
|
|
|
|
|
for (MySqlSplitState mySqlSplitState : finishedSplitIds.values()) {
|
|
|
|
|
MySqlSplit mySqlSplit = mySqlSplitState.toMySqlSplit();
|
|
|
|
|
if (mySqlSplit.isBinlogSplit()) {
|
|
|
|
|
LOG.info(
|
|
|
|
|
"binlog split reader suspended due to newly added table, offset {}",
|
|
|
|
|
mySqlSplitState.asBinlogSplitState().getStartingOffset());
|
|
|
|
|
|
|
|
|
|
mySqlSourceReaderContext.resetStopBinlogSplitReader();
|
|
|
|
|
suspendedBinlogSplit = toSuspendedBinlogSplit(mySqlSplit.asBinlogSplit());
|
|
|
|
|
context.sendSourceEventToCoordinator(new SuspendBinlogReaderAckEvent());
|
|
|
|
|
// do not request next split when the reader is suspended, the suspended reader will
|
|
|
|
|
// automatically request the next split after it has been wakeup
|
|
|
|
|
requestNextSplit = false;
|
|
|
|
|
} else {
|
|
|
|
|
finishedUnackedSplits.put(mySqlSplit.splitId(), mySqlSplit.asSnapshotSplit());
|
|
|
|
|
if (isNewlyAddedTableSplitAndBinlogSplit(finishedSplitIds)) {
|
|
|
|
|
MySqlSplitState mySqlBinlogSplitState = finishedSplitIds.remove(BINLOG_SPLIT_ID);
|
|
|
|
|
finishedSplitIds
|
|
|
|
|
.values()
|
|
|
|
|
.forEach(
|
|
|
|
|
newAddedSplitState ->
|
|
|
|
|
finishedUnackedSplits.put(
|
|
|
|
|
newAddedSplitState.toMySqlSplit().splitId(),
|
|
|
|
|
newAddedSplitState.toMySqlSplit().asSnapshotSplit()));
|
|
|
|
|
this.addSplits(Collections.singletonList(mySqlBinlogSplitState.toMySqlSplit()));
|
|
|
|
|
} else {
|
|
|
|
|
Preconditions.checkState(finishedSplitIds.size() == 1);
|
|
|
|
|
for (MySqlSplitState mySqlSplitState : finishedSplitIds.values()) {
|
|
|
|
|
MySqlSplit mySqlSplit = mySqlSplitState.toMySqlSplit();
|
|
|
|
|
if (mySqlSplit.isBinlogSplit()) {
|
|
|
|
|
suspendedBinlogSplit = toSuspendedBinlogSplit(mySqlSplit.asBinlogSplit());
|
|
|
|
|
LOG.info(
|
|
|
|
|
"binlog split reader suspended success after the newly added table process, current offset {}",
|
|
|
|
|
suspendedBinlogSplit.getStartingOffset());
|
|
|
|
|
context.sendSourceEventToCoordinator(
|
|
|
|
|
new LatestFinishedSplitsNumberRequestEvent());
|
|
|
|
|
// do not request next split when the reader is suspended
|
|
|
|
|
requestNextSplit = false;
|
|
|
|
|
} else {
|
|
|
|
|
finishedUnackedSplits.put(mySqlSplit.splitId(), mySqlSplit.asSnapshotSplit());
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
reportFinishedSnapshotSplitsIfNeed();
|
|
|
|
|
}
|
|
|
|
|
reportFinishedSnapshotSplitsIfNeed();
|
|
|
|
|
|
|
|
|
|
if (requestNextSplit) {
|
|
|
|
|
context.sendSplitRequest();
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* During the newly added table process, for the source reader who holds the binlog split, we
|
|
|
|
|
* return the latest finished snapshot split and binlog split as well, this design let us have
|
|
|
|
|
* opportunity to exchange binlog reading and snapshot reading, we put the binlog split back.
|
|
|
|
|
*/
|
|
|
|
|
private boolean isNewlyAddedTableSplitAndBinlogSplit(
|
|
|
|
|
Map<String, MySqlSplitState> finishedSplitIds) {
|
|
|
|
|
return finishedSplitIds.containsKey(BINLOG_SPLIT_ID) && finishedSplitIds.size() == 2;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Override
|
|
|
|
|
public void addSplits(List<MySqlSplit> splits) {
|
|
|
|
|
// restore for finishedUnackedSplits
|
|
|
|
@ -185,6 +206,13 @@ public class MySqlSourceReader<T>
|
|
|
|
|
}
|
|
|
|
|
} else {
|
|
|
|
|
MySqlBinlogSplit binlogSplit = split.asBinlogSplit();
|
|
|
|
|
// Try to discovery table schema once for newly added tables when source reader
|
|
|
|
|
// start or restore
|
|
|
|
|
boolean checkNewlyAddedTableSchema =
|
|
|
|
|
!mySqlSourceReaderContext.isHasAssignedBinlogSplit()
|
|
|
|
|
&& sourceConfig.isScanNewlyAddedTableEnabled();
|
|
|
|
|
mySqlSourceReaderContext.setHasAssignedBinlogSplit(true);
|
|
|
|
|
|
|
|
|
|
// the binlog split is suspended
|
|
|
|
|
if (binlogSplit.isSuspended()) {
|
|
|
|
|
suspendedBinlogSplit = binlogSplit;
|
|
|
|
@ -194,7 +222,10 @@ public class MySqlSourceReader<T>
|
|
|
|
|
} else {
|
|
|
|
|
uncompletedBinlogSplits.remove(split.splitId());
|
|
|
|
|
MySqlBinlogSplit mySqlBinlogSplit =
|
|
|
|
|
discoverTableSchemasForBinlogSplit(split.asBinlogSplit());
|
|
|
|
|
discoverTableSchemasForBinlogSplit(
|
|
|
|
|
split.asBinlogSplit(),
|
|
|
|
|
sourceConfig,
|
|
|
|
|
checkNewlyAddedTableSchema);
|
|
|
|
|
unfinishedSplits.add(mySqlBinlogSplit);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
@ -207,26 +238,6 @@ public class MySqlSourceReader<T>
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private MySqlBinlogSplit discoverTableSchemasForBinlogSplit(MySqlBinlogSplit split) {
|
|
|
|
|
final String splitId = split.splitId();
|
|
|
|
|
if (split.getTableSchemas().isEmpty()) {
|
|
|
|
|
try (MySqlConnection jdbc = DebeziumUtils.createMySqlConnection(sourceConfig)) {
|
|
|
|
|
Map<TableId, TableChanges.TableChange> tableSchemas =
|
|
|
|
|
TableDiscoveryUtils.discoverCapturedTableSchemas(sourceConfig, jdbc);
|
|
|
|
|
LOG.info("The table schema discovery for binlog split {} success", splitId);
|
|
|
|
|
return MySqlBinlogSplit.fillTableSchemas(split, tableSchemas);
|
|
|
|
|
} catch (SQLException e) {
|
|
|
|
|
LOG.error("Failed to obtains table schemas due to {}", e.getMessage());
|
|
|
|
|
throw new FlinkRuntimeException(e);
|
|
|
|
|
}
|
|
|
|
|
} else {
|
|
|
|
|
LOG.warn(
|
|
|
|
|
"The binlog split {} has table schemas yet, skip the table schema discovery",
|
|
|
|
|
split);
|
|
|
|
|
return split;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Override
|
|
|
|
|
public void handleSourceEvents(SourceEvent sourceEvent) {
|
|
|
|
|
if (sourceEvent instanceof FinishedSnapshotSplitsAckEvent) {
|
|
|
|
@ -249,33 +260,38 @@ public class MySqlSourceReader<T>
|
|
|
|
|
"The subtask {} receives binlog meta with group id {}.",
|
|
|
|
|
subtaskId,
|
|
|
|
|
((BinlogSplitMetaEvent) sourceEvent).getMetaGroupId());
|
|
|
|
|
fillMetaDataForBinlogSplit((BinlogSplitMetaEvent) sourceEvent);
|
|
|
|
|
} else if (sourceEvent instanceof SuspendBinlogReaderEvent) {
|
|
|
|
|
mySqlSourceReaderContext.setStopBinlogSplitReader();
|
|
|
|
|
} else if (sourceEvent instanceof WakeupReaderEvent) {
|
|
|
|
|
WakeupReaderEvent wakeupReaderEvent = (WakeupReaderEvent) sourceEvent;
|
|
|
|
|
if (wakeupReaderEvent.getTarget() == SNAPSHOT_READER) {
|
|
|
|
|
context.sendSplitRequest();
|
|
|
|
|
} else {
|
|
|
|
|
if (suspendedBinlogSplit != null) {
|
|
|
|
|
context.sendSourceEventToCoordinator(
|
|
|
|
|
new LatestFinishedSplitsSizeRequestEvent());
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
} else if (sourceEvent instanceof LatestFinishedSplitsSizeEvent) {
|
|
|
|
|
if (suspendedBinlogSplit != null) {
|
|
|
|
|
final int finishedSplitsSize =
|
|
|
|
|
((LatestFinishedSplitsSizeEvent) sourceEvent).getLatestFinishedSplitsSize();
|
|
|
|
|
final MySqlBinlogSplit binlogSplit =
|
|
|
|
|
toNormalBinlogSplit(suspendedBinlogSplit, finishedSplitsSize);
|
|
|
|
|
suspendedBinlogSplit = null;
|
|
|
|
|
this.addSplits(Collections.singletonList(binlogSplit));
|
|
|
|
|
}
|
|
|
|
|
fillMetadataForBinlogSplit((BinlogSplitMetaEvent) sourceEvent);
|
|
|
|
|
} else if (sourceEvent instanceof BinlogSplitUpdateRequestEvent) {
|
|
|
|
|
handleBinlogSplitUpdateRequest();
|
|
|
|
|
} else if (sourceEvent instanceof LatestFinishedSplitsNumberEvent) {
|
|
|
|
|
updateBinlogSplit((LatestFinishedSplitsNumberEvent) sourceEvent);
|
|
|
|
|
} else {
|
|
|
|
|
super.handleSourceEvents(sourceEvent);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private void handleBinlogSplitUpdateRequest() {
|
|
|
|
|
mySqlSourceReaderContext.suspendBinlogSplitReader();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private void updateBinlogSplit(LatestFinishedSplitsNumberEvent sourceEvent) {
|
|
|
|
|
if (suspendedBinlogSplit != null) {
|
|
|
|
|
final int finishedSplitsSize = sourceEvent.getLatestFinishedSplitsNumber();
|
|
|
|
|
final MySqlBinlogSplit binlogSplit =
|
|
|
|
|
toNormalBinlogSplit(suspendedBinlogSplit, finishedSplitsSize);
|
|
|
|
|
suspendedBinlogSplit = null;
|
|
|
|
|
this.addSplits(Collections.singletonList(binlogSplit));
|
|
|
|
|
|
|
|
|
|
context.sendSourceEventToCoordinator(new BinlogSplitUpdateAckEvent());
|
|
|
|
|
LOG.info("Notify enumerator that binlog split has been updated.");
|
|
|
|
|
|
|
|
|
|
mySqlSourceReaderContext.wakeupSuspendedBinlogSplitReader();
|
|
|
|
|
LOG.info("Wake up suspended binlog reader as binlog split has been updated.");
|
|
|
|
|
} else {
|
|
|
|
|
LOG.warn("Unexpected event {}, this should not happen.", sourceEvent);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private void reportFinishedSnapshotSplitsIfNeed() {
|
|
|
|
|
if (!finishedUnackedSplits.isEmpty()) {
|
|
|
|
|
final Map<String, BinlogOffset> finishedOffsets = new HashMap<>();
|
|
|
|
@ -308,7 +324,7 @@ public class MySqlSourceReader<T>
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private void fillMetaDataForBinlogSplit(BinlogSplitMetaEvent metadataEvent) {
|
|
|
|
|
private void fillMetadataForBinlogSplit(BinlogSplitMetaEvent metadataEvent) {
|
|
|
|
|
MySqlBinlogSplit binlogSplit = uncompletedBinlogSplits.get(metadataEvent.getSplitId());
|
|
|
|
|
if (binlogSplit != null) {
|
|
|
|
|
final int receivedMetaGroupId = metadataEvent.getMetaGroupId();
|
|
|
|
@ -317,15 +333,22 @@ public class MySqlSourceReader<T>
|
|
|
|
|
binlogSplit.getFinishedSnapshotSplitInfos().size(),
|
|
|
|
|
sourceConfig.getSplitMetaGroupSize());
|
|
|
|
|
if (receivedMetaGroupId == expectedMetaGroupId) {
|
|
|
|
|
List<FinishedSnapshotSplitInfo> metaDataGroup =
|
|
|
|
|
List<FinishedSnapshotSplitInfo> newAddedMetadataGroup;
|
|
|
|
|
Set<String> existedSplitsOfLastGroup =
|
|
|
|
|
getExistedSplitsOfLastGroup(
|
|
|
|
|
binlogSplit.getFinishedSnapshotSplitInfos(),
|
|
|
|
|
sourceConfig.getSplitMetaGroupSize());
|
|
|
|
|
newAddedMetadataGroup =
|
|
|
|
|
metadataEvent.getMetaGroup().stream()
|
|
|
|
|
.map(FinishedSnapshotSplitInfo::deserialize)
|
|
|
|
|
.filter(r -> !existedSplitsOfLastGroup.contains(r.getSplitId()))
|
|
|
|
|
.collect(Collectors.toList());
|
|
|
|
|
|
|
|
|
|
uncompletedBinlogSplits.put(
|
|
|
|
|
binlogSplit.splitId(),
|
|
|
|
|
MySqlBinlogSplit.appendFinishedSplitInfos(binlogSplit, metaDataGroup));
|
|
|
|
|
|
|
|
|
|
LOG.info("Fill meta data of group {} to binlog split", metaDataGroup.size());
|
|
|
|
|
MySqlBinlogSplit.appendFinishedSplitInfos(
|
|
|
|
|
binlogSplit, newAddedMetadataGroup));
|
|
|
|
|
LOG.info("Fill metadata of group {} to binlog split", newAddedMetadataGroup.size());
|
|
|
|
|
} else {
|
|
|
|
|
LOG.warn(
|
|
|
|
|
"Received out of oder binlog meta event for split {}, the received meta group id is {}, but expected is {}, ignore it",
|
|
|
|
@ -341,6 +364,58 @@ public class MySqlSourceReader<T>
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private MySqlBinlogSplit discoverTableSchemasForBinlogSplit(
|
|
|
|
|
MySqlBinlogSplit split,
|
|
|
|
|
MySqlSourceConfig sourceConfig,
|
|
|
|
|
boolean checkNewlyAddedTableSchema) {
|
|
|
|
|
if (split.getTableSchemas().isEmpty() || checkNewlyAddedTableSchema) {
|
|
|
|
|
try (MySqlConnection jdbc = DebeziumUtils.createMySqlConnection(sourceConfig)) {
|
|
|
|
|
Map<TableId, TableChanges.TableChange> tableSchemas;
|
|
|
|
|
if (split.getTableSchemas().isEmpty()) {
|
|
|
|
|
tableSchemas =
|
|
|
|
|
TableDiscoveryUtils.discoverSchemaForCapturedTables(sourceConfig, jdbc);
|
|
|
|
|
LOG.info(
|
|
|
|
|
"The table schema discovery for binlog split {} success",
|
|
|
|
|
split.splitId());
|
|
|
|
|
} else {
|
|
|
|
|
List<TableId> existedTables = new ArrayList<>(split.getTableSchemas().keySet());
|
|
|
|
|
tableSchemas =
|
|
|
|
|
discoverSchemaForNewAddedTables(existedTables, sourceConfig, jdbc);
|
|
|
|
|
LOG.info(
|
|
|
|
|
"The table schema discovery for new added tables of binlog split {} success",
|
|
|
|
|
split.splitId());
|
|
|
|
|
}
|
|
|
|
|
return MySqlBinlogSplit.fillTableSchemas(split, tableSchemas);
|
|
|
|
|
} catch (SQLException e) {
|
|
|
|
|
LOG.error("Failed to obtains table schemas due to {}", e.getMessage());
|
|
|
|
|
throw new FlinkRuntimeException(e);
|
|
|
|
|
}
|
|
|
|
|
} else {
|
|
|
|
|
LOG.warn(
|
|
|
|
|
"The binlog split {} has table schemas yet, skip the table schema discovery",
|
|
|
|
|
split);
|
|
|
|
|
return split;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private Set<String> getExistedSplitsOfLastGroup(
|
|
|
|
|
List<FinishedSnapshotSplitInfo> finishedSnapshotSplits, int metaGroupSize) {
|
|
|
|
|
Set<String> existedSplitsOfLastGroup = new HashSet<>();
|
|
|
|
|
int splitsNumOfLastGroup =
|
|
|
|
|
finishedSnapshotSplits.size() % sourceConfig.getSplitMetaGroupSize();
|
|
|
|
|
if (splitsNumOfLastGroup != 0) {
|
|
|
|
|
int lastGroupStart =
|
|
|
|
|
((int) (finishedSnapshotSplits.size() / sourceConfig.getSplitMetaGroupSize()))
|
|
|
|
|
* metaGroupSize;
|
|
|
|
|
existedSplitsOfLastGroup =
|
|
|
|
|
finishedSnapshotSplits
|
|
|
|
|
.subList(lastGroupStart, lastGroupStart + splitsNumOfLastGroup).stream()
|
|
|
|
|
.map(FinishedSnapshotSplitInfo::getSplitId)
|
|
|
|
|
.collect(Collectors.toSet());
|
|
|
|
|
}
|
|
|
|
|
return existedSplitsOfLastGroup;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private void logCurrentBinlogOffsets(List<MySqlSplit> splits, long checkpointId) {
|
|
|
|
|
if (!LOG.isInfoEnabled()) {
|
|
|
|
|
return;
|
|
|
|
|