mirror of https://github.com/apache/kafka.git
				
				
				
			KAFKA-8859: Refactor cache-level metrics (#7367)
Cache-level metrics are refactor according to KIP-444: tag client-id changed to thread-id name hitRatio changed to hit-ratio made backward compatible by using streams config built.in.metrics.version Reviewers: Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bbejeck@gmail.com>
This commit is contained in:
		
							parent
							
								
									d91a94e7bf
								
							
						
					
					
						commit
						e98e239a0c
					
				|  | @ -25,6 +25,7 @@ import org.apache.kafka.common.metrics.stats.Avg; | |||
| import org.apache.kafka.common.metrics.stats.CumulativeCount; | ||||
| import org.apache.kafka.common.metrics.stats.CumulativeSum; | ||||
| import org.apache.kafka.common.metrics.stats.Max; | ||||
| import org.apache.kafka.common.metrics.stats.Min; | ||||
| import org.apache.kafka.common.metrics.stats.Rate; | ||||
| import org.apache.kafka.common.metrics.stats.Value; | ||||
| import org.apache.kafka.common.metrics.stats.WindowedCount; | ||||
|  | @ -63,9 +64,11 @@ public class StreamsMetricsImpl implements StreamsMetrics { | |||
|     private static final String SENSOR_PREFIX_DELIMITER = "."; | ||||
|     private static final String SENSOR_NAME_DELIMITER = ".s."; | ||||
| 
 | ||||
|     public static final String THREAD_ID_TAG = "client-id"; | ||||
|     public static final String THREAD_ID_TAG = "thread-id"; | ||||
|     public static final String THREAD_ID_TAG_0100_TO_23 = "client-id"; | ||||
|     public static final String TASK_ID_TAG = "task-id"; | ||||
|     public static final String STORE_ID_TAG = "state-id"; | ||||
|     public static final String RECORD_CACHE_ID_TAG = "record-cache-id"; | ||||
| 
 | ||||
|     public static final String ROLLUP_VALUE = "all"; | ||||
| 
 | ||||
|  | @ -77,6 +80,11 @@ public class StreamsMetricsImpl implements StreamsMetrics { | |||
|     public static final String TOTAL_SUFFIX = "-total"; | ||||
|     public static final String RATIO_SUFFIX = "-ratio"; | ||||
| 
 | ||||
|     public static final String AVG_VALUE_DOC = "The average value of "; | ||||
|     public static final String MAX_VALUE_DOC = "The maximum value of "; | ||||
|     public static final String AVG_LATENCY_DOC = "The average latency of "; | ||||
|     public static final String MAX_LATENCY_DOC = "The maximum latency of "; | ||||
| 
 | ||||
|     public static final String GROUP_PREFIX_WO_DELIMITER = "stream"; | ||||
|     public static final String GROUP_PREFIX = GROUP_PREFIX_WO_DELIMITER + "-"; | ||||
|     public static final String GROUP_SUFFIX = "-metrics"; | ||||
|  | @ -84,6 +92,7 @@ public class StreamsMetricsImpl implements StreamsMetrics { | |||
|     public static final String THREAD_LEVEL_GROUP = GROUP_PREFIX_WO_DELIMITER + GROUP_SUFFIX; | ||||
|     public static final String TASK_LEVEL_GROUP = GROUP_PREFIX + "task" + GROUP_SUFFIX; | ||||
|     public static final String STATE_LEVEL_GROUP = GROUP_PREFIX + "state" + GROUP_SUFFIX; | ||||
|     public static final String CACHE_LEVEL_GROUP = GROUP_PREFIX + "record-cache" + GROUP_SUFFIX; | ||||
| 
 | ||||
|     public static final String PROCESSOR_NODE_METRICS_GROUP = "stream-processor-node-metrics"; | ||||
|     public static final String PROCESSOR_NODE_ID_TAG = "processor-node-id"; | ||||
|  | @ -130,7 +139,7 @@ public class StreamsMetricsImpl implements StreamsMetrics { | |||
| 
 | ||||
|     public Map<String, String> threadLevelTagMap() { | ||||
|         final Map<String, String> tagMap = new LinkedHashMap<>(); | ||||
|         tagMap.put(THREAD_ID_TAG, threadName); | ||||
|         tagMap.put(THREAD_ID_TAG_0100_TO_23, threadName); | ||||
|         return tagMap; | ||||
|     } | ||||
| 
 | ||||
|  | @ -237,12 +246,12 @@ public class StreamsMetricsImpl implements StreamsMetrics { | |||
|         return taskSensorPrefix(taskName) + SENSOR_PREFIX_DELIMITER + "node" + SENSOR_PREFIX_DELIMITER + processorNodeName; | ||||
|     } | ||||
| 
 | ||||
|     public final Sensor cacheLevelSensor(final String taskName, | ||||
|                                          final String cacheName, | ||||
|     public Sensor cacheLevelSensor(final String taskName, | ||||
|                                    final String storeName, | ||||
|                                    final String sensorName, | ||||
|                                    final Sensor.RecordingLevel recordingLevel, | ||||
|                                    final Sensor... parents) { | ||||
|         final String key = cacheSensorPrefix(taskName, cacheName); | ||||
|         final String key = cacheSensorPrefix(taskName, storeName); | ||||
|         synchronized (cacheLevelSensors) { | ||||
|             if (!cacheLevelSensors.containsKey(key)) { | ||||
|                 cacheLevelSensors.put(key, new LinkedList<>()); | ||||
|  | @ -258,6 +267,18 @@ public class StreamsMetricsImpl implements StreamsMetrics { | |||
|         } | ||||
|     } | ||||
| 
 | ||||
|     public Map<String, String> cacheLevelTagMap(final String taskName, final String storeName) { | ||||
|         final Map<String, String> tagMap = new LinkedHashMap<>(); | ||||
|         tagMap.put(TASK_ID_TAG, taskName); | ||||
|         tagMap.put(RECORD_CACHE_ID_TAG, storeName); | ||||
|         if (version == Version.FROM_100_TO_23) { | ||||
|             tagMap.put(THREAD_ID_TAG_0100_TO_23, Thread.currentThread().getName()); | ||||
|         } else { | ||||
|             tagMap.put(THREAD_ID_TAG, Thread.currentThread().getName()); | ||||
|         } | ||||
|         return tagMap; | ||||
|     } | ||||
| 
 | ||||
|     public final void removeAllCacheLevelSensors(final String taskName, final String cacheName) { | ||||
|         final String key = cacheSensorPrefix(taskName, cacheName); | ||||
|         synchronized (cacheLevelSensors) { | ||||
|  | @ -423,15 +444,17 @@ public class StreamsMetricsImpl implements StreamsMetrics { | |||
|     } | ||||
| 
 | ||||
| 
 | ||||
|     public static void addAvgAndMaxToSensor(final Sensor sensor, | ||||
|     private static void addAvgAndMaxToSensor(final Sensor sensor, | ||||
|                                             final String group, | ||||
|                                             final Map<String, String> tags, | ||||
|                                             final String operation) { | ||||
|                                             final String operation, | ||||
|                                             final String descriptionOfAvg, | ||||
|                                             final String descriptionOfMax) { | ||||
|         sensor.add( | ||||
|             new MetricName( | ||||
|                 operation + AVG_SUFFIX, | ||||
|                 group, | ||||
|                 "The average value of " + operation + ".", | ||||
|                 descriptionOfAvg, | ||||
|                 tags), | ||||
|             new Avg() | ||||
|         ); | ||||
|  | @ -439,12 +462,26 @@ public class StreamsMetricsImpl implements StreamsMetrics { | |||
|             new MetricName( | ||||
|                 operation + MAX_SUFFIX, | ||||
|                 group, | ||||
|                 "The max value of " + operation + ".", | ||||
|                 descriptionOfMax, | ||||
|                 tags), | ||||
|             new Max() | ||||
|         ); | ||||
|     } | ||||
| 
 | ||||
|     public static void addAvgAndMaxToSensor(final Sensor sensor, | ||||
|                                             final String group, | ||||
|                                             final Map<String, String> tags, | ||||
|                                             final String operation) { | ||||
|         addAvgAndMaxToSensor( | ||||
|             sensor, | ||||
|             group, | ||||
|             tags, | ||||
|             operation, | ||||
|             AVG_VALUE_DOC + operation + ".", | ||||
|             MAX_VALUE_DOC + operation + "." | ||||
|         ); | ||||
|     } | ||||
| 
 | ||||
|     public static void addAvgAndMaxLatencyToSensor(final Sensor sensor, | ||||
|                                                    final String group, | ||||
|                                                    final Map<String, String> tags, | ||||
|  | @ -453,7 +490,7 @@ public class StreamsMetricsImpl implements StreamsMetrics { | |||
|             new MetricName( | ||||
|                 operation + "-latency-avg", | ||||
|                 group, | ||||
|                 "The average latency of " + operation + " operation.", | ||||
|                 AVG_LATENCY_DOC + operation + " operation.", | ||||
|                 tags), | ||||
|             new Avg() | ||||
|         ); | ||||
|  | @ -461,12 +498,30 @@ public class StreamsMetricsImpl implements StreamsMetrics { | |||
|             new MetricName( | ||||
|                 operation + "-latency-max", | ||||
|                 group, | ||||
|                 "The max latency of " + operation + " operation.", | ||||
|                 MAX_LATENCY_DOC + operation + " operation.", | ||||
|                 tags), | ||||
|             new Max() | ||||
|         ); | ||||
|     } | ||||
| 
 | ||||
|     public static void addAvgAndMinAndMaxToSensor(final Sensor sensor, | ||||
|                                                   final String group, | ||||
|                                                   final Map<String, String> tags, | ||||
|                                                   final String operation, | ||||
|                                                   final String descriptionOfAvg, | ||||
|                                                   final String descriptionOfMin, | ||||
|                                                   final String descriptionOfMax) { | ||||
|         addAvgAndMaxToSensor(sensor, group, tags, operation, descriptionOfAvg, descriptionOfMax); | ||||
|         sensor.add( | ||||
|             new MetricName( | ||||
|                 operation + MIN_SUFFIX, | ||||
|                 group, | ||||
|                 descriptionOfMin, | ||||
|                 tags), | ||||
|             new Min() | ||||
|         ); | ||||
|     } | ||||
| 
 | ||||
|     public static void addInvocationRateAndCountToSensor(final Sensor sensor, | ||||
|                                                          final String group, | ||||
|                                                          final Map<String, String> tags, | ||||
|  |  | |||
|  | @ -19,14 +19,12 @@ package org.apache.kafka.streams.state.internals; | |||
| import java.util.NavigableMap; | ||||
| import java.util.TreeMap; | ||||
| import java.util.TreeSet; | ||||
| import org.apache.kafka.common.MetricName; | ||||
| 
 | ||||
| import org.apache.kafka.common.metrics.Sensor; | ||||
| import org.apache.kafka.common.metrics.stats.Avg; | ||||
| import org.apache.kafka.common.metrics.stats.Max; | ||||
| import org.apache.kafka.common.metrics.stats.Min; | ||||
| import org.apache.kafka.common.utils.Bytes; | ||||
| import org.apache.kafka.streams.KeyValue; | ||||
| import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; | ||||
| import org.apache.kafka.streams.state.internals.metrics.NamedCacheMetrics; | ||||
| import org.slf4j.Logger; | ||||
| import org.slf4j.LoggerFactory; | ||||
| 
 | ||||
|  | @ -34,19 +32,22 @@ import java.util.ArrayList; | |||
| import java.util.Iterator; | ||||
| import java.util.LinkedHashSet; | ||||
| import java.util.List; | ||||
| import java.util.Map; | ||||
| import java.util.Set; | ||||
| 
 | ||||
| class NamedCache { | ||||
|     private static final Logger log = LoggerFactory.getLogger(NamedCache.class); | ||||
|     private final String name; | ||||
|     private final String storeName; | ||||
|     private final String taskName; | ||||
|     private final NavigableMap<Bytes, LRUNode> cache = new TreeMap<>(); | ||||
|     private final Set<Bytes> dirtyKeys = new LinkedHashSet<>(); | ||||
|     private ThreadCache.DirtyEntryFlushListener listener; | ||||
|     private LRUNode tail; | ||||
|     private LRUNode head; | ||||
|     private long currentSizeBytes; | ||||
|     private final NamedCacheMetrics namedCacheMetrics; | ||||
| 
 | ||||
|     private final StreamsMetricsImpl streamsMetrics; | ||||
|     private final Sensor hitRatioSensor; | ||||
| 
 | ||||
|     // internal stats | ||||
|     private long numReadHits = 0; | ||||
|  | @ -54,9 +55,12 @@ class NamedCache { | |||
|     private long numOverwrites = 0; | ||||
|     private long numFlushes = 0; | ||||
| 
 | ||||
|     NamedCache(final String name, final StreamsMetricsImpl metrics) { | ||||
|     NamedCache(final String name, final StreamsMetricsImpl streamsMetrics) { | ||||
|         this.name = name; | ||||
|         this.namedCacheMetrics = new NamedCacheMetrics(metrics, name); | ||||
|         this.streamsMetrics = streamsMetrics; | ||||
|         storeName = ThreadCache.underlyingStoreNamefromCacheName(name); | ||||
|         taskName = ThreadCache.taskIDfromCacheName(name); | ||||
|         hitRatioSensor = NamedCacheMetrics.hitRatioSensor(streamsMetrics, taskName, storeName); | ||||
|     } | ||||
| 
 | ||||
|     synchronized final String name() { | ||||
|  | @ -187,7 +191,7 @@ class NamedCache { | |||
|             return null; | ||||
|         } else { | ||||
|             numReadHits++; | ||||
|             namedCacheMetrics.hitRatioSensor.record((double) numReadHits / (double) (numReadHits + numReadMisses)); | ||||
|             hitRatioSensor.record((double) numReadHits / (double) (numReadHits + numReadMisses)); | ||||
|         } | ||||
|         return node; | ||||
|     } | ||||
|  | @ -311,7 +315,7 @@ class NamedCache { | |||
|         currentSizeBytes = 0; | ||||
|         dirtyKeys.clear(); | ||||
|         cache.clear(); | ||||
|         namedCacheMetrics.removeAllSensors(); | ||||
|         streamsMetrics.removeAllCacheLevelSensors(taskName, storeName); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|  | @ -357,68 +361,4 @@ class NamedCache { | |||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private static class NamedCacheMetrics { | ||||
|         private final StreamsMetricsImpl metrics; | ||||
| 
 | ||||
|         private final Sensor hitRatioSensor; | ||||
|         private final String taskName; | ||||
|         private final String cacheName; | ||||
| 
 | ||||
|         private NamedCacheMetrics(final StreamsMetricsImpl metrics, final String cacheName) { | ||||
|             taskName = ThreadCache.taskIDfromCacheName(cacheName); | ||||
|             this.cacheName = cacheName; | ||||
|             this.metrics = metrics; | ||||
|             final String group = "stream-record-cache-metrics"; | ||||
| 
 | ||||
|             // add parent | ||||
|             final Map<String, String> allMetricTags = metrics.tagMap( | ||||
|                  "task-id", taskName, | ||||
|                 "record-cache-id", "all" | ||||
|             ); | ||||
|             final Sensor taskLevelHitRatioSensor = metrics.taskLevelSensor(taskName, "hitRatio", Sensor.RecordingLevel.DEBUG); | ||||
|             taskLevelHitRatioSensor.add( | ||||
|                 new MetricName("hitRatio-avg", group, "The average cache hit ratio.", allMetricTags), | ||||
|                 new Avg() | ||||
|             ); | ||||
|             taskLevelHitRatioSensor.add( | ||||
|                 new MetricName("hitRatio-min", group, "The minimum cache hit ratio.", allMetricTags), | ||||
|                 new Min() | ||||
|             ); | ||||
|             taskLevelHitRatioSensor.add( | ||||
|                 new MetricName("hitRatio-max", group, "The maximum cache hit ratio.", allMetricTags), | ||||
|                 new Max() | ||||
|             ); | ||||
| 
 | ||||
|             // add child | ||||
|             final Map<String, String> metricTags = metrics.tagMap( | ||||
|                  "task-id", taskName, | ||||
|                 "record-cache-id", ThreadCache.underlyingStoreNamefromCacheName(cacheName) | ||||
|             ); | ||||
| 
 | ||||
|             hitRatioSensor = metrics.cacheLevelSensor( | ||||
|                 taskName, | ||||
|                 cacheName, | ||||
|                 "hitRatio", | ||||
|                 Sensor.RecordingLevel.DEBUG, | ||||
|                 taskLevelHitRatioSensor | ||||
|             ); | ||||
|             hitRatioSensor.add( | ||||
|                 new MetricName("hitRatio-avg", group, "The average cache hit ratio.", metricTags), | ||||
|                 new Avg() | ||||
|             ); | ||||
|             hitRatioSensor.add( | ||||
|                 new MetricName("hitRatio-min", group, "The minimum cache hit ratio.", metricTags), | ||||
|                 new Min() | ||||
|             ); | ||||
|             hitRatioSensor.add( | ||||
|                 new MetricName("hitRatio-max", group, "The maximum cache hit ratio.", metricTags), | ||||
|                 new Max() | ||||
|             ); | ||||
| 
 | ||||
|         } | ||||
| 
 | ||||
|         private void removeAllSensors() { | ||||
|             metrics.removeAllCacheLevelSensors(taskName, cacheName); | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  |  | |||
|  | @ -0,0 +1,86 @@ | |||
| /* | ||||
|  * Licensed to the Apache Software Foundation (ASF) under one or more | ||||
|  * contributor license agreements. See the NOTICE file distributed with | ||||
|  * this work for additional information regarding copyright ownership. | ||||
|  * The ASF licenses this file to You under the Apache License, Version 2.0 | ||||
|  * (the "License"); you may not use this file except in compliance with | ||||
|  * the License. You may obtain a copy of the License at | ||||
|  * | ||||
|  *    http://www.apache.org/licenses/LICENSE-2.0 | ||||
|  * | ||||
|  * Unless required by applicable law or agreed to in writing, software | ||||
|  * distributed under the License is distributed on an "AS IS" BASIS, | ||||
|  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||||
|  * See the License for the specific language governing permissions and | ||||
|  * limitations under the License. | ||||
|  */ | ||||
| package org.apache.kafka.streams.state.internals.metrics; | ||||
| 
 | ||||
| import org.apache.kafka.common.metrics.Sensor; | ||||
| import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; | ||||
| 
 | ||||
| import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.CACHE_LEVEL_GROUP; | ||||
| import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.ROLLUP_VALUE; | ||||
| import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.Version.FROM_100_TO_23; | ||||
| import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.addAvgAndMinAndMaxToSensor; | ||||
| 
 | ||||
| public class NamedCacheMetrics { | ||||
|     private NamedCacheMetrics() {} | ||||
| 
 | ||||
|     private static final String HIT_RATIO_0100_TO_23 = "hitRatio"; | ||||
|     private static final String HIT_RATIO = "hit-ratio"; | ||||
|     private static final String HIT_RATIO_AVG_DESCRIPTION = "The average cache hit ratio"; | ||||
|     private static final String HIT_RATIO_MIN_DESCRIPTION = "The minimum cache hit ratio"; | ||||
|     private static final String HIT_RATIO_MAX_DESCRIPTION = "The maximum cache hit ratio"; | ||||
| 
 | ||||
| 
 | ||||
|     public static Sensor hitRatioSensor(final StreamsMetricsImpl streamsMetrics, | ||||
|                                         final String taskName, | ||||
|                                         final String storeName) { | ||||
| 
 | ||||
|         final Sensor hitRatioSensor; | ||||
|         final String hitRatioName; | ||||
|         if (streamsMetrics.version() == FROM_100_TO_23) { | ||||
|             hitRatioName = HIT_RATIO_0100_TO_23; | ||||
|             final Sensor taskLevelHitRatioSensor = streamsMetrics.taskLevelSensor( | ||||
|                 taskName, | ||||
|                 hitRatioName, | ||||
|                 Sensor.RecordingLevel.DEBUG | ||||
|             ); | ||||
|             addAvgAndMinAndMaxToSensor( | ||||
|                 taskLevelHitRatioSensor, | ||||
|                 CACHE_LEVEL_GROUP, | ||||
|                 streamsMetrics.cacheLevelTagMap(taskName, ROLLUP_VALUE), | ||||
|                 hitRatioName, | ||||
|                 HIT_RATIO_AVG_DESCRIPTION, | ||||
|                 HIT_RATIO_MIN_DESCRIPTION, | ||||
|                 HIT_RATIO_MAX_DESCRIPTION | ||||
|             ); | ||||
|             hitRatioSensor = streamsMetrics.cacheLevelSensor( | ||||
|                 taskName, | ||||
|                 storeName, | ||||
|                 hitRatioName, | ||||
|                 Sensor.RecordingLevel.DEBUG, | ||||
|                 taskLevelHitRatioSensor | ||||
|             ); | ||||
|         } else { | ||||
|             hitRatioName = HIT_RATIO; | ||||
|             hitRatioSensor = streamsMetrics.cacheLevelSensor( | ||||
|                 taskName, | ||||
|                 storeName, | ||||
|                 hitRatioName, | ||||
|                 Sensor.RecordingLevel.DEBUG | ||||
|             ); | ||||
|         } | ||||
|         addAvgAndMinAndMaxToSensor( | ||||
|             hitRatioSensor, | ||||
|             CACHE_LEVEL_GROUP, | ||||
|             streamsMetrics.cacheLevelTagMap(taskName, storeName), | ||||
|             hitRatioName, | ||||
|             HIT_RATIO_AVG_DESCRIPTION, | ||||
|             HIT_RATIO_MIN_DESCRIPTION, | ||||
|             HIT_RATIO_MAX_DESCRIPTION | ||||
|         ); | ||||
|         return hitRatioSensor; | ||||
|     } | ||||
| } | ||||
|  | @ -151,9 +151,12 @@ public class MetricsIntegrationTest { | |||
|     private static final String SKIPPED_RECORDS_TOTAL = "skipped-records-total"; | ||||
|     private static final String RECORD_LATENESS_AVG = "record-lateness-avg"; | ||||
|     private static final String RECORD_LATENESS_MAX = "record-lateness-max"; | ||||
|     private static final String HIT_RATIO_AVG = "hitRatio-avg"; | ||||
|     private static final String HIT_RATIO_MIN = "hitRatio-min"; | ||||
|     private static final String HIT_RATIO_MAX = "hitRatio-max"; | ||||
|     private static final String HIT_RATIO_AVG_BEFORE_24 = "hitRatio-avg"; | ||||
|     private static final String HIT_RATIO_MIN_BEFORE_24 = "hitRatio-min"; | ||||
|     private static final String HIT_RATIO_MAX_BEFORE_24 = "hitRatio-max"; | ||||
|     private static final String HIT_RATIO_AVG = "hit-ratio-avg"; | ||||
|     private static final String HIT_RATIO_MIN = "hit-ratio-min"; | ||||
|     private static final String HIT_RATIO_MAX = "hit-ratio-max"; | ||||
| 
 | ||||
|     // RocksDB metrics | ||||
|     private static final String BYTES_WRITTEN_RATE = "bytes-written-rate"; | ||||
|  | @ -275,7 +278,18 @@ public class MetricsIntegrationTest { | |||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void shouldAddMetricsOnAllLevels() throws Exception { | ||||
|     public void shouldAddMetricsOnAllLevelsWithBuiltInMetricsLatestVersion() throws Exception { | ||||
|         shouldAddMetricsOnAllLevels(StreamsConfig.METRICS_LATEST); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void shouldAddMetricsOnAllLevelsWithBuiltInMetricsVersion0100To23() throws Exception { | ||||
|         shouldAddMetricsOnAllLevels(StreamsConfig.METRICS_0100_TO_23); | ||||
|     } | ||||
| 
 | ||||
|     private void shouldAddMetricsOnAllLevels(final String builtInMetricsVersion) throws Exception { | ||||
|         streamsConfiguration.put(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG, builtInMetricsVersion); | ||||
| 
 | ||||
|         builder.stream(STREAM_INPUT, Consumed.with(Serdes.Integer(), Serdes.String())) | ||||
|             .to(STREAM_OUTPUT_1, Produced.with(Serdes.Integer(), Serdes.String())); | ||||
|         builder.table(STREAM_OUTPUT_1, | ||||
|  | @ -299,7 +313,7 @@ public class MetricsIntegrationTest { | |||
|         checkKeyValueStoreMetricsByGroup(STREAM_STORE_ROCKSDB_STATE_METRICS); | ||||
|         checkKeyValueStoreMetricsByGroup(STREAM_STORE_IN_MEMORY_LRU_STATE_METRICS); | ||||
|         checkRocksDBMetricsByTag("rocksdb-state-id"); | ||||
|         checkCacheMetrics(); | ||||
|         checkCacheMetrics(builtInMetricsVersion); | ||||
| 
 | ||||
|         closeApplication(); | ||||
| 
 | ||||
|  | @ -526,13 +540,25 @@ public class MetricsIntegrationTest { | |||
|         assertThat(listMetricAfterClosingApp.size(), is(0)); | ||||
|     } | ||||
| 
 | ||||
|     private void checkCacheMetrics() { | ||||
|     private void checkCacheMetrics(final String builtInMetricsVersion) { | ||||
|         final List<Metric> listMetricCache = new ArrayList<Metric>(kafkaStreams.metrics().values()).stream() | ||||
|             .filter(m -> m.metricName().group().equals(STREAM_CACHE_NODE_METRICS)) | ||||
|             .collect(Collectors.toList()); | ||||
|         checkMetricByName(listMetricCache, HIT_RATIO_AVG, 6); | ||||
|         checkMetricByName(listMetricCache, HIT_RATIO_MIN, 6); | ||||
|         checkMetricByName(listMetricCache, HIT_RATIO_MAX, 6); | ||||
|         checkMetricByName( | ||||
|             listMetricCache, | ||||
|             builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? HIT_RATIO_AVG : HIT_RATIO_AVG_BEFORE_24, | ||||
|             builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? 3 : 6 /* includes parent sensors */ | ||||
|         ); | ||||
|         checkMetricByName( | ||||
|             listMetricCache, | ||||
|             builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? HIT_RATIO_MIN : HIT_RATIO_MIN_BEFORE_24, | ||||
|             builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? 3 : 6 /* includes parent sensors */ | ||||
|         ); | ||||
|         checkMetricByName( | ||||
|             listMetricCache, | ||||
|             builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? HIT_RATIO_MAX : HIT_RATIO_MAX_BEFORE_24, | ||||
|             builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? 3 : 6 /* includes parent sensors */ | ||||
|         ); | ||||
|     } | ||||
| 
 | ||||
|     private void checkWindowStoreMetrics() { | ||||
|  |  | |||
|  | @ -62,6 +62,7 @@ public class StreamsMetricsImplTest extends EasyMockSupport { | |||
|     private final Map<String, String> tags = mkMap(mkEntry("tag", "value")); | ||||
|     private final String description1 = "description number one"; | ||||
|     private final String description2 = "description number two"; | ||||
|     private final String description3 = "description number three"; | ||||
|     private final MockTime time = new MockTime(0); | ||||
|     private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, THREAD_NAME, VERSION); | ||||
| 
 | ||||
|  | @ -252,11 +253,40 @@ public class StreamsMetricsImplTest extends EasyMockSupport { | |||
|         final Map<String, String> tagMap = streamsMetrics.storeLevelTagMap(taskName, storeType, storeName); | ||||
| 
 | ||||
|         assertThat(tagMap.size(), equalTo(3)); | ||||
|         assertThat(tagMap.get(StreamsMetricsImpl.THREAD_ID_TAG), equalTo(THREAD_NAME)); | ||||
|         assertThat(tagMap.get(StreamsMetricsImpl.THREAD_ID_TAG_0100_TO_23), equalTo(THREAD_NAME)); | ||||
|         assertThat(tagMap.get(StreamsMetricsImpl.TASK_ID_TAG), equalTo(taskName)); | ||||
|         assertThat(tagMap.get(storeType + "-" + StreamsMetricsImpl.STORE_ID_TAG), equalTo(storeName)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void shouldGetCacheLevelTagMapForBuiltInMetricsLatestVersion() { | ||||
|         shouldGetCacheLevelTagMap(StreamsConfig.METRICS_LATEST); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void shouldGetCacheLevelTagMapForBuiltInMetricsVersion0100To23() { | ||||
|         shouldGetCacheLevelTagMap(StreamsConfig.METRICS_0100_TO_23); | ||||
|     } | ||||
| 
 | ||||
|     private void shouldGetCacheLevelTagMap(final String builtInMetricsVersion) { | ||||
|         final StreamsMetricsImpl streamsMetrics = | ||||
|             new StreamsMetricsImpl(metrics, THREAD_NAME, builtInMetricsVersion); | ||||
|         final String taskName = "taskName"; | ||||
|         final String storeName = "storeName"; | ||||
| 
 | ||||
|         final Map<String, String> tagMap = streamsMetrics.cacheLevelTagMap(taskName, storeName); | ||||
| 
 | ||||
|         assertThat(tagMap.size(), equalTo(3)); | ||||
|         assertThat( | ||||
|             tagMap.get( | ||||
|                 builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? StreamsMetricsImpl.THREAD_ID_TAG | ||||
|                     : StreamsMetricsImpl.THREAD_ID_TAG_0100_TO_23), | ||||
|             equalTo(Thread.currentThread().getName()) | ||||
|         ); | ||||
|         assertThat(tagMap.get(StreamsMetricsImpl.TASK_ID_TAG), equalTo(taskName)); | ||||
|         assertThat(tagMap.get(StreamsMetricsImpl.RECORD_CACHE_ID_TAG), equalTo(storeName)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void shouldAddAmountRateAndSum() { | ||||
|         StreamsMetricsImpl | ||||
|  | @ -325,6 +355,20 @@ public class StreamsMetricsImplTest extends EasyMockSupport { | |||
|         assertThat(metrics.metrics().size(), equalTo(2 + 1)); // one metric is added automatically in the constructor of Metrics | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void shouldAddAvgAndMinAndMaxMetricsToSensor() { | ||||
|         StreamsMetricsImpl | ||||
|             .addAvgAndMinAndMaxToSensor(sensor, group, tags, metricNamePrefix, description1, description2, description3); | ||||
| 
 | ||||
|         final double valueToRecord1 = 18.0; | ||||
|         final double valueToRecord2 = 42.0; | ||||
|         final double expectedAvgMetricValue = (valueToRecord1 + valueToRecord2) / 2; | ||||
|         verifyMetric(metricNamePrefix + "-avg", description1, valueToRecord1, valueToRecord2, expectedAvgMetricValue); | ||||
|         verifyMetric(metricNamePrefix + "-min", description2, valueToRecord1, valueToRecord2, valueToRecord1); | ||||
|         verifyMetric(metricNamePrefix + "-max", description3, valueToRecord1, valueToRecord2, valueToRecord2); | ||||
|         assertThat(metrics.metrics().size(), equalTo(3 + 1)); // one metric is added automatically in the constructor of Metrics | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void shouldReturnMetricsVersionCurrent() { | ||||
|         assertThat( | ||||
|  |  | |||
|  | @ -20,7 +20,6 @@ import org.apache.kafka.common.header.Header; | |||
| import org.apache.kafka.common.header.Headers; | ||||
| import org.apache.kafka.common.header.internals.RecordHeader; | ||||
| import org.apache.kafka.common.header.internals.RecordHeaders; | ||||
| import org.apache.kafka.common.metrics.JmxReporter; | ||||
| import org.apache.kafka.common.metrics.Metrics; | ||||
| import org.apache.kafka.common.utils.Bytes; | ||||
| import org.apache.kafka.streams.KeyValue; | ||||
|  | @ -29,20 +28,15 @@ import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; | |||
| import org.junit.Before; | ||||
| import org.junit.Test; | ||||
| 
 | ||||
| import java.io.IOException; | ||||
| import java.util.ArrayList; | ||||
| import java.util.Arrays; | ||||
| import java.util.LinkedHashMap; | ||||
| import java.util.List; | ||||
| import java.util.Map; | ||||
| 
 | ||||
| import static org.apache.kafka.test.StreamsTestUtils.getMetricByNameFilterByTags; | ||||
| import static org.junit.Assert.assertArrayEquals; | ||||
| import static org.junit.Assert.assertEquals; | ||||
| import static org.junit.Assert.assertNotNull; | ||||
| import static org.junit.Assert.assertNull; | ||||
| import static org.junit.Assert.assertSame; | ||||
| import static org.junit.Assert.assertTrue; | ||||
| 
 | ||||
| public class NamedCacheTest { | ||||
| 
 | ||||
|  | @ -61,7 +55,7 @@ public class NamedCacheTest { | |||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void shouldKeepTrackOfMostRecentlyAndLeastRecentlyUsed() throws IOException { | ||||
|     public void shouldKeepTrackOfMostRecentlyAndLeastRecentlyUsed() { | ||||
|         final List<KeyValue<String, String>> toInsert = Arrays.asList( | ||||
|                 new KeyValue<>("K1", "V1"), | ||||
|                 new KeyValue<>("K2", "V2"), | ||||
|  | @ -83,31 +77,6 @@ public class NamedCacheTest { | |||
|         } | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testMetrics() { | ||||
|         final Map<String, String> metricTags = new LinkedHashMap<>(); | ||||
|         metricTags.put("record-cache-id", underlyingStoreName); | ||||
|         metricTags.put("task-id", taskIDString); | ||||
|         metricTags.put("client-id", "test"); | ||||
| 
 | ||||
|         getMetricByNameFilterByTags(metrics.metrics(), "hitRatio-avg", "stream-record-cache-metrics", metricTags); | ||||
|         getMetricByNameFilterByTags(metrics.metrics(), "hitRatio-min", "stream-record-cache-metrics", metricTags); | ||||
|         getMetricByNameFilterByTags(metrics.metrics(), "hitRatio-max", "stream-record-cache-metrics", metricTags); | ||||
| 
 | ||||
|         // test "all" | ||||
|         metricTags.put("record-cache-id", "all"); | ||||
|         getMetricByNameFilterByTags(metrics.metrics(), "hitRatio-avg", "stream-record-cache-metrics", metricTags); | ||||
|         getMetricByNameFilterByTags(metrics.metrics(), "hitRatio-min", "stream-record-cache-metrics", metricTags); | ||||
|         getMetricByNameFilterByTags(metrics.metrics(), "hitRatio-max", "stream-record-cache-metrics", metricTags); | ||||
| 
 | ||||
|         final JmxReporter reporter = new JmxReporter("kafka.streams"); | ||||
|         innerMetrics.addReporter(reporter); | ||||
|         assertTrue(reporter.containsMbean(String.format("kafka.streams:type=stream-record-cache-metrics,client-id=test,task-id=%s,record-cache-id=%s", | ||||
|                 taskIDString, underlyingStoreName))); | ||||
|         assertTrue(reporter.containsMbean(String.format("kafka.streams:type=stream-record-cache-metrics,client-id=test,task-id=%s,record-cache-id=%s", | ||||
|                 taskIDString, "all"))); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void shouldKeepTrackOfSize() { | ||||
|         final LRUCacheEntry value = new LRUCacheEntry(new byte[]{0}); | ||||
|  |  | |||
|  | @ -0,0 +1,117 @@ | |||
| /* | ||||
|  * Licensed to the Apache Software Foundation (ASF) under one or more | ||||
|  * contributor license agreements. See the NOTICE file distributed with | ||||
|  * this work for additional information regarding copyright ownership. | ||||
|  * The ASF licenses this file to You under the Apache License, Version 2.0 | ||||
|  * (the "License"); you may not use this file except in compliance with | ||||
|  * the License. You may obtain a copy of the License at | ||||
|  * | ||||
|  *    http://www.apache.org/licenses/LICENSE-2.0 | ||||
|  * | ||||
|  * Unless required by applicable law or agreed to in writing, software | ||||
|  * distributed under the License is distributed on an "AS IS" BASIS, | ||||
|  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||||
|  * See the License for the specific language governing permissions and | ||||
|  * limitations under the License. | ||||
|  */ | ||||
| package org.apache.kafka.streams.state.internals.metrics; | ||||
| 
 | ||||
| import org.apache.kafka.common.metrics.Sensor; | ||||
| import org.apache.kafka.common.metrics.Sensor.RecordingLevel; | ||||
| import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; | ||||
| import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.Version; | ||||
| import org.junit.Test; | ||||
| import org.junit.runner.RunWith; | ||||
| import org.powermock.core.classloader.annotations.PrepareForTest; | ||||
| import org.powermock.modules.junit4.PowerMockRunner; | ||||
| 
 | ||||
| import java.util.Map; | ||||
| 
 | ||||
| import static org.apache.kafka.common.utils.Utils.mkEntry; | ||||
| import static org.apache.kafka.common.utils.Utils.mkMap; | ||||
| import static org.easymock.EasyMock.expect; | ||||
| import static org.easymock.EasyMock.mock; | ||||
| import static org.hamcrest.CoreMatchers.is; | ||||
| import static org.hamcrest.MatcherAssert.assertThat; | ||||
| import static org.powermock.api.easymock.PowerMock.createMock; | ||||
| import static org.powermock.api.easymock.PowerMock.mockStatic; | ||||
| import static org.powermock.api.easymock.PowerMock.replay; | ||||
| import static org.powermock.api.easymock.PowerMock.verify; | ||||
| 
 | ||||
| 
 | ||||
| @RunWith(PowerMockRunner.class) | ||||
| @PrepareForTest({StreamsMetricsImpl.class, Sensor.class}) | ||||
| public class NamedCacheMetricsTest { | ||||
| 
 | ||||
|     private static final String TASK_NAME = "taskName"; | ||||
|     private static final String STORE_NAME = "storeName"; | ||||
|     private static final String HIT_RATIO_AVG_DESCRIPTION = "The average cache hit ratio"; | ||||
|     private static final String HIT_RATIO_MIN_DESCRIPTION = "The minimum cache hit ratio"; | ||||
|     private static final String HIT_RATIO_MAX_DESCRIPTION = "The maximum cache hit ratio"; | ||||
| 
 | ||||
|     private final StreamsMetricsImpl streamsMetrics = createMock(StreamsMetricsImpl.class); | ||||
|     private final Sensor expectedSensor = mock(Sensor.class); | ||||
|     private final Map<String, String> tagMap = mkMap(mkEntry("key", "value")); | ||||
| 
 | ||||
|     @Test | ||||
|     public void shouldGetHitRatioSensorWithBuiltInMetricsVersionCurrent() { | ||||
|         final String hitRatio = "hit-ratio"; | ||||
|         mockStatic(StreamsMetricsImpl.class); | ||||
|         setUpStreamsMetrics(Version.LATEST, hitRatio); | ||||
|         replay(streamsMetrics); | ||||
|         replay(StreamsMetricsImpl.class); | ||||
| 
 | ||||
|         final Sensor sensor = NamedCacheMetrics.hitRatioSensor(streamsMetrics, TASK_NAME, STORE_NAME); | ||||
| 
 | ||||
|         verifyResult(sensor); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void shouldGetHitRatioSensorWithBuiltInMetricsVersionBefore24() { | ||||
|         final Map<String, String> parentTagMap = mkMap(mkEntry("key", "all")); | ||||
|         final String hitRatio = "hitRatio"; | ||||
|         final RecordingLevel recordingLevel = RecordingLevel.DEBUG; | ||||
|         mockStatic(StreamsMetricsImpl.class); | ||||
|         final Sensor parentSensor = mock(Sensor.class); | ||||
|         expect(streamsMetrics.taskLevelSensor(TASK_NAME, hitRatio, recordingLevel)).andReturn(parentSensor); | ||||
|         expect(streamsMetrics.cacheLevelTagMap(TASK_NAME, StreamsMetricsImpl.ROLLUP_VALUE)).andReturn(parentTagMap); | ||||
|         StreamsMetricsImpl.addAvgAndMinAndMaxToSensor( | ||||
|             parentSensor, | ||||
|             StreamsMetricsImpl.CACHE_LEVEL_GROUP, | ||||
|             parentTagMap, | ||||
|             hitRatio, | ||||
|             HIT_RATIO_AVG_DESCRIPTION, | ||||
|             HIT_RATIO_MIN_DESCRIPTION, | ||||
|             HIT_RATIO_MAX_DESCRIPTION); | ||||
|         setUpStreamsMetrics(Version.FROM_100_TO_23, hitRatio, parentSensor); | ||||
|         replay(streamsMetrics); | ||||
|         replay(StreamsMetricsImpl.class); | ||||
| 
 | ||||
|         final Sensor sensor = NamedCacheMetrics.hitRatioSensor(streamsMetrics, TASK_NAME, STORE_NAME); | ||||
| 
 | ||||
|         verifyResult(sensor); | ||||
|     } | ||||
| 
 | ||||
|     private void setUpStreamsMetrics(final Version builtInMetricsVersion, | ||||
|                                      final String hitRatio, | ||||
|                                      final Sensor... parents) { | ||||
|         expect(streamsMetrics.version()).andReturn(builtInMetricsVersion); | ||||
|         expect(streamsMetrics.cacheLevelSensor(TASK_NAME, STORE_NAME, hitRatio, RecordingLevel.DEBUG, parents)) | ||||
|             .andReturn(expectedSensor); | ||||
|         expect(streamsMetrics.cacheLevelTagMap(TASK_NAME, STORE_NAME)).andReturn(tagMap); | ||||
|         StreamsMetricsImpl.addAvgAndMinAndMaxToSensor( | ||||
|             expectedSensor, | ||||
|             StreamsMetricsImpl.CACHE_LEVEL_GROUP, | ||||
|             tagMap, | ||||
|             hitRatio, | ||||
|             HIT_RATIO_AVG_DESCRIPTION, | ||||
|             HIT_RATIO_MIN_DESCRIPTION, | ||||
|             HIT_RATIO_MAX_DESCRIPTION); | ||||
|     } | ||||
| 
 | ||||
|     private void verifyResult(final Sensor sensor) { | ||||
|         verify(streamsMetrics); | ||||
|         verify(StreamsMetricsImpl.class); | ||||
|         assertThat(sensor, is(expectedSensor)); | ||||
|     } | ||||
| } | ||||
		Loading…
	
		Reference in New Issue