diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/NoOpReadResponseStats.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/NoOpReadResponseStats.java index 22922a51c13..862326280e2 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/NoOpReadResponseStats.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/NoOpReadResponseStats.java @@ -80,4 +80,9 @@ public void setStorageExecutionQueueLen(int storageExecutionQueueLen) { public void incrementMultiChunkLargeValueCount() { } + + @Override + public void incrementKeyNotFoundCount() { + + } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/ReadResponseStats.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/ReadResponseStats.java index 191d926be73..9f175fde15d 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/ReadResponseStats.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/ReadResponseStats.java @@ -45,4 +45,6 @@ public interface ReadResponseStats { void setStorageExecutionQueueLen(int storageExecutionQueueLen); void incrementMultiChunkLargeValueCount(); + + void incrementKeyNotFoundCount(); } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/storagenode/KeyNotFoundMetricIntegrationTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/storagenode/KeyNotFoundMetricIntegrationTest.java new file mode 100644 index 00000000000..7154e828952 --- /dev/null +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/storagenode/KeyNotFoundMetricIntegrationTest.java @@ -0,0 +1,132 @@ +package com.linkedin.venice.storagenode; + +import com.linkedin.venice.client.store.AvroGenericStoreClient; +import com.linkedin.venice.client.store.ClientConfig; +import com.linkedin.venice.client.store.ClientFactory; +import com.linkedin.venice.controllerapi.UpdateStoreQueryParams; +import com.linkedin.venice.integration.utils.ServiceFactory; +import com.linkedin.venice.integration.utils.VeniceClusterCreateOptions; +import com.linkedin.venice.integration.utils.VeniceClusterWrapper; +import com.linkedin.venice.tehuti.MetricsUtils; +import com.linkedin.venice.utils.TestUtils; +import com.linkedin.venice.utils.Utils; +import java.util.AbstractMap; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + + +public class KeyNotFoundMetricIntegrationTest { + private VeniceClusterWrapper veniceCluster; + private String storeName; + private AvroGenericStoreClient client; + + @BeforeClass + public void setUp() { + veniceCluster = ServiceFactory.getVeniceCluster(new VeniceClusterCreateOptions.Builder().build()); + storeName = Utils.getUniqueString("test-store"); + String valueSchemaStr = "{" + " \"type\": \"record\"," + " \"name\": \"User\"," + " \"fields\": [" + + " {\"name\": \"name\", \"type\": \"string\"}" + " ]" + "}"; + veniceCluster.useControllerClient(controllerClient -> { + controllerClient.createNewStore(storeName, "owner", "\"string\"", valueSchemaStr); + controllerClient.updateStore(storeName, new UpdateStoreQueryParams().setReadComputationEnabled(true)); + }); + + Schema valueSchema = Schema.parse(valueSchemaStr); + GenericRecord value1 = new GenericData.Record(valueSchema); + value1.put("name", "value1"); + + veniceCluster.createVersion( + storeName, + "\"string\"", + valueSchemaStr, + Stream.of(new AbstractMap.SimpleEntry<>("key1", value1))); + + client = ClientFactory.getAndStartGenericAvroClient( + ClientConfig.defaultGenericClientConfig(storeName).setVeniceURL(veniceCluster.getRandomRouterURL())); + } + + @AfterClass + public void cleanUp() { + Utils.closeQuietlyWithErrorLogged(client); + Utils.closeQuietlyWithErrorLogged(veniceCluster); + } + + @Test(singleThreaded = true) + public void testKeyNotFoundMetric() { + // Single Get - Key Found + try { + client.get("key1").get(); + } catch (Exception e) { + Assert.fail("Get failed", e); + } + + // Single Get - Key Not Found + try { + client.get("key2").get(); + } catch (Exception e) { + Assert.fail("Get failed", e); + } + + // Multi Get - 1 Key Found, 1 Key Not Found + Set keys = new HashSet<>(); + keys.add("key1"); + keys.add("key3"); + try { + client.batchGet(keys).get(); + } catch (Exception e) { + Assert.fail("Batch Get failed", e); + } + + // Compute - 1 Key Found, 1 Key Not Found + try { + client.compute().project("name").execute(keys).get(); + } catch (Exception e) { + Assert.fail("Compute failed", e); + } + + String singleGetMetricName = "." + storeName + "--key_not_found.Rate"; + String multiGetMetricName = "." + storeName + "--multiget_key_not_found.Rate"; + String computeMetricName = "." + storeName + "--compute_key_not_found.Rate"; + String totalSingleGetMetricName = ".total--key_not_found.Rate"; + String totalMultiGetMetricName = ".total--multiget_key_not_found.Rate"; + String totalComputeMetricName = ".total--compute_key_not_found.Rate"; + + TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, () -> { + double singleGetNotFoundRate = MetricsUtils.getMax(singleGetMetricName, veniceCluster.getVeniceServers()); + double multiGetNotFoundRate = MetricsUtils.getMax(multiGetMetricName, veniceCluster.getVeniceServers()); + double computeNotFoundRate = MetricsUtils.getMax(computeMetricName, veniceCluster.getVeniceServers()); + double totalSingleGetNotFoundRate = + MetricsUtils.getMax(totalSingleGetMetricName, veniceCluster.getVeniceServers()); + double totalMultiGetNotFoundRate = MetricsUtils.getMax(totalMultiGetMetricName, veniceCluster.getVeniceServers()); + double totalComputeNotFoundRate = MetricsUtils.getMax(totalComputeMetricName, veniceCluster.getVeniceServers()); + + Assert.assertTrue( + singleGetNotFoundRate > 0, + "Single Get key_not_found metric should be positive. Metric: " + singleGetMetricName); + Assert.assertTrue( + multiGetNotFoundRate > 0, + "Multi Get key_not_found metric should be positive. Metric: " + multiGetMetricName); + Assert.assertTrue( + computeNotFoundRate > 0, + "Compute key_not_found metric should be positive. Metric: " + computeMetricName); + Assert.assertTrue( + totalSingleGetNotFoundRate > 0, + "Total Single Get key_not_found metric should be positive. Metric: " + totalSingleGetMetricName); + Assert.assertTrue( + totalMultiGetNotFoundRate > 0, + "Total Multi Get key_not_found metric should be positive. Metric: " + totalMultiGetMetricName); + Assert.assertTrue( + totalComputeNotFoundRate > 0, + "Total Compute key_not_found metric should be positive. Metric: " + totalComputeMetricName); + }); + } +} diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/StorageReadRequestHandler.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/StorageReadRequestHandler.java index 49cdd4162f2..4ffb4540f9c 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/StorageReadRequestHandler.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/StorageReadRequestHandler.java @@ -491,6 +491,10 @@ public CompletableFuture handleSingleGetRequest(GetRouterRequest r SingleGetChunkingAdapter.get(storageEngine, request.getPartition(), key, isChunked, response.getStats()); response.setValueRecord(valueRecord); + if (valueRecord == null) { + response.getStats().incrementKeyNotFoundCount(); + } + response.getStats().addKeySize(key.length); response.getStats().setStorageExecutionSubmissionWaitTime(submissionWaitTime); response.getStats().setStorageExecutionQueueLen(queueLen); @@ -587,6 +591,7 @@ record = BatchGetChunkingAdapter.get( requestContext.isChunked, response.getStats()); if (record == null) { + response.getStats().incrementKeyNotFoundCount(); if (requestContext.isStreaming) { // For streaming, we would like to send back non-existing keys since the end-user won't know the status of // non-existing keys in the response if the response is partial. @@ -783,13 +788,16 @@ record = new ComputeResponseRecordV1(); response.addRecord(record); hits++; - } else if (requestContext.isStreaming) { - // For streaming, we need to send back non-existing keys - record = new ComputeResponseRecordV1(); - // Negative key index to indicate non-existing key - record.keyIndex = Math.negateExact(key.getKeyIndex()); - record.value = StreamingUtils.EMPTY_BYTE_BUFFER; - response.addRecord(record); + } else { + response.getStats().incrementKeyNotFoundCount(); + if (requestContext.isStreaming) { + // For streaming, we need to send back non-existing keys + record = new ComputeResponseRecordV1(); + // Negative key index to indicate non-existing key + record.keyIndex = Math.negateExact(key.getKeyIndex()); + record.value = StreamingUtils.EMPTY_BYTE_BUFFER; + response.addRecord(record); + } } } diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/AbstractReadResponseStats.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/AbstractReadResponseStats.java index af18eee9634..fdd9fe0951b 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/AbstractReadResponseStats.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/AbstractReadResponseStats.java @@ -30,6 +30,7 @@ public abstract class AbstractReadResponseStats implements ReadResponseStats, Re private double storageExecutionSubmissionWaitTime = UNINITIALIZED; private int storageExecutionQueueLen = UNINITIALIZED; protected int multiChunkLargeValueCount = 0; + private int keyNotFoundCount = 0; protected abstract int getRecordCount(); @@ -57,6 +58,15 @@ public void incrementMultiChunkLargeValueCount() { this.multiChunkLargeValueCount++; } + @Override + public void incrementKeyNotFoundCount() { + this.keyNotFoundCount++; + } + + public int getKeyNotFoundCount() { + return this.keyNotFoundCount; + } + @Override public void recordMetrics(ServerHttpRequestStats stats) { consumeDoubleAndBooleanIfAbove( @@ -66,6 +76,7 @@ public void recordMetrics(ServerHttpRequestStats stats) { 0); consumeIntIfAbove(stats::recordMultiChunkLargeValueCount, this.multiChunkLargeValueCount, 0); consumeIntIfAbove(stats::recordStorageExecutionQueueLen, this.storageExecutionQueueLen, UNINITIALIZED); + consumeIntIfAbove(stats::recordKeyNotFoundCount, this.keyNotFoundCount, 0); recordUnmergedMetrics(stats); @@ -86,6 +97,7 @@ public void merge(ReadResponseStatsRecorder other) { AbstractReadResponseStats otherStats = (AbstractReadResponseStats) other; this.databaseLookupLatency += otherStats.databaseLookupLatency; this.multiChunkLargeValueCount += otherStats.multiChunkLargeValueCount; + this.keyNotFoundCount += otherStats.keyNotFoundCount; } } diff --git a/services/venice-server/src/main/java/com/linkedin/venice/stats/AggServerHttpRequestStats.java b/services/venice-server/src/main/java/com/linkedin/venice/stats/AggServerHttpRequestStats.java index d3eadfb5ca9..aae7ff62fc0 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/stats/AggServerHttpRequestStats.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/stats/AggServerHttpRequestStats.java @@ -75,4 +75,8 @@ public void recordErrorRequestLatency(double latency) { public void recordMisroutedStoreVersionRequest() { totalStats.recordMisroutedStoreVersionRequest(); } + + public void recordKeyNotFoundCount(int count) { + totalStats.recordKeyNotFoundCount(count); + } } diff --git a/services/venice-server/src/main/java/com/linkedin/venice/stats/ServerHttpRequestStats.java b/services/venice-server/src/main/java/com/linkedin/venice/stats/ServerHttpRequestStats.java index 065675c0317..584a173f376 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/stats/ServerHttpRequestStats.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/stats/ServerHttpRequestStats.java @@ -28,6 +28,7 @@ public class ServerHttpRequestStats extends AbstractVeniceHttpStats { private final Sensor databaseLookupLatencyForLargeValueSensor; private final Sensor multiChunkLargeValueCountSensor; private final Sensor requestKeyCountSensor; + private final Sensor keyNotFoundSensor; private final Sensor requestSizeInBytesSensor; private final Sensor storageExecutionHandlerSubmissionWaitTime; private final Sensor storageExecutionQueueLenSensor; @@ -183,6 +184,8 @@ public ServerHttpRequestStats( } else { requestKeyCountSensor = null; } + keyNotFoundSensor = + registerPerStoreAndTotal("key_not_found", totalStats, () -> totalStats.keyNotFoundSensor, new Rate()); requestSizeInBytesSensor = registerPerStoreAndTotal( "request_size_in_bytes", totalStats, @@ -357,6 +360,10 @@ public void recordRequestKeyCount(int keyCount) { } } + public void recordKeyNotFoundCount(int count) { + keyNotFoundSensor.record(count); + } + public void recordRequestSizeInBytes(int requestSizeInBytes) { requestSizeInBytesSensor.record(requestSizeInBytes); } diff --git a/services/venice-server/src/test/java/com/linkedin/venice/listener/StorageReadRequestHandlerTest.java b/services/venice-server/src/test/java/com/linkedin/venice/listener/StorageReadRequestHandlerTest.java index e9868573c1e..d94b667e456 100644 --- a/services/venice-server/src/test/java/com/linkedin/venice/listener/StorageReadRequestHandlerTest.java +++ b/services/venice-server/src/test/java/com/linkedin/venice/listener/StorageReadRequestHandlerTest.java @@ -9,6 +9,7 @@ import static org.mockito.ArgumentMatchers.intThat; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; @@ -68,7 +69,9 @@ import com.linkedin.venice.listener.response.ComputeResponseWrapper; import com.linkedin.venice.listener.response.HttpShortcutResponse; import com.linkedin.venice.listener.response.MultiGetResponseWrapper; +import com.linkedin.venice.listener.response.MultiKeyResponseWrapper; import com.linkedin.venice.listener.response.SingleGetResponseWrapper; +import com.linkedin.venice.listener.response.stats.AbstractReadResponseStats; import com.linkedin.venice.listener.response.stats.MultiKeyResponseStats; import com.linkedin.venice.listener.response.stats.ReadResponseStatsRecorder; import com.linkedin.venice.meta.PartitionerConfig; @@ -210,6 +213,7 @@ public void setUp() { doReturn(partitionerConfig).when(version).getPartitionerConfig(); doReturn(storageEngine).when(storageEngineRepository).getLocalStorageEngine(any()); + doReturn(true).when(storeRepository).isReadComputationEnabled(any()); doReturn(new NoopCompressor()).when(compressorFactory).getCompressor(any(), any(), anyInt()); RocksDBServerConfig rocksDBServerConfig = mock(RocksDBServerConfig.class); @@ -903,6 +907,150 @@ public void testNoStorageEngineReturn503() throws Exception { Assert.assertEquals(shortcutResponseArgumentCaptor.getValue().getStatus(), BAD_REQUEST); } + @Test + public void testSingleGetWithKeyNotFound() throws Exception { + String keyString = "missing-key"; + int partition = 2; + doReturn(null).when(storageEngine).get(partition, ByteBuffer.wrap(keyString.getBytes())); + + // [0]""/[1]"action"/[2]"store"/[3]"partition"/[4]"key" + String uri = "/" + TYPE_STORAGE + "/test-topic_v1/" + partition + "/" + keyString; + HttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri); + GetRouterRequest request = + GetRouterRequest.parseGetHttpRequest(httpRequest, RequestHelper.getRequestParts(URI.create(httpRequest.uri()))); + + StorageReadRequestHandler requestHandler = createStorageReadRequestHandler(); + requestHandler.channelRead(context, request); + + verify(context, times(1)).writeAndFlush(argumentCaptor.capture()); + SingleGetResponseWrapper responseObject = (SingleGetResponseWrapper) argumentCaptor.getValue(); + Assert.assertNull(responseObject.getValueRecord()); + assertEquals(((AbstractReadResponseStats) responseObject.getStats()).getKeyNotFoundCount(), 1); + } + + @Test + public void testMultiGetWithKeyNotFound() throws Exception { + int recordCount = 10; + int missingRecordCount = 3; + RecordSerializer serializer = + SerializerDeserializerFactory.getAvroGenericSerializer(MultiGetRouterRequestKeyV1.SCHEMA$); + List keys = new ArrayList<>(); + String stringSchema = "\"string\""; + VeniceKafkaSerializer keySerializer = new VeniceAvroKafkaSerializer(stringSchema); + + for (int i = 0; i < recordCount; ++i) { + MultiGetRouterRequestKeyV1 requestKey = new MultiGetRouterRequestKeyV1(); + String keyString = "key_" + i; + byte[] keyBytes = keySerializer.serialize(null, keyString); + requestKey.keyBytes = ByteBuffer.wrap(keyBytes); + requestKey.keyIndex = i; + requestKey.partitionId = 0; + + if (i < (recordCount - missingRecordCount)) { + String valueString = "value_" + i; + byte[] valueBytes = ValueRecord.create(1, valueString.getBytes()).serialize(); + doReturn(valueBytes).when(storageEngine).get(eq(0), eq(requestKey.keyBytes)); + } else { + doReturn(null).when(storageEngine).get(eq(0), eq(requestKey.keyBytes)); + } + keys.add(requestKey); + } + + String uri = "/" + TYPE_STORAGE + "/test-topic_v1"; + FullHttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, uri); + httpRequest.headers() + .set( + HttpConstants.VENICE_API_VERSION, + ReadAvroProtocolDefinition.MULTI_GET_ROUTER_REQUEST_V1.getProtocolVersion()); + httpRequest.content().writeBytes(serializer.serializeObjects(keys)); + MultiGetRouterRequestWrapper request = MultiGetRouterRequestWrapper + .parseMultiGetHttpRequest(httpRequest, RequestHelper.getRequestParts(URI.create(uri))); + + StorageReadRequestHandler requestHandler = createStorageReadRequestHandler(); + requestHandler.channelRead(context, request); + + verify(context, timeout(1000).times(1)).writeAndFlush(argumentCaptor.capture()); + MultiKeyResponseWrapper responseObject = (MultiKeyResponseWrapper) argumentCaptor.getValue(); + assertEquals(((AbstractReadResponseStats) responseObject.getStats()).getKeyNotFoundCount(), missingRecordCount); + } + + @Test + public void testComputeWithKeyNotFound() throws Exception { + int recordCount = 2; + int missingRecordCount = 1; + + String valueSchemaStr = "{" + " \"type\": \"record\"," + " \"name\": \"User\"," + " \"fields\": [" + + " {\"name\": \"name\", \"type\": \"string\"}" + " ]" + "}"; + Schema valueSchema = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(valueSchemaStr); + SchemaEntry valueSchemaEntry = new SchemaEntry(1, valueSchema); + doReturn(valueSchemaEntry).when(schemaRepository).getValueSchema(any(), anyInt()); + + RecordSerializer valueSerializer = + SerializerDeserializerFactory.getAvroGenericSerializer(valueSchema); + + String key1Str = "key_1"; + ByteBuffer key1Bytes = ByteBuffer.wrap(key1Str.getBytes()); + ComputeRouterRequestKeyV1 requestKey1 = new ComputeRouterRequestKeyV1(0, key1Bytes, 0); + + String key2Str = "key_2"; + ByteBuffer key2Bytes = ByteBuffer.wrap(key2Str.getBytes()); + ComputeRouterRequestKeyV1 requestKey2 = new ComputeRouterRequestKeyV1(1, key2Bytes, 0); + + List keys = Arrays.asList(requestKey1, requestKey2); + + GenericRecord value = new GenericData.Record(valueSchema); + value.put("name", "name_1"); + byte[] valueBytes = ValueRecord.create(1, valueSerializer.serialize(value)).serialize(); + + byte[] key1BytesArray = key1Str.getBytes(); + + doAnswer(invocation -> { + byte[] bytes = invocation.getArgument(1); + if (Arrays.equals(bytes, key1BytesArray)) { + return valueBytes; + } + return null; + }).when(storageEngine).get(eq(0), any(byte[].class)); + + doAnswer(invocation -> { + byte[] bytes = invocation.getArgument(1); + if (Arrays.equals(bytes, key1BytesArray)) { + return ByteBuffer.wrap(valueBytes); + } + return null; + }).when(storageEngine).get(eq(0), any(byte[].class), any(ByteBuffer.class)); + + doAnswer(invocation -> { + ByteBuffer bb = invocation.getArgument(1); + if (bb != null && bb.equals(key1Bytes)) { + return valueBytes; + } + return null; + }).when(storageEngine).get(eq(0), any(ByteBuffer.class)); + + ComputeRequest computeRequest = new ComputeRequest(); + computeRequest.setOperations(Collections.emptyList()); + computeRequest.setResultSchemaStr(new org.apache.avro.util.Utf8(valueSchemaStr)); + + ComputeRouterRequestWrapper request = mock(ComputeRouterRequestWrapper.class); + doReturn(keys).when(request).getKeys(); + doReturn(recordCount).when(request).getKeyCount(); + doReturn("test-store_v1").when(request).getResourceName(); + doReturn("test-store").when(request).getStoreName(); + doReturn(RequestType.COMPUTE).when(request).getRequestType(); + doReturn(computeRequest).when(request).getComputeRequest(); + doReturn(1).when(request).getValueSchemaId(); + doReturn(false).when(request).shouldRequestBeTerminatedEarly(); + doReturn(false).when(request).isStreamingRequest(); + + StorageReadRequestHandler requestHandler = createStorageReadRequestHandler(); + requestHandler.channelRead(context, request); + + verify(context, timeout(1000).times(1)).writeAndFlush(argumentCaptor.capture()); + MultiKeyResponseWrapper responseObject = (MultiKeyResponseWrapper) argumentCaptor.getValue(); + assertEquals(((AbstractReadResponseStats) responseObject.getStats()).getKeyNotFoundCount(), missingRecordCount); + } + private SchemaReader getMockSchemaReader(Schema keySchema, Schema valueSchema) { SchemaReader schemaReader = mock(SchemaReader.class); doReturn(keySchema).when(schemaReader).getKeySchema(); diff --git a/services/venice-server/src/test/java/com/linkedin/venice/stats/AggServerHttpRequestStatsTest.java b/services/venice-server/src/test/java/com/linkedin/venice/stats/AggServerHttpRequestStatsTest.java index 75b444aca13..8ae44d4e367 100644 --- a/services/venice-server/src/test/java/com/linkedin/venice/stats/AggServerHttpRequestStatsTest.java +++ b/services/venice-server/src/test/java/com/linkedin/venice/stats/AggServerHttpRequestStatsTest.java @@ -107,10 +107,17 @@ public void testMetrics() { computeServerStatsFoo.recordCosineSimilarityCount(10); computeServerStatsFoo.recordHadamardProduct(10); computeServerStatsFoo.recordCountOperator(10); + computeServerStatsFoo.recordKeyNotFoundCount(5); Assert.assertTrue( reporter.query("." + STORE_FOO + "--success_request.OccurrenceRate").value() > 0, "success_request rate should be positive"); + Assert.assertTrue( + reporter.query("." + STORE_FOO + "--compute_key_not_found.Rate").value() > 0, + "compute_key_not_found rate should be positive"); + Assert.assertTrue( + reporter.query(".total--compute_key_not_found.Rate").value() > 0, + "total compute_key_not_found rate should be positive"); Assert.assertTrue( reporter.query(".total--error_request.OccurrenceRate").value() > 0, "error_request rate should be positive");