Skip to content

Commit

Permalink
[FLINK-36315][cdc-connector][base&pg&mongodb]The flink-cdc-base modul…
Browse files Browse the repository at this point in the history
…e supports source metric statistics (#3619)


Co-authored-by: molin.lxd <[email protected]>
Co-authored-by: Hang Ruan <[email protected]>
  • Loading branch information
3 people authored Dec 6, 2024
1 parent 26f5880 commit ee9cd82
Show file tree
Hide file tree
Showing 24 changed files with 2,932 additions and 45 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,6 @@ public IncrementalSourceReader<T, C> createReader(SourceReaderContext readerCont
final SourceReaderMetrics sourceReaderMetrics =
new SourceReaderMetrics(readerContext.metricGroup());

sourceReaderMetrics.registerMetrics();
IncrementalSourceReaderContext incrementalSourceReaderContext =
new IncrementalSourceReaderContext(readerContext);
Supplier<IncrementalSourceSplitReader<C>> splitReaderSupplier =
Expand Down Expand Up @@ -161,13 +160,16 @@ public SplitEnumerator<SourceSplitBase, PendingSplitsState> createEnumerator(
remainingTables,
isTableIdCaseSensitive,
dataSourceDialect,
offsetFactory);
offsetFactory,
enumContext);
} catch (Exception e) {
throw new FlinkRuntimeException(
"Failed to discover captured tables for enumerator", e);
}
} else {
splitAssigner = new StreamSplitAssigner(sourceConfig, dataSourceDialect, offsetFactory);
splitAssigner =
new StreamSplitAssigner(
sourceConfig, dataSourceDialect, offsetFactory, enumContext);
}

return new IncrementalSourceEnumerator(
Expand All @@ -187,14 +189,16 @@ public SplitEnumerator<SourceSplitBase, PendingSplitsState> restoreEnumerator(
enumContext.currentParallelism(),
(HybridPendingSplitsState) checkpoint,
dataSourceDialect,
offsetFactory);
offsetFactory,
enumContext);
} else if (checkpoint instanceof StreamPendingSplitsState) {
splitAssigner =
new StreamSplitAssigner(
sourceConfig,
(StreamPendingSplitsState) checkpoint,
dataSourceDialect,
offsetFactory);
offsetFactory,
enumContext);
} else {
throw new UnsupportedOperationException(
"Unsupported restored PendingSplitsState: " + checkpoint);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.flink.cdc.connectors.base.source.assigner;

import org.apache.flink.api.connector.source.SourceSplit;
import org.apache.flink.api.connector.source.SplitEnumeratorContext;
import org.apache.flink.cdc.connectors.base.config.SourceConfig;
import org.apache.flink.cdc.connectors.base.dialect.DataSourceDialect;
import org.apache.flink.cdc.connectors.base.source.assigner.state.HybridPendingSplitsState;
Expand All @@ -27,6 +29,7 @@
import org.apache.flink.cdc.connectors.base.source.meta.split.SchemalessSnapshotSplit;
import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitBase;
import org.apache.flink.cdc.connectors.base.source.meta.split.StreamSplit;
import org.apache.flink.cdc.connectors.base.source.metrics.SourceEnumeratorMetrics;

import io.debezium.relational.TableId;
import org.slf4j.Logger;
Expand Down Expand Up @@ -61,13 +64,17 @@ public class HybridSplitAssigner<C extends SourceConfig> implements SplitAssigne

private final OffsetFactory offsetFactory;

private final SplitEnumeratorContext<? extends SourceSplit> enumeratorContext;
private SourceEnumeratorMetrics enumeratorMetrics;

public HybridSplitAssigner(
C sourceConfig,
int currentParallelism,
List<TableId> remainingTables,
boolean isTableIdCaseSensitive,
DataSourceDialect<C> dialect,
OffsetFactory offsetFactory) {
OffsetFactory offsetFactory,
SplitEnumeratorContext<? extends SourceSplit> enumeratorContext) {
this(
sourceConfig,
new SnapshotSplitAssigner<>(
Expand All @@ -79,15 +86,17 @@ public HybridSplitAssigner(
offsetFactory),
false,
sourceConfig.getSplitMetaGroupSize(),
offsetFactory);
offsetFactory,
enumeratorContext);
}

public HybridSplitAssigner(
C sourceConfig,
int currentParallelism,
HybridPendingSplitsState checkpoint,
DataSourceDialect<C> dialect,
OffsetFactory offsetFactory) {
OffsetFactory offsetFactory,
SplitEnumeratorContext<? extends SourceSplit> enumeratorContext) {
this(
sourceConfig,
new SnapshotSplitAssigner<>(
Expand All @@ -98,25 +107,38 @@ public HybridSplitAssigner(
offsetFactory),
checkpoint.isStreamSplitAssigned(),
sourceConfig.getSplitMetaGroupSize(),
offsetFactory);
offsetFactory,
enumeratorContext);
}

private HybridSplitAssigner(
C sourceConfig,
SnapshotSplitAssigner<C> snapshotSplitAssigner,
boolean isStreamSplitAssigned,
int splitMetaGroupSize,
OffsetFactory offsetFactory) {
OffsetFactory offsetFactory,
SplitEnumeratorContext<? extends SourceSplit> enumeratorContext) {
this.sourceConfig = sourceConfig;
this.snapshotSplitAssigner = snapshotSplitAssigner;
this.isStreamSplitAssigned = isStreamSplitAssigned;
this.splitMetaGroupSize = splitMetaGroupSize;
this.offsetFactory = offsetFactory;
this.enumeratorContext = enumeratorContext;
}

@Override
public void open() {
this.enumeratorMetrics = new SourceEnumeratorMetrics(enumeratorContext.metricGroup());

if (isStreamSplitAssigned) {
enumeratorMetrics.enterStreamReading();
} else {
enumeratorMetrics.exitStreamReading();
}

snapshotSplitAssigner.open();
// init enumerator metrics
snapshotSplitAssigner.initEnumeratorMetrics(enumeratorMetrics);
}

@Override
Expand All @@ -126,6 +148,7 @@ public Optional<SourceSplitBase> getNext() {
return Optional.empty();
}
if (snapshotSplitAssigner.noMoreSplits()) {
enumeratorMetrics.exitSnapshotPhase();
// stream split assigning
if (isStreamSplitAssigned) {
// no more splits for the assigner
Expand All @@ -137,6 +160,7 @@ public Optional<SourceSplitBase> getNext() {
// assigning the stream split. Otherwise, records emitted from stream split
// might be out-of-order in terms of same primary key with snapshot splits.
isStreamSplitAssigned = true;
enumeratorMetrics.enterStreamReading();
StreamSplit streamSplit = createStreamSplit();
LOG.trace(
"SnapshotSplitAssigner is finished: creating a new stream split {}",
Expand All @@ -145,6 +169,7 @@ public Optional<SourceSplitBase> getNext() {
} else if (isNewlyAddedAssigningFinished(snapshotSplitAssigner.getAssignerStatus())) {
// do not need to create stream split, but send event to wake up the binlog reader
isStreamSplitAssigned = true;
enumeratorMetrics.enterStreamReading();
return Optional.empty();
} else {
// stream split is not ready by now
Expand Down Expand Up @@ -184,6 +209,9 @@ public void addSplits(Collection<SourceSplitBase> splits) {
isStreamSplitAssigned = false;
}
}
if (!snapshotSplits.isEmpty()) {
enumeratorMetrics.exitStreamReading();
}
snapshotSplitAssigner.addSplits(snapshotSplits);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.flink.cdc.connectors.base.source.meta.split.SchemalessSnapshotSplit;
import org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit;
import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitBase;
import org.apache.flink.cdc.connectors.base.source.metrics.SourceEnumeratorMetrics;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.Preconditions;

Expand All @@ -49,6 +50,7 @@
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;

import static org.apache.flink.cdc.connectors.base.source.assigner.AssignerStatus.INITIAL_ASSIGNING;
Expand Down Expand Up @@ -81,6 +83,10 @@ public class SnapshotSplitAssigner<C extends SourceConfig> implements SplitAssig
private final DataSourceDialect<C> dialect;
private final OffsetFactory offsetFactory;

private SourceEnumeratorMetrics enumeratorMetrics;
private final Map<String, Long> splitFinishedCheckpointIds;
private static final long UNDEFINED_CHECKPOINT_ID = -1;

public SnapshotSplitAssigner(
C sourceConfig,
int currentParallelism,
Expand All @@ -101,7 +107,8 @@ public SnapshotSplitAssigner(
isTableIdCaseSensitive,
true,
dialect,
offsetFactory);
offsetFactory,
new ConcurrentHashMap<>());
}

public SnapshotSplitAssigner(
Expand All @@ -123,7 +130,8 @@ public SnapshotSplitAssigner(
checkpoint.isTableIdCaseSensitive(),
checkpoint.isRemainingTablesCheckpointed(),
dialect,
offsetFactory);
offsetFactory,
new ConcurrentHashMap<>());
}

private SnapshotSplitAssigner(
Expand All @@ -139,7 +147,8 @@ private SnapshotSplitAssigner(
boolean isTableIdCaseSensitive,
boolean isRemainingTablesCheckpointed,
DataSourceDialect<C> dialect,
OffsetFactory offsetFactory) {
OffsetFactory offsetFactory,
Map<String, Long> splitFinishedCheckpointIds) {
this.sourceConfig = sourceConfig;
this.currentParallelism = currentParallelism;
this.alreadyProcessedTables = alreadyProcessedTables;
Expand All @@ -163,6 +172,7 @@ private SnapshotSplitAssigner(
this.isTableIdCaseSensitive = isTableIdCaseSensitive;
this.dialect = dialect;
this.offsetFactory = offsetFactory;
this.splitFinishedCheckpointIds = splitFinishedCheckpointIds;
}

@Override
Expand Down Expand Up @@ -269,6 +279,46 @@ private void captureNewlyAddedTables() {
}
}

/** This should be invoked after this class's open method. */
public void initEnumeratorMetrics(SourceEnumeratorMetrics enumeratorMetrics) {
this.enumeratorMetrics = enumeratorMetrics;

this.enumeratorMetrics.enterSnapshotPhase();
this.enumeratorMetrics.registerMetrics(
alreadyProcessedTables::size, assignedSplits::size, remainingSplits::size);
this.enumeratorMetrics.addNewTables(computeTablesPendingSnapshot());
for (SchemalessSnapshotSplit snapshotSplit : remainingSplits) {
this.enumeratorMetrics
.getTableMetrics(snapshotSplit.getTableId())
.addNewSplit(snapshotSplit.splitId());
}
for (SchemalessSnapshotSplit snapshotSplit : assignedSplits.values()) {
this.enumeratorMetrics
.getTableMetrics(snapshotSplit.getTableId())
.addProcessedSplit(snapshotSplit.splitId());
}
for (String splitId : splitFinishedOffsets.keySet()) {
TableId tableId = SnapshotSplit.extractTableId(splitId);
this.enumeratorMetrics.getTableMetrics(tableId).addFinishedSplit(splitId);
}
}

// remainingTables + tables has been split but not processed
private int computeTablesPendingSnapshot() {
int numTablesPendingSnapshot = remainingTables.size();
Set<TableId> computedTables = new HashSet<>();
for (SchemalessSnapshotSplit split : remainingSplits) {
TableId tableId = split.getTableId();
if (!computedTables.contains(tableId)
&& !alreadyProcessedTables.contains(tableId)
&& !remainingTables.contains(tableId)) {
computedTables.add(tableId);
numTablesPendingSnapshot++;
}
}
return numTablesPendingSnapshot;
}

@Override
public Optional<SourceSplitBase> getNext() {
if (!remainingSplits.isEmpty()) {
Expand All @@ -277,6 +327,9 @@ public Optional<SourceSplitBase> getNext() {
SchemalessSnapshotSplit split = iterator.next();
iterator.remove();
assignedSplits.put(split.splitId(), split);
enumeratorMetrics
.getTableMetrics(split.getTableId())
.finishProcessSplit(split.splitId());
return Optional.of(split.toSnapshotSplit(tableSchemas.get(split.getTableId())));
} else {
// it's turn for new table
Expand All @@ -294,7 +347,15 @@ public Optional<SourceSplitBase> getNext() {
.collect(Collectors.toList());
remainingSplits.addAll(schemalessSnapshotSplits);
tableSchemas.putAll(tableSchema);
if (!alreadyProcessedTables.contains(nextTable)) {
enumeratorMetrics.startSnapshotTables(1);
}
alreadyProcessedTables.add(nextTable);
List<String> splitIds =
schemalessSnapshotSplits.stream()
.map(SchemalessSnapshotSplit::splitId)
.collect(Collectors.toList());
enumeratorMetrics.getTableMetrics(nextTable).addNewSplits(splitIds);
return getNext();
} else {
return Optional.empty();
Expand Down Expand Up @@ -335,6 +396,12 @@ public List<FinishedSnapshotSplitInfo> getFinishedSplitInfos() {
@Override
public void onFinishedSplits(Map<String, Offset> splitFinishedOffsets) {
this.splitFinishedOffsets.putAll(splitFinishedOffsets);
for (String splitId : splitFinishedOffsets.keySet()) {
splitFinishedCheckpointIds.put(splitId, UNDEFINED_CHECKPOINT_ID);
}
LOG.info(
"splitFinishedCheckpointIds size in onFinishedSplits: {}",
splitFinishedCheckpointIds == null ? 0 : splitFinishedCheckpointIds.size());
if (allSnapshotSplitsFinished() && isAssigningSnapshotSplits(assignerStatus)) {
// Skip the waiting checkpoint when current parallelism is 1 which means we do not need
// to care about the global output data order of snapshot splits and stream split.
Expand All @@ -359,11 +426,31 @@ public void addSplits(Collection<SourceSplitBase> splits) {
// because they are failed
assignedSplits.remove(split.splitId());
splitFinishedOffsets.remove(split.splitId());

enumeratorMetrics
.getTableMetrics(split.asSnapshotSplit().getTableId())
.reprocessSplit(split.splitId());
TableId tableId = split.asSnapshotSplit().getTableId();

enumeratorMetrics.getTableMetrics(tableId).removeFinishedSplit(split.splitId());
}
}

@Override
public SnapshotPendingSplitsState snapshotState(long checkpointId) {
if (splitFinishedCheckpointIds != null && !splitFinishedCheckpointIds.isEmpty()) {
for (Map.Entry<String, Long> splitFinishedCheckpointId :
splitFinishedCheckpointIds.entrySet()) {
if (splitFinishedCheckpointId.getValue() == UNDEFINED_CHECKPOINT_ID) {
splitFinishedCheckpointId.setValue(checkpointId);
}
}
LOG.info(
"SnapshotSplitAssigner snapshotState on checkpoint {} with splitFinishedCheckpointIds size {}.",
checkpointId,
splitFinishedCheckpointIds.size());
}

SnapshotPendingSplitsState state =
new SnapshotPendingSplitsState(
alreadyProcessedTables,
Expand All @@ -374,7 +461,8 @@ public SnapshotPendingSplitsState snapshotState(long checkpointId) {
assignerStatus,
remainingTables,
isTableIdCaseSensitive,
true);
true,
splitFinishedCheckpointIds);
// we need a complete checkpoint before mark this assigner to be finished, to wait for all
// records of snapshot splits are completely processed
if (checkpointIdToFinish == null
Expand All @@ -397,6 +485,27 @@ && allSnapshotSplitsFinished()) {
}
LOG.info("Snapshot split assigner is turn into finished status.");
}

if (splitFinishedCheckpointIds != null && !splitFinishedCheckpointIds.isEmpty()) {
Iterator<Map.Entry<String, Long>> iterator =
splitFinishedCheckpointIds.entrySet().iterator();
while (iterator.hasNext()) {
Map.Entry<String, Long> splitFinishedCheckpointId = iterator.next();
String splitId = splitFinishedCheckpointId.getKey();
Long splitCheckpointId = splitFinishedCheckpointId.getValue();
if (splitCheckpointId != UNDEFINED_CHECKPOINT_ID
&& checkpointId >= splitCheckpointId) {
// record table-level splits metrics
TableId tableId = SnapshotSplit.extractTableId(splitId);
enumeratorMetrics.getTableMetrics(tableId).addFinishedSplit(splitId);
iterator.remove();
}
}
LOG.info(
"Checkpoint completed on checkpoint {} with splitFinishedCheckpointIds size {}.",
checkpointId,
splitFinishedCheckpointIds.size());
}
}

@Override
Expand Down
Loading

0 comments on commit ee9cd82

Please sign in to comment.