SML0127 commented on code in PR #3456: URL: https://github.com/apache/flink-cdc/pull/3456#discussion_r1677533767
########## 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 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. ```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