guozhangwang commented on code in PR #13300:
URL: https://github.com/apache/kafka/pull/13300#discussion_r1117803154


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java:
##########
@@ -767,20 +768,6 @@ public static void addInvocationRateToSensor(final Sensor 
sensor,
         );
     }
 
-    public static void addInvocationRateAndCountToSensor(final Sensor sensor,

Review Comment:
   This function is not used in prod code, hence cleaning it up.



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/TaskMetrics.java:
##########
@@ -213,7 +272,7 @@ public static Sensor recordLatenessSensor(final String 
threadId,
     public static Sensor droppedRecordsSensor(final String threadId,
                                               final String taskId,
                                               final StreamsMetricsImpl 
streamsMetrics) {
-        return invocationRateAndCountSensor(
+        return invocationRateAndTotalSensor(

Review Comment:
   This is a piggy-backed metric fix: we should use cumulativeSum than 
cumulativeCount for dropped records, even though today with most callees as 
`sensor.record()` it is effectively the same as it only increment by 1, it is 
still vulnerable in case we record a non-one value in the future.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/StateStoreMetrics.java:
##########
@@ -146,13 +144,6 @@ private StateStoreMetrics() {}
     private static final String SUPPRESSION_BUFFER_SIZE_MAX_DESCRIPTION =
         MAX_DESCRIPTION_PREFIX + SUPPRESSION_BUFFER_SIZE_DESCRIPTION;
 
-    private static final String EXPIRED_WINDOW_RECORD_DROP = 
"expired-window-record-drop";

Review Comment:
   This metric is removed as part of KIP-743, and it's only used in tests 
(which I also cleaned up as a piggy-back).



-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to