Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[cdc-connector][mysql]Mysql add numRecordsOutBySnapshot and numRecordsOutByIncremental metrics #3456

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
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 @@ -17,18 +17,37 @@

package org.apache.flink.cdc.connectors.mysql.source.metrics;

import org.apache.flink.cdc.common.event.OperationType;
import org.apache.flink.cdc.connectors.mysql.source.reader.MySqlSourceReader;
import org.apache.flink.metrics.Counter;
import org.apache.flink.metrics.Gauge;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.runtime.metrics.MetricNames;

import io.debezium.relational.TableId;

import java.util.concurrent.ConcurrentHashMap;

/** A collection class for handling metrics in {@link MySqlSourceReader}. */
public class MySqlSourceReaderMetrics {

public static final long UNDEFINED = -1;

private final MetricGroup metricGroup;

public static final String IO_NUM_RECORDS_OUT_SNAPSHOT = ".numRecordsOutBySnapshot";

public static final String IO_NUM_RECORDS_OUT_INCREMENTAL_INSERT =
".numRecordsOutByIncrementalInsert";

public static final String IO_NUM_RECORDS_OUT_INCREMENTAL_DELETE =
".numRecordsOutByIncrementalDelete";

public static final String IO_NUM_RECORDS_OUT_INCREMENTAL_UPDATE =
".numRecordsOutByIncrementalUpdate";
Copy link
Contributor

@SML0127 SML0127 Jul 12, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thx for contirbution @ChengJie1053

Nit: How about changing INCREMENTAL to BINLOG or CHANGE_EVENT? INCREMENTAL can be understood as a Incremental Snapshot step.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think your suggestion is great, and I will CHANGE it to CHANGE EVENT


private final ConcurrentHashMap<String, Counter> numRecordsOutMap = new ConcurrentHashMap();

/**
* currentFetchEventTimeLag = FetchTime - messageTimestamp, where the FetchTime is the time the
* record fetched into the source operator.
Expand All @@ -44,6 +63,49 @@ public void registerMetrics() {
MetricNames.CURRENT_FETCH_EVENT_TIME_LAG, (Gauge<Long>) this::getFetchDelay);
}

public void numRecordsOutSnapshotIncrease(TableId tableId) {
Counter counter =
numRecordsOutMap.computeIfAbsent(
tableId.identifier(),
k ->
metricGroup.counter(
tableId.identifier() + IO_NUM_RECORDS_OUT_SNAPSHOT));
counter.inc();
}

public void numRecordsOutInsertIncrease(TableId tableId) {
Counter counter =
numRecordsOutMap.computeIfAbsent(
tableId.identifier() + OperationType.INSERT,
k ->
metricGroup.counter(
tableId.identifier()
+ IO_NUM_RECORDS_OUT_INCREMENTAL_INSERT));
counter.inc();
}

public void numRecordsOutUpdateIncrease(TableId tableId) {
Counter counter =
numRecordsOutMap.computeIfAbsent(
tableId.identifier() + OperationType.UPDATE,
k ->
metricGroup.counter(
tableId.identifier()
+ IO_NUM_RECORDS_OUT_INCREMENTAL_UPDATE));
counter.inc();
}

public void numRecordsOutDeleteIncrease(TableId tableId) {
Counter counter =
numRecordsOutMap.computeIfAbsent(
tableId.identifier() + OperationType.DELETE,
k ->
metricGroup.counter(
tableId.identifier()
+ IO_NUM_RECORDS_OUT_INCREMENTAL_DELETE));
counter.inc();
}

public long getFetchDelay() {
return fetchDelay;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,12 @@
import org.apache.flink.connector.base.source.reader.RecordEmitter;
import org.apache.flink.util.Collector;

import io.debezium.data.Envelope;
import io.debezium.document.Array;
import io.debezium.relational.TableId;
import io.debezium.relational.history.HistoryRecord;
import io.debezium.relational.history.TableChanges;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceRecord;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -97,7 +100,7 @@ protected void processElement(
}
} else if (RecordUtils.isDataChangeRecord(element)) {
updateStartingOffsetForSplit(splitState, element);
reportMetrics(element);
reportMetrics(element, splitState);
emitElement(element, output);
} else if (RecordUtils.isHeartbeatEvent(element)) {
updateStartingOffsetForSplit(splitState, element);
Expand All @@ -120,7 +123,31 @@ private void emitElement(SourceRecord element, SourceOutput<T> output) throws Ex
debeziumDeserializationSchema.deserialize(element, outputCollector);
}

private void reportMetrics(SourceRecord element) {
private void reportMetrics(SourceRecord element, MySqlSplitState splitState) {
TableId tableId = RecordUtils.getTableId(element);
if (splitState.isSnapshotSplitState()) {
sourceReaderMetrics.numRecordsOutSnapshotIncrease(tableId);
} else if (splitState.isBinlogSplitState()) {
if (RecordUtils.isDataChangeRecord(element)) {
Struct value = (Struct) element.value();
if (value != null) {
Envelope.Operation operation =
Envelope.Operation.forCode(
value.getString(Envelope.FieldName.OPERATION));
switch (operation) {
case CREATE:
sourceReaderMetrics.numRecordsOutInsertIncrease(tableId);
break;
case UPDATE:
sourceReaderMetrics.numRecordsOutUpdateIncrease(tableId);
break;
case DELETE:
sourceReaderMetrics.numRecordsOutDeleteIncrease(tableId);
break;
}
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We checked whether element is data change record or not in line 101.
How about removing this condition?

Suggested change
if (RecordUtils.isDataChangeRecord(element)) {
Struct value = (Struct) element.value();
if (value != null) {
Envelope.Operation operation =
Envelope.Operation.forCode(
value.getString(Envelope.FieldName.OPERATION));
switch (operation) {
case CREATE:
sourceReaderMetrics.numRecordsOutInsertIncrease(tableId);
break;
case UPDATE:
sourceReaderMetrics.numRecordsOutUpdateIncrease(tableId);
break;
case DELETE:
sourceReaderMetrics.numRecordsOutDeleteIncrease(tableId);
break;
}
}
}
Struct value = (Struct) element.value();
if (value != null) {
Envelope.Operation operation =
Envelope.Operation.forCode(
value.getString(Envelope.FieldName.OPERATION));
switch (operation) {
case CREATE:
sourceReaderMetrics.numRecordsOutInsertIncrease(tableId);
break;
case UPDATE:
sourceReaderMetrics.numRecordsOutUpdateIncrease(tableId);
break;
case DELETE:
sourceReaderMetrics.numRecordsOutDeleteIncrease(tableId);
break;
}
}

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you very much for helping me review the code.
I will delete the isDataChangeRecord judgment

}

Long messageTimestamp = RecordUtils.getMessageTimestamp(element);

Expand Down
Loading