ruanhang1993 commented on code in PR #3619:
URL: https://github.com/apache/flink-cdc/pull/3619#discussion_r1816066674
##########
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/HybridSplitAssigner.java:
##########
@@ -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);
Review Comment:
This log is useless.
##########
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java:
##########
@@ -352,18 +425,41 @@ public void onFinishedSplits(Map<String, Offset>
splitFinishedOffsets) {
@Override
public void addSplits(Collection<SourceSplitBase> splits) {
+ enumeratorMetrics.exitStreamReading();
for (SourceSplitBase split : splits) {
tableSchemas.putAll(split.asSnapshotSplit().getTableSchemas());
remainingSplits.add(split.asSnapshotSplit().toSchemalessSnapshotSplit());
// we should remove the add-backed splits from the assigned list,
// 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();
+ if (finishedSplits.containsKey(tableId)) {
+ finishedSplits.get(tableId).remove(split.splitId());
+
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(
Review Comment:
```suggestion
LOG.info("SnapshotSplitAssigner snapshotState on checkpoint {} with
splitFinishedCheckpointIds size {}, finishedSplits size {}.",
checkpointId,
splitFinishedCheckpointIds == null ? 0 :
splitFinishedCheckpointIds.size(),
finishedSplits == null ? 0 : finishedSplits.size());
```
##########
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/metrics/SourceEnumeratorMetrics.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.base.source.metrics;
+
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.SplitEnumeratorMetricGroup;
+
+import io.debezium.relational.TableId;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/** A collection class for handling metrics in {@link
SourceEnumeratorMetrics}. */
+public class SourceEnumeratorMetrics {
+ private static final Logger LOGGER =
LoggerFactory.getLogger(SourceEnumeratorMetrics.class);
+ // Constants
+ public static final int UNDEFINED = 0;
+
+ // Metric names
+ public static final String IS_SNAPSHOTTING = "isSnapshotting";
+ public static final String IS_STREAM_READING = "isStreamReading";
+ public static final String NUM_TABLES_SNAPSHOTTED = "numTablesSnapshotted";
+ public static final String NUM_TABLES_REMAINING = "numTablesRemaining";
+ public static final String NUM_SNAPSHOT_SPLITS_PROCESSED =
"numSnapshotSplitsProcessed";
+ public static final String NUM_SNAPSHOT_SPLITS_REMAINING =
"numSnapshotSplitsRemaining";
+ public static final String NUM_SNAPSHOT_SPLITS_FINISHED =
"numSnapshotSplitsFinished";
+ public static final String SNAPSHOT_START_TIME = "snapshotStartTime";
+ public static final String SNAPSHOT_END_TIME = "snapshotEndTime";
+ public static final String NAMESPACE_GROUP_KEY = "namespace";
+ public static final String SCHEMA_GROUP_KEY = "schema";
+ public static final String TABLE_GROUP_KEY = "table";
+
+ private final SplitEnumeratorMetricGroup metricGroup;
+
+ private volatile int isSnapshotting = UNDEFINED;
+ private volatile int isStreamReading = UNDEFINED;
+ private volatile int numTablesRemaining = 0;
+
+ // Map for managing per-table metrics by table identifier
+ // Key: Identifier of the table
+ // Value: TableMetrics related to the table
+ private final Map<TableId, TableMetrics> tableMetricsMap = new
ConcurrentHashMap<>();
+
+ public SourceEnumeratorMetrics(SplitEnumeratorMetricGroup metricGroup) {
+ this.metricGroup = metricGroup;
+ metricGroup.gauge(IS_SNAPSHOTTING, () -> isSnapshotting);
+ metricGroup.gauge(IS_STREAM_READING, () -> isStreamReading);
+ metricGroup.gauge(NUM_TABLES_REMAINING, () -> numTablesRemaining);
+ }
+
+ public void enterSnapshotPhase() {
+ this.isSnapshotting = 1;
+ }
+
+ public void exitSnapshotPhase() {
+ this.isSnapshotting = 0;
+ }
+
+ public void enterStreamReading() {
+ this.isStreamReading = 1;
+ }
+
+ public void exitStreamReading() {
+ this.isStreamReading = 0;
+ }
+
+ public void registerMetrics(
+ Gauge<Integer> numTablesSnapshotted,
+ Gauge<Integer> numSnapshotSplitsProcessed,
+ Gauge<Integer> numSnapshotSplitsRemaining) {
+ metricGroup.gauge(NUM_TABLES_SNAPSHOTTED, numTablesSnapshotted);
+ metricGroup.gauge(NUM_SNAPSHOT_SPLITS_PROCESSED,
numSnapshotSplitsProcessed);
+ metricGroup.gauge(NUM_SNAPSHOT_SPLITS_REMAINING,
numSnapshotSplitsRemaining);
+ }
+
+ public void addNewTables(int numNewTables) {
+ numTablesRemaining += numNewTables;
+ }
+
+ public void startSnapshotTables(int numSnapshottedTables) {
+ numTablesRemaining -= numSnapshottedTables;
+ }
+
+ public TableMetrics getTableMetrics(TableId tableId) {
+ return tableMetricsMap.computeIfAbsent(
+ tableId,
+ key -> new TableMetrics(key.catalog(), key.schema(),
key.table(), metricGroup));
+ }
+
+ // ----------------------------------- Helper classes
--------------------------------
+
+ /**
+ * Collection class for managing metrics of a table.
+ *
+ * <p>Metrics of table level are registered in its corresponding subgroup
under the {@link
+ * SplitEnumeratorMetricGroup}.
+ */
+ public static class TableMetrics {
+ private AtomicInteger numSnapshotSplitsProcessed = new
AtomicInteger(0);
+ private AtomicInteger numSnapshotSplitsRemaining = new
AtomicInteger(0);
+ private AtomicInteger numSnapshotSplitsFinished = new AtomicInteger(0);
+ private volatile long snapshotStartTime = UNDEFINED;
+ private volatile long snapshotEndTime = UNDEFINED;
+
+ private Set<String> remainingSplitIds =
+ Collections.newSetFromMap(new ConcurrentHashMap<>());
+ private Set<String> processedSplitIds =
+ Collections.newSetFromMap(new ConcurrentHashMap<>());
+ private Set<String> finishedSplitIds = Collections.newSetFromMap(new
ConcurrentHashMap<>());
Review Comment:
I think we don't need these fields.
- There is no metric registered about `remainingSplitIds` and
`processedSplitIds`
- `finishedSplitIds` could be stored in the assigner instead of this metric
class
##########
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/HybridSplitAssigner.java:
##########
@@ -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();
Review Comment:
> > `enumeratorMetrics.exitStreamReading()` should be invoked in `addSplits`.
>
> Inside `snapshotSplitAssigner.addSplits`, it will handle
We should not handle this in snapshot split assigner.
It is possible to add an empty list to it.
##########
flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java:
##########
@@ -397,6 +495,30 @@ && 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.parseTableId(splitId);
+
enumeratorMetrics.getTableMetrics(tableId).addFinishedSplit(splitId);
+ finishedSplits.put(
+ tableId,
+
enumeratorMetrics.getTableMetrics(tableId).getFinishedSplitIds());
+ iterator.remove();
+ }
+ }
+ LOG.info(
Review Comment:
```suggestion
LOG.info(
"Checkpoint completed on checkpoint {} with
splitFinishedCheckpointIds size {}, finishedSplits size {}.",
checkpointId,
splitFinishedCheckpointIds == null ? 0 :
splitFinishedCheckpointIds.size(),
finishedSplits == null ? 0 : finishedSplits.size());
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]