Skip to content

Commit

Permalink
[FLINK-36315][cdc-base]The flink-cdc-base module supports source metr…
Browse files Browse the repository at this point in the history
…ic statistics
  • Loading branch information
molin.lxd committed Sep 23, 2024
1 parent a5b666a commit 9d94c8a
Show file tree
Hide file tree
Showing 20 changed files with 1,158 additions and 51 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,39 @@ 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());

LOG.info("HybridSplitAssigner open, isStreamSplitAssigned: {}", isStreamSplitAssigned);
if (isStreamSplitAssigned) {
enumeratorMetrics.enterStreamReading();
} else {
enumeratorMetrics.exitStreamReading();
}

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

@Override
Expand All @@ -126,6 +149,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 +161,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 Down
Loading

0 comments on commit 9d94c8a

Please sign in to comment.