From 36baab67fb302d5b0424a406eb35ead6b4df305b Mon Sep 17 00:00:00 2001 From: David Zane Date: Wed, 22 Jan 2025 16:20:15 -0800 Subject: [PATCH] Add field type cache stats Signed-off-by: David Zane --- .../core/listener/QueryInsightsListener.java | 1 + .../core/service/QueryInsightsService.java | 25 +++++++- .../categorizer/IndicesFieldTypeCache.java | 64 +++++++++++++++++-- .../categorizer/QueryShapeGenerator.java | 40 +++++++++++- .../healthStats/QueryInsightsHealthStats.java | 36 ++++++++++- .../service/QueryInsightsServiceTests.java | 18 +++++- .../HealthStatsNodeResponseTests.java | 1 + .../HealthStatsResponseTests.java | 3 +- .../QueryInsightsHealthStatsTests.java | 36 ++++++++++- 9 files changed, 209 insertions(+), 15 deletions(-) diff --git a/src/main/java/org/opensearch/plugin/insights/core/listener/QueryInsightsListener.java b/src/main/java/org/opensearch/plugin/insights/core/listener/QueryInsightsListener.java index e2c6255f..54d4d863 100644 --- a/src/main/java/org/opensearch/plugin/insights/core/listener/QueryInsightsListener.java +++ b/src/main/java/org/opensearch/plugin/insights/core/listener/QueryInsightsListener.java @@ -87,6 +87,7 @@ public QueryInsightsListener( this.clusterService = clusterService; this.queryInsightsService = queryInsightsService; this.queryShapeGenerator = new QueryShapeGenerator(clusterService); + queryInsightsService.setQueryShapeGenerator(queryShapeGenerator); // Setting endpoints set up for top n queries, including enabling top n queries, window size, and top n size // Expected metricTypes are Latency, CPU, and Memory. diff --git a/src/main/java/org/opensearch/plugin/insights/core/service/QueryInsightsService.java b/src/main/java/org/opensearch/plugin/insights/core/service/QueryInsightsService.java index 48fc0aa9..808e4ff5 100644 --- a/src/main/java/org/opensearch/plugin/insights/core/service/QueryInsightsService.java +++ b/src/main/java/org/opensearch/plugin/insights/core/service/QueryInsightsService.java @@ -15,10 +15,12 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -36,6 +38,7 @@ import org.opensearch.plugin.insights.core.metrics.OperationalMetric; import org.opensearch.plugin.insights.core.metrics.OperationalMetricsCounter; import org.opensearch.plugin.insights.core.reader.QueryInsightsReaderFactory; +import org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator; import org.opensearch.plugin.insights.core.service.categorizer.SearchQueryCategorizer; import org.opensearch.plugin.insights.rules.model.GroupingType; import org.opensearch.plugin.insights.rules.model.MetricType; @@ -100,9 +103,14 @@ public class QueryInsightsService extends AbstractLifecycleComponent { private volatile boolean searchQueryMetricsEnabled; - private SearchQueryCategorizer searchQueryCategorizer; + private final SearchQueryCategorizer searchQueryCategorizer; - private NamedXContentRegistry namedXContentRegistry; + private final NamedXContentRegistry namedXContentRegistry; + + /** + * Query shape generator instance + */ + private QueryShapeGenerator queryShapeGenerator; /** * Constructor of the QueryInsightsService @@ -496,10 +504,14 @@ public QueryInsightsHealthStats getHealthStats() { Map topQueriesHealthStatsMap = topQueriesServices.entrySet() .stream() .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().getHealthStats())); + Map fieldTypeCacheStats = Optional.ofNullable(queryShapeGenerator) + .map(QueryShapeGenerator::getFieldTypeCacheStats) + .orElse(Collections.emptyMap()); return new QueryInsightsHealthStats( threadPool.info(QUERY_INSIGHTS_EXECUTOR), this.queryRecordsQueue.size(), - topQueriesHealthStatsMap + topQueriesHealthStatsMap, + fieldTypeCacheStats ); } @@ -511,4 +523,11 @@ private void deleteExpiredTopNIndices() { topQueriesServices.get(metricType).deleteExpiredTopNIndices(clusterService.state().metadata().indices()); } } + + /** + * Set query shape generator + */ + public void setQueryShapeGenerator(final QueryShapeGenerator queryShapeGenerator) { + this.queryShapeGenerator = queryShapeGenerator; + } } diff --git a/src/main/java/org/opensearch/plugin/insights/core/service/categorizer/IndicesFieldTypeCache.java b/src/main/java/org/opensearch/plugin/insights/core/service/categorizer/IndicesFieldTypeCache.java index abf79759..cb442d35 100644 --- a/src/main/java/org/opensearch/plugin/insights/core/service/categorizer/IndicesFieldTypeCache.java +++ b/src/main/java/org/opensearch/plugin/insights/core/service/categorizer/IndicesFieldTypeCache.java @@ -28,6 +28,18 @@ public class IndicesFieldTypeCache { private static final Logger logger = LogManager.getLogger(IndicesFieldTypeCache.class); private final Cache cache; + /** + * Count of cache evictions + */ + private final CounterMetric evictionCount; + /** + * Count of items in cache + */ + private final CounterMetric entryCount; + /** + * Weight of cache in bytes + */ + private final CounterMetric weight; public IndicesFieldTypeCache(Settings settings) { final long sizeInBytes = QueryCategorizationSettings.SEARCH_QUERY_FIELD_TYPE_CACHE_SIZE_KEY.get(settings).getBytes(); @@ -36,9 +48,12 @@ public IndicesFieldTypeCache(Settings settings) { cacheBuilder.setMaximumWeight(sizeInBytes).weigher((k, v) -> RamUsageEstimator.sizeOfObject(k) + v.weight()); } cache = cacheBuilder.build(); + evictionCount = new CounterMetric(); + entryCount = new CounterMetric(); + weight = new CounterMetric(); } - public IndexFieldMap getOrInitialize(Index index) { + IndexFieldMap getOrInitialize(Index index) { try { return cache.computeIfAbsent(index, k -> new IndexFieldMap()); } catch (ExecutionException ex) { @@ -52,6 +67,10 @@ public IndexFieldMap getOrInitialize(Index index) { } public void invalidate(Index index) { + IndexFieldMap indexFieldMap = cache.get(index); + evictionCount.inc(indexFieldMap.fieldTypeMap.size()); + entryCount.dec(indexFieldMap.fieldTypeMap.size()); + weight.dec(indexFieldMap.weight()); cache.invalidate(index); } @@ -59,9 +78,39 @@ public Iterable keySet() { return cache.keys(); } + public void incrementCountAndWeight(String key, String value) { + entryCount.inc(); + weight.inc(RamUsageEstimator.sizeOf(key) + RamUsageEstimator.sizeOf(value)); + } + + /** + * Get eviction count + */ + public Long getEvictionCount() { + return evictionCount.count(); + } + + /** + * Get entry count + */ + public Long getEntryCount() { + return entryCount.count(); + } + + /** + * Get cache weight in bytes + */ + public Long getWeight() { + return weight.count(); + } + static class IndexFieldMap { - private ConcurrentHashMap fieldTypeMap; - private CounterMetric weight; + private final ConcurrentHashMap fieldTypeMap; + + /** + * Estimated memory consumption of fieldTypeMap in bytes + */ + private final CounterMetric weight; IndexFieldMap() { fieldTypeMap = new ConcurrentHashMap<>(); @@ -72,11 +121,18 @@ public String get(String fieldName) { return fieldTypeMap.get(fieldName); } - public void putIfAbsent(String key, String value) { + /** + * Stores key, value if absent + * + * @return {@code true} if key was absent, else {@code false} + */ + public boolean putIfAbsent(String key, String value) { // Increment the weight only if the key value pair added to the Map if (fieldTypeMap.putIfAbsent(key, value) == null) { weight.inc(RamUsageEstimator.sizeOf(key) + RamUsageEstimator.sizeOf(value)); + return true; } + return false; } public long weight() { diff --git a/src/main/java/org/opensearch/plugin/insights/core/service/categorizer/QueryShapeGenerator.java b/src/main/java/org/opensearch/plugin/insights/core/service/categorizer/QueryShapeGenerator.java index 191c6065..b51bb22d 100644 --- a/src/main/java/org/opensearch/plugin/insights/core/service/categorizer/QueryShapeGenerator.java +++ b/src/main/java/org/opensearch/plugin/insights/core/service/categorizer/QueryShapeGenerator.java @@ -38,13 +38,23 @@ public class QueryShapeGenerator implements ClusterStateListener { static final String EMPTY_STRING = ""; static final String ONE_SPACE_INDENT = " "; private final ClusterService clusterService; - private final String NO_FIELD_TYPE_VALUE = ""; private final IndicesFieldTypeCache indicesFieldTypeCache; + private long cacheHitCount; + private long cacheMissCount; + + private final String NO_FIELD_TYPE_VALUE = ""; + public static final String HIT_COUNT = "hit_count"; + public static final String MISS_COUNT = "miss_count"; + public static final String EVICTIONS = "evictions"; + public static final String ENTRY_COUNT = "entry_count"; + public static final String SIZE_IN_BYTES = "size_in_bytes"; public QueryShapeGenerator(ClusterService clusterService) { this.clusterService = clusterService; clusterService.addListener(this); this.indicesFieldTypeCache = new IndicesFieldTypeCache(clusterService.getSettings()); + this.cacheHitCount = 0; + this.cacheMissCount = 0; } public void clusterChanged(ClusterChangedEvent event) { @@ -369,14 +379,20 @@ String getFieldType(String fieldName, Map propertiesAsMap, Index String fieldType = getFieldTypeFromCache(fieldName, index); if (fieldType != null) { + cacheHitCount += 1; return fieldType; + } else { + cacheMissCount += 1; } // Retrieve field type from mapping and cache it if found fieldType = getFieldTypeFromProperties(fieldName, propertiesAsMap); // Cache field type or NO_FIELD_TYPE_VALUE if not found - indicesFieldTypeCache.getOrInitialize(index).putIfAbsent(fieldName, fieldType != null ? fieldType : NO_FIELD_TYPE_VALUE); + fieldType = fieldType != null ? fieldType : NO_FIELD_TYPE_VALUE; + if (indicesFieldTypeCache.getOrInitialize(index).putIfAbsent(fieldName, fieldType)) { + indicesFieldTypeCache.incrementCountAndWeight(fieldName, fieldType); + } return fieldType; } @@ -420,4 +436,24 @@ else if (currentMap.containsKey("type")) { String getFieldTypeFromCache(String fieldName, Index index) { return indicesFieldTypeCache.getOrInitialize(index).get(fieldName); } + + /** + * Get field type cache stats + * + * @return Map containing cache hit count, miss count, and byte stats + */ + public Map getFieldTypeCacheStats() { + return Map.of( + SIZE_IN_BYTES, + indicesFieldTypeCache.getWeight(), + ENTRY_COUNT, + indicesFieldTypeCache.getEntryCount(), + EVICTIONS, + indicesFieldTypeCache.getEvictionCount(), + HIT_COUNT, + cacheHitCount, + MISS_COUNT, + cacheMissCount + ); + } } diff --git a/src/main/java/org/opensearch/plugin/insights/rules/model/healthStats/QueryInsightsHealthStats.java b/src/main/java/org/opensearch/plugin/insights/rules/model/healthStats/QueryInsightsHealthStats.java index 1fcd94eb..0c0aec65 100644 --- a/src/main/java/org/opensearch/plugin/insights/rules/model/healthStats/QueryInsightsHealthStats.java +++ b/src/main/java/org/opensearch/plugin/insights/rules/model/healthStats/QueryInsightsHealthStats.java @@ -8,8 +8,17 @@ package org.opensearch.plugin.insights.rules.model.healthStats; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.ENTRY_COUNT; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.EVICTIONS; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.HIT_COUNT; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.MISS_COUNT; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.SIZE_IN_BYTES; + import java.io.IOException; +import java.util.List; import java.util.Map; +import java.util.Objects; +import org.opensearch.Version; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; @@ -26,10 +35,12 @@ public class QueryInsightsHealthStats implements ToXContentFragment, Writeable { private final ThreadPool.Info threadPoolInfo; private final int queryRecordsQueueSize; private final Map topQueriesHealthStats; + private Map fieldTypeCacheStats; private static final String THREAD_POOL_INFO = "ThreadPoolInfo"; private static final String QUERY_RECORDS_QUEUE_SIZE = "QueryRecordsQueueSize"; private static final String TOP_QUERIES_HEALTH_STATS = "TopQueriesHealthStats"; + private static final String FIELD_TYPE_CACHE_STATS = "FieldTypeCacheStats"; /** * Constructor to read QueryInsightsHealthStats from a StreamInput. @@ -41,6 +52,9 @@ public QueryInsightsHealthStats(final StreamInput in) throws IOException { this.threadPoolInfo = new ThreadPool.Info(in); this.queryRecordsQueueSize = in.readInt(); this.topQueriesHealthStats = in.readMap(MetricType::readFromStream, TopQueriesHealthStats::new); + if (in.getVersion().onOrAfter(Version.V_2_19_0)) { + this.fieldTypeCacheStats = in.readMap(StreamInput::readString, StreamInput::readLong); + } } /** @@ -53,7 +67,8 @@ public QueryInsightsHealthStats(final StreamInput in) throws IOException { public QueryInsightsHealthStats( final ThreadPool.Info threadPoolInfo, final int queryRecordsQueueSize, - final Map topQueriesHealthStats + final Map topQueriesHealthStats, + final Map fieldTypeCacheStats ) { if (threadPoolInfo == null || topQueriesHealthStats == null) { throw new IllegalArgumentException("Parameters cannot be null"); @@ -61,6 +76,7 @@ public QueryInsightsHealthStats( this.threadPoolInfo = threadPoolInfo; this.queryRecordsQueueSize = queryRecordsQueueSize; this.topQueriesHealthStats = topQueriesHealthStats; + this.fieldTypeCacheStats = Objects.requireNonNull(fieldTypeCacheStats, "fieldTypeCacheStats cannot be null"); } /** @@ -87,6 +103,12 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa builder.endObject(); } builder.endObject(); + // Write field type cache stats + builder.startObject(FIELD_TYPE_CACHE_STATS); + for (String key : List.of(SIZE_IN_BYTES, ENTRY_COUNT, EVICTIONS, HIT_COUNT, MISS_COUNT)) { + builder.field(key, fieldTypeCacheStats.getOrDefault(key, 0L)); + } + builder.endObject(); return builder; } @@ -105,6 +127,9 @@ public void writeTo(final StreamOutput out) throws IOException { MetricType::writeTo, (streamOutput, topQueriesHealthStats) -> topQueriesHealthStats.writeTo(out) ); + if (out.getVersion().onOrAfter(Version.V_2_19_0)) { + out.writeMap(fieldTypeCacheStats, StreamOutput::writeString, StreamOutput::writeLong); + } } /** @@ -133,4 +158,13 @@ public int getQueryRecordsQueueSize() { public Map getTopQueriesHealthStats() { return topQueriesHealthStats; } + + /** + * Get the field type cache stats. + * + * @return the field type cache stats + */ + public Map getFieldTypeCacheStats() { + return fieldTypeCacheStats; + } } diff --git a/src/test/java/org/opensearch/plugin/insights/core/service/QueryInsightsServiceTests.java b/src/test/java/org/opensearch/plugin/insights/core/service/QueryInsightsServiceTests.java index 33d89b29..d48757a9 100644 --- a/src/test/java/org/opensearch/plugin/insights/core/service/QueryInsightsServiceTests.java +++ b/src/test/java/org/opensearch/plugin/insights/core/service/QueryInsightsServiceTests.java @@ -12,6 +12,11 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.ENTRY_COUNT; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.EVICTIONS; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.HIT_COUNT; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.MISS_COUNT; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.SIZE_IN_BYTES; import java.util.List; import java.util.Map; @@ -25,6 +30,7 @@ import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.plugin.insights.QueryInsightsTestUtils; import org.opensearch.plugin.insights.core.metrics.OperationalMetricsCounter; +import org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator; import org.opensearch.plugin.insights.rules.model.GroupingType; import org.opensearch.plugin.insights.rules.model.MetricType; import org.opensearch.plugin.insights.rules.model.SearchQueryRecord; @@ -59,8 +65,9 @@ public void setup() { "QueryInsightsHealthStatsTests", new ScalingExecutorBuilder(QueryInsightsSettings.QUERY_INSIGHTS_EXECUTOR, 1, 5, TimeValue.timeValueMinutes(5)) ); + ClusterService clusterService = new ClusterService(settings, clusterSettings, threadPool); queryInsightsService = new QueryInsightsService( - new ClusterService(settings, clusterSettings, threadPool), + clusterService, threadPool, client, NoopMetricsRegistry.INSTANCE, @@ -69,6 +76,7 @@ public void setup() { queryInsightsService.enableCollection(MetricType.LATENCY, true); queryInsightsService.enableCollection(MetricType.CPU, true); queryInsightsService.enableCollection(MetricType.MEMORY, true); + queryInsightsService.setQueryShapeGenerator(new QueryShapeGenerator(clusterService)); queryInsightsServiceSpy = spy(queryInsightsService); MetricsRegistry metricsRegistry = mock(MetricsRegistry.class); @@ -207,5 +215,13 @@ public void testGetHealthStats() { assertTrue(topQueriesHealthStatsMap.containsKey(MetricType.LATENCY)); assertTrue(topQueriesHealthStatsMap.containsKey(MetricType.CPU)); assertTrue(topQueriesHealthStatsMap.containsKey(MetricType.MEMORY)); + Map fieldTypeCacheStats = healthStats.getFieldTypeCacheStats(); + assertNotNull(fieldTypeCacheStats); + assertEquals(5, fieldTypeCacheStats.size()); + assertTrue(fieldTypeCacheStats.containsKey(SIZE_IN_BYTES)); + assertTrue(fieldTypeCacheStats.containsKey(ENTRY_COUNT)); + assertTrue(fieldTypeCacheStats.containsKey(EVICTIONS)); + assertTrue(fieldTypeCacheStats.containsKey(HIT_COUNT)); + assertTrue(fieldTypeCacheStats.containsKey(MISS_COUNT)); } } diff --git a/src/test/java/org/opensearch/plugin/insights/rules/action/health_stats/HealthStatsNodeResponseTests.java b/src/test/java/org/opensearch/plugin/insights/rules/action/health_stats/HealthStatsNodeResponseTests.java index 4185a4c8..4d6e0142 100644 --- a/src/test/java/org/opensearch/plugin/insights/rules/action/health_stats/HealthStatsNodeResponseTests.java +++ b/src/test/java/org/opensearch/plugin/insights/rules/action/health_stats/HealthStatsNodeResponseTests.java @@ -53,6 +53,7 @@ public void setup() { this.healthStats = new QueryInsightsHealthStats( threadPool.info(QueryInsightsSettings.QUERY_INSIGHTS_EXECUTOR), 10, + new HashMap<>(), new HashMap<>() ); } diff --git a/src/test/java/org/opensearch/plugin/insights/rules/action/health_stats/HealthStatsResponseTests.java b/src/test/java/org/opensearch/plugin/insights/rules/action/health_stats/HealthStatsResponseTests.java index 28ca22ac..3dddad20 100644 --- a/src/test/java/org/opensearch/plugin/insights/rules/action/health_stats/HealthStatsResponseTests.java +++ b/src/test/java/org/opensearch/plugin/insights/rules/action/health_stats/HealthStatsResponseTests.java @@ -58,6 +58,7 @@ public void setup() { this.healthStats = new QueryInsightsHealthStats( threadPool.info(QueryInsightsSettings.QUERY_INSIGHTS_EXECUTOR), 10, + new HashMap<>(), new HashMap<>() ); } @@ -113,7 +114,7 @@ public void testToXContent() throws IOException { XContentBuilder builder = XContentFactory.jsonBuilder(); response.toXContent(builder, ToXContent.EMPTY_PARAMS); String expectedJson = - "{\"node_for_health_stats_test\":{\"ThreadPoolInfo\":{\"query_insights_executor\":{\"type\":\"scaling\",\"core\":1,\"max\":5,\"keep_alive\":\"5m\",\"queue_size\":-1}},\"QueryRecordsQueueSize\":10,\"TopQueriesHealthStats\":{}}}"; + "{\"node_for_health_stats_test\":{\"ThreadPoolInfo\":{\"query_insights_executor\":{\"type\":\"scaling\",\"core\":1,\"max\":5,\"keep_alive\":\"5m\",\"queue_size\":-1}},\"QueryRecordsQueueSize\":10,\"TopQueriesHealthStats\":{},\"FieldTypeCacheStats\":{\"size_in_bytes\":0,\"entry_count\":0,\"evictions\":0,\"hit_count\":0,\"miss_count\":0}}}"; assertEquals(expectedJson, builder.toString()); } } diff --git a/src/test/java/org/opensearch/plugin/insights/rules/model/healthStats/QueryInsightsHealthStatsTests.java b/src/test/java/org/opensearch/plugin/insights/rules/model/healthStats/QueryInsightsHealthStatsTests.java index ac4e8223..c9fba035 100644 --- a/src/test/java/org/opensearch/plugin/insights/rules/model/healthStats/QueryInsightsHealthStatsTests.java +++ b/src/test/java/org/opensearch/plugin/insights/rules/model/healthStats/QueryInsightsHealthStatsTests.java @@ -8,6 +8,12 @@ package org.opensearch.plugin.insights.rules.model.healthStats; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.ENTRY_COUNT; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.EVICTIONS; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.HIT_COUNT; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.MISS_COUNT; +import static org.opensearch.plugin.insights.core.service.categorizer.QueryShapeGenerator.SIZE_IN_BYTES; + import java.io.IOException; import java.util.HashMap; import java.util.Map; @@ -34,6 +40,7 @@ public class QueryInsightsHealthStatsTests extends OpenSearchTestCase { private ThreadPool.Info threadPoolInfo; private int queryRecordsQueueSize; private Map topQueriesHealthStats; + private Map fieldTypeCacheStats; @Before public void setUpQueryInsightsHealthStats() { @@ -45,6 +52,7 @@ public void setUpQueryInsightsHealthStats() { queryRecordsQueueSize = 100; topQueriesHealthStats = new HashMap<>(); topQueriesHealthStats.put(MetricType.LATENCY, new TopQueriesHealthStats(10, new QueryGrouperHealthStats(20, 15))); + fieldTypeCacheStats = Map.of(HIT_COUNT, 5L, MISS_COUNT, 3L, EVICTIONS, 1L, ENTRY_COUNT, 4L, SIZE_IN_BYTES, 300L); } @Override @@ -54,7 +62,12 @@ public void tearDown() throws Exception { } public void testConstructorAndGetters() { - QueryInsightsHealthStats healthStats = new QueryInsightsHealthStats(threadPoolInfo, queryRecordsQueueSize, topQueriesHealthStats); + QueryInsightsHealthStats healthStats = new QueryInsightsHealthStats( + threadPoolInfo, + queryRecordsQueueSize, + topQueriesHealthStats, + fieldTypeCacheStats + ); assertNotNull(healthStats); assertEquals(threadPoolInfo, healthStats.getThreadPoolInfo()); assertEquals(queryRecordsQueueSize, healthStats.getQueryRecordsQueueSize()); @@ -62,7 +75,12 @@ public void testConstructorAndGetters() { } public void testSerialization() throws IOException { - QueryInsightsHealthStats healthStats = new QueryInsightsHealthStats(threadPoolInfo, queryRecordsQueueSize, topQueriesHealthStats); + QueryInsightsHealthStats healthStats = new QueryInsightsHealthStats( + threadPoolInfo, + queryRecordsQueueSize, + topQueriesHealthStats, + fieldTypeCacheStats + ); // Write to StreamOutput BytesStreamOutput out = new BytesStreamOutput(); healthStats.writeTo(out); @@ -75,7 +93,12 @@ public void testSerialization() throws IOException { } public void testToXContent() throws IOException { - QueryInsightsHealthStats healthStats = new QueryInsightsHealthStats(threadPoolInfo, queryRecordsQueueSize, topQueriesHealthStats); + QueryInsightsHealthStats healthStats = new QueryInsightsHealthStats( + threadPoolInfo, + queryRecordsQueueSize, + topQueriesHealthStats, + fieldTypeCacheStats + ); XContentBuilder builder = XContentFactory.jsonBuilder(); builder.startObject(); @@ -100,6 +123,13 @@ public void testToXContent() throws IOException { + " \"QueryGroupCount_Total\": 20,\n" + " \"QueryGroupCount_MaxHeap\": 15\n" + " }\n" + + " },\n" + + " \"FieldTypeCacheStats\": {\n" + + " \"size_in_bytes\": 300,\n" + + " \"entry_count\": 4,\n" + + " \"evictions\": 1,\n" + + " \"hit_count\": 5,\n" + + " \"miss_count\": 3\n" + " }\n" + "}"; assertEquals(expectedJson.replaceAll("\\s", ""), jsonOutput.replaceAll("\\s", ""));