SML0127 commented on code in PR #3456: URL: https://github.com/apache/flink-cdc/pull/3456#discussion_r1677535110
########## flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/metrics/MySqlSourceReaderMetrics.java: ########## @@ -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"; Review Comment: I suggest to remove dot(.) from metric name. ```suggestion 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"; ``` ########## flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/metrics/MySqlSourceReaderMetrics.java: ########## @@ -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(); + } + Review Comment: I applied this metrics my code, and left some comments. - Metric names should be general. - In your case, It generate uniqe metric for each tables. But in monitorig tool such as grafana, we distingush tables by property in metric, not by mtric name So I suggest to remove tableId. ```suggestion 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(); } ``` -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org