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 all commits
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 @@ -18,17 +18,42 @@
package org.apache.flink.cdc.connectors.mysql.source.metrics;

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.Map;
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_DATA_CHANGE_EVENT_INSERT =
".numRecordsOutByDataChangeEventInsert";

public static final String IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_DELETE =
".numRecordsOutByDataChangeEventDelete";

public static final String IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_UPDATE =
".numRecordsOutByDataChangeEventUpdate";
Comment on lines +38 to +47
Copy link
Contributor

Choose a reason for hiding this comment

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

I suggest to remove dot(.) from metric name.

Suggested change
public static final String IO_NUM_RECORDS_OUT_SNAPSHOT = ".numRecordsOutBySnapshot";
public static final String IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_INSERT =
".numRecordsOutByDataChangeEventInsert";
public static final String IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_DELETE =
".numRecordsOutByDataChangeEventDelete";
public static final String IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_UPDATE =
".numRecordsOutByDataChangeEventUpdate";
public static final String IO_NUM_RECORDS_OUT_SNAPSHOT = "numRecordsOutBySnapshot";
public static final String IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_INSERT =
"numRecordsOutByDataChangeEventInsert";
public static final String IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_DELETE =
"numRecordsOutByDataChangeEventDelete";
public static final String IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_UPDATE =
"numRecordsOutByDataChangeEventUpdate";

Copy link
Member Author

Choose a reason for hiding this comment

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

Ok, thank you for helping me review the code, I will remove (.)


private final Map<TableId, Counter> snapshotNumRecordsOutMap = new ConcurrentHashMap();

private final Map<TableId, Counter> insertNumRecordsOutMap = new ConcurrentHashMap();

private final Map<TableId, Counter> updateNumRecordsOutMap = new ConcurrentHashMap();

private final Map<TableId, Counter> deleteNumRecordsOutMap = new ConcurrentHashMap();

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

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

public void numRecordsOutInsertIncrease(TableId tableId) {
Counter counter =
insertNumRecordsOutMap.computeIfAbsent(
tableId,
k ->
metricGroup.counter(
tableId.identifier()
+ IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_INSERT));
counter.inc();
}

public void numRecordsOutUpdateIncrease(TableId tableId) {
Counter counter =
updateNumRecordsOutMap.computeIfAbsent(
tableId,
k ->
metricGroup.counter(
tableId.identifier()
+ IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_UPDATE));
counter.inc();
}

public void numRecordsOutDeleteIncrease(TableId tableId) {
Counter counter =
deleteNumRecordsOutMap.computeIfAbsent(
tableId,
k ->
metricGroup.counter(
tableId.identifier()
+ IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_DELETE));
counter.inc();
}

Comment on lines +72 to +114
Copy link
Contributor

@SML0127 SML0127 Jul 15, 2024

Choose a reason for hiding this comment

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

I applied this metrics to my code, and left some comments.

  • Metric names should be general.
  • In your case, It generate uniqe metrics for each tables. But in monitorig tool such as grafana, it distingush tables by property such as job name in metric, not by metric name
    So I suggest to remove tableId.identifier() in metric name.
Suggested change
public void numRecordsOutSnapshotIncrease(TableId tableId) {
Counter counter =
snapshotNumRecordsOutMap.computeIfAbsent(
tableId,
k ->
metricGroup.counter(
tableId.identifier() + IO_NUM_RECORDS_OUT_SNAPSHOT));
counter.inc();
}
public void numRecordsOutInsertIncrease(TableId tableId) {
Counter counter =
insertNumRecordsOutMap.computeIfAbsent(
tableId,
k ->
metricGroup.counter(
tableId.identifier()
+ IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_INSERT));
counter.inc();
}
public void numRecordsOutUpdateIncrease(TableId tableId) {
Counter counter =
updateNumRecordsOutMap.computeIfAbsent(
tableId,
k ->
metricGroup.counter(
tableId.identifier()
+ IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_UPDATE));
counter.inc();
}
public void numRecordsOutDeleteIncrease(TableId tableId) {
Counter counter =
deleteNumRecordsOutMap.computeIfAbsent(
tableId,
k ->
metricGroup.counter(
tableId.identifier()
+ IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_DELETE));
counter.inc();
}
public void numRecordsOutSnapshotIncrease(TableId tableId) {
Counter counter =
snapshotNumRecordsOutMap.computeIfAbsent(
tableId,
k ->
metricGroup.counter(IO_NUM_RECORDS_OUT_SNAPSHOT));
counter.inc();
}
public void numRecordsOutInsertIncrease(TableId tableId) {
Counter counter =
insertNumRecordsOutMap.computeIfAbsent(
tableId,
k ->
metricGroup.counter(IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_INSERT));
counter.inc();
}
public void numRecordsOutUpdateIncrease(TableId tableId) {
Counter counter =
updateNumRecordsOutMap.computeIfAbsent(
tableId,
k ->
metricGroup.counter(IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_UPDATE));
counter.inc();
}
public void numRecordsOutDeleteIncrease(TableId tableId) {
Counter counter =
deleteNumRecordsOutMap.computeIfAbsent(
tableId,
k ->
metricGroup.counter(IO_NUM_RECORDS_OUT_DATA_CHANGE_EVENT_DELETE));
counter.inc();
}

Copy link
Member Author

Choose a reason for hiding this comment

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

Ok, thanks for reviewing the code for me, but the original purpose of this metric is to see each table at a granular level, if you need to see the total, I will add other metrics

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,28 @@ 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()) {
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;
}
}
}

Long messageTimestamp = RecordUtils.getMessageTimestamp(element);

Expand Down
Loading