Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
38 commits
Select commit Hold shift + click to select a range
9d94c8a
[FLINK-36315][cdc-base]The flink-cdc-base module supports source metr…
Sep 23, 2024
6c00324
[FLINK-36315][cdc-base]Fix code style issues
Oct 8, 2024
d5c663e
[FLINK-36315][cdc-base]add enterStreamReading while isNewlyAddedAssig…
Oct 8, 2024
9c36e28
[FLINK-36315][cdc-base]remove unused variable currentReadTimestampMs
Oct 9, 2024
84ff54e
[FLINK-36315][cdc-base]Variable database changed to namespace
Oct 9, 2024
f344746
[FLINK-36315][cdc-base]Variable database changed to namespace
Oct 9, 2024
4b97a54
Update flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/s…
liuxiao2shf Oct 28, 2024
3b29823
[FLINK-36315][cdc-base]Adjust log format
Oct 28, 2024
755f6b9
[FLINK-36315][cdc-base]Remove the finishedSplits variable from the Sn…
Oct 31, 2024
99755d3
[FLINK-36315][cdc-base]Update MySQL Source Metric UTs
Nov 5, 2024
404172f
[FLINK-36315][cdc-base]Add MySQL Source Metric UTs for DDL
Nov 6, 2024
eb3c306
[FLINK-36315][cdc-base]Roll back some unrelated UT changes
Nov 8, 2024
3ca4488
[FLINK-36315][cdc-base]Roll back some unrelated UT changes
Nov 12, 2024
25fb8ac
[FLINK-36315][cdc-base]Roll back some unrelated UT changes
Nov 12, 2024
cb0f4bf
[FLINK-36315][cdc-base]The flink-cdc-base module supports source metr…
Sep 23, 2024
ed45a3e
[FLINK-36315][cdc-base]Fix code style issues
Oct 8, 2024
de656f7
[FLINK-36315][cdc-base]add enterStreamReading while isNewlyAddedAssig…
Oct 8, 2024
eb9f75d
[FLINK-36315][cdc-base]remove unused variable currentReadTimestampMs
Oct 9, 2024
d4d1d45
[FLINK-36315][cdc-base]Variable database changed to namespace
Oct 9, 2024
d970e0a
[FLINK-36315][cdc-base]Variable database changed to namespace
Oct 9, 2024
d45445a
Update flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/s…
liuxiao2shf Oct 28, 2024
e5aa8f2
[FLINK-36315][cdc-base]Adjust log format
Oct 28, 2024
9eac032
[FLINK-36315][cdc-base]Remove the finishedSplits variable from the Sn…
Oct 31, 2024
eb7d4ec
[FLINK-36315][cdc-base]Update MySQL Source Metric UTs
Nov 5, 2024
db4eaee
[FLINK-36315][cdc-base]Add MySQL Source Metric UTs for DDL
Nov 6, 2024
9669ef3
[FLINK-36315][cdc-base]Roll back some unrelated UT changes
Nov 8, 2024
37dba3b
[FLINK-36315][cdc-base]Roll back some unrelated UT changes
Nov 12, 2024
fbd543a
[FLINK-36315][cdc-base]Roll back some unrelated UT changes
Nov 12, 2024
10ce78b
Merge remote-tracking branch 'origin/cdc_base_support_metrics_36315' …
Nov 13, 2024
6ab30bf
Merge branch 'master' into cdc_base_support_metrics_36315
Nov 19, 2024
b87410b
[FLINK-36315][cdc-base]PendingSplitsStateSerializer UT support version 7
Nov 22, 2024
ce2f838
[FLINK-36315][cdc-base]PendingSplitsStateSerializer UT support version 7
Nov 22, 2024
c1413a7
[FLINK-36315][cdc-base]PendingSplitsStateSerializer UT support version 7
Nov 22, 2024
97ab34a
[FLINK-36315][cdc-base]PendingSplitsStateSerializer UT support version 7
Nov 22, 2024
518de73
[FLINK-36315][cdc-base]Fix code style issues
Nov 22, 2024
4c54afc
[FLINK-36315][cdc-base]Merge branch 'master' into cdc_base_support_me…
Dec 2, 2024
9966018
[FLINK-36315][cdc-base]Adjust the key of TableMetrics from splitId to…
Dec 2, 2024
6ca265f
[FLINK-36315][cdc-base]Adjust the key of TableMetrics from splitId to…
Dec 5, 2024
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 @@ -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
Loading