From a0d5909fefe52a7ad6b90658455cb4724ccbf8b3 Mon Sep 17 00:00:00 2001 From: Felix GV Date: Thu, 5 Sep 2024 17:04:03 -0400 Subject: [PATCH] [server][dvc] Server request/response handling overhaul (#1152) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This refactors many parts of server request and response handling, including how metrics are recorded. The significant functional changes are: 1. Parallel batch gets no longer have a race condition in the way metrics are recorded. Previously, the metrics which aggregate a value for all keys in the query could be missing data points and thus under-report due to this race. 2. The configs to enable and tune parallel batch gets now apply to compute as well, which was not supported before. 3. K/V size profiling is now always enabled for single gets, since it is considered cheap enough for that workload. The config still controls whether it is enabled or not for batch get and compute. Non-functional changes include: 1. The elimination of locking in parallel query processing. 2. The serialization of responses to bytes now happens inside each subtask, rather than sequentially after all subtasks are completed. These are then presented as a CompositeByteBuf, which may result in smaller allocations since there is no need for one large byte[] holding the whole response anymore. 3. The objects used to record stats are now specialized for each query type (single get, batch get, compute) as well as for whether K/V size profiling is enabled or not, leading to a reduction of memory footprint for workloads that don’t need certain stats. See the class hierarchy details below for more info. 4. As much as possible, when some behavior is fixed for the whole duration of the server’s runtime, a property is allocated to hold the closure to be executed according to the relevant config, rather than evaluating the immutable config in the hot path. Similary, removed as much branching as possible from ChunkingUtils and related classes. Below are some of the implementation details for achieving the above changes as well as to clean up some tech debt. The ReadResponse is now split into two interfaces, each of which has its own hierarchy of implementations: - ReadResponse, which holds the state required to achieve the functional goals of the server, as seen by the querier (i.e. populating the payload and headers). This is now moved from the DVC module to the server module. The following subclass hierarchy exists: - AbstractReadResponse: holds common functional state and behavior for all read requests. - SingleGetResponseWrapper: renamed from StorageResponseObject. - MultiKeyResponseWrapper: used for single-threaded batch get requests as well as subtasks of parallel batch get requests. - ComputeResponseWrapper: used for single-threaded compute requests as well as subtasks of parallel compute requests. - ParallelMultiKeyResponseWrapper: used in parallel batch get & compute requests to hold an array of MultiKeyResponseWrapper, each of which are used exclusively by one subtask. This class is then responsible for aggregating the results of all these wrappers. - ReadResponseStats: holds the state required for recording metrics, after the response is flushed. This replaces the ReadResponse parameter passed into all chunking-related utilities, thus more clearly constraining the role of that parameter in those classes. The purpose of this interface is to accumulate stats, but it doesn’t offer any API for reading or using those stats (see below how that part is done). The following subclass hierarchy exists: - AbstractReadResponseStats: holds the common metrics state reported for all read responses. The class also implements ReadResponseStatsRecorder, a new interface which is responsible for recording the accumulated metric state. Each subclass is responsible for defining the recording logic pertaining to the extra state it carries. - MultiKeyResponseStats: holds the record count. - MultiGetResponseStatsWithSizeProfiling: holds K/V sizes for each record. - ComputeResponseStats: holds nine primitives used exclusively in read compute stats. - ComputeResponseStatsWithSizeProfiling: holds K/V sizes for each record. - NoOpReadResponseStats: a singleton intended to be passed into the various ChunkingUtils APIs by code paths which do not need any stats recorded via this interface. This allows the ChunkingUtils code to assume this parameter is not null and systematically call whatever APIs it needs, rather than null-checking on the hot path. Miscellaneous main code changes: - MultiKeyRouterRequestWrapper::getKeys now returns a List rather than an Iterable. - DaVinciBackend::getStoreOrThrow now makes use of CHM::computeIfAbsent, rather than syncrhonized. This helped stabilize an integration test. - Muted a common stacktrace in IsolatedIngestionUtils. - ThinClientMetaStoreBasedRepository::getStoreMetaValue now has a lower timeout and retries, rather than trying only one request with the default timeout of 10 seconds. Total time spent should be the same. - Refactored ReplicationMetadataRocksDBStoragePartition so that the getReplicationMetadata API takes a ByteBuffer param, rather than byte[], to bring it in line with the RocksDBStoragePartition::get API. - Discovered and fixed an unrelated bug in the ChunkingUtils' getValueAndSchemaIdFromStorage API, where it would carry the ChunkValueManifest's special schema ID rather than the real value's schema ID, when chunking kicks in. --- .../com/linkedin/davinci/DaVinciBackend.java | 9 +- .../com/linkedin/davinci/VersionBackend.java | 6 +- .../utils/IsolatedIngestionUtils.java | 5 +- .../ActiveActiveStoreIngestionTask.java | 2 +- .../LeaderFollowerStoreIngestionTask.java | 3 +- .../response/NoOpReadResponseStats.java | 83 +++ .../listener/response/ReadResponse.java | 197 +---- .../listener/response/ReadResponseStats.java | 48 ++ .../ThinClientMetaStoreBasedRepository.java | 36 +- .../storage/DiskHealthCheckService.java | 2 +- .../chunking/AbstractAvroChunkingAdapter.java | 66 +- .../chunking/BatchGetChunkingAdapter.java | 4 +- .../storage/chunking/ChunkingAdapter.java | 18 +- .../storage/chunking/ChunkingUtils.java | 158 ++-- .../chunking/SingleGetChunkingAdapter.java | 14 +- .../davinci/store/AbstractStorageEngine.java | 2 +- .../store/AbstractStoragePartition.java | 2 +- ...cationMetadataRocksDBStoragePartition.java | 10 +- .../davinci/utils/ChunkAssembler.java | 5 +- .../ActiveActiveStoreIngestionTaskTest.java | 15 +- .../consumer/StoreIngestionTaskTest.java | 4 +- .../storage/chunking/ChunkingTest.java | 34 +- ...onMetadataRocksDBStoragePartitionTest.java | 10 +- .../utils/IsolatedIngestionUtilsTest.java | 39 +- .../utils/DoubleAndBooleanConsumer.java | 5 + .../venice/endToEnd/DaVinciClientTest.java | 45 +- .../venice/endToEnd/PartialUpdateTest.java | 7 +- ...entIndividualFeatureConfigurationTest.java | 6 +- .../IngestionHeartBeatTest.java | 10 +- .../linkedin/venice/router/TestStreaming.java | 4 +- .../storagenode/StorageNodeComputeTest.java | 15 +- .../com/linkedin/venice/utils/ValueSize.java | 11 + .../listener/OutboundHttpWrapperHandler.java | 25 +- .../venice/listener/ServerStatsContext.java | 233 +----- .../venice/listener/StatsHandler.java | 78 -- .../listener/StorageReadRequestHandler.java | 690 +++++++++--------- ...eniceRequestEarlyTerminationException.java | 6 +- .../GrpcStorageReadRequestHandler.java | 22 +- .../request/MultiKeyRouterRequestWrapper.java | 2 +- .../response/AbstractReadResponse.java | 58 ++ .../response/ComputeResponseWrapper.java | 19 +- .../response/MultiGetResponseWrapper.java | 10 +- .../response/MultiKeyResponseWrapper.java | 70 +- .../ParallelMultiKeyResponseWrapper.java | 134 ++++ ...ect.java => SingleGetResponseWrapper.java} | 27 +- .../stats/AbstractReadResponseStats.java | 165 +++++ .../response/stats/ComputeResponseStats.java | 110 +++ ...ComputeResponseStatsWithSizeProfiling.java | 41 ++ ...ultiGetResponseStatsWithSizeProfiling.java | 39 + .../response/stats/MultiKeyResponseStats.java | 31 + .../stats/ReadResponseStatsRecorder.java | 40 + .../response/stats/ResponseStatsUtil.java | 41 ++ .../stats/SingleGetResponseStats.java | 33 + .../stats/AggServerHttpRequestStats.java | 8 - .../venice/stats/ServerHttpRequestStats.java | 7 +- .../venice/grpc/ServerStatsContextTest.java | 35 +- .../StorageReadRequestHandlerTest.java | 207 +++++- .../response/MultiKeyResponseWrapperTest.java | 101 +++ .../stats/ReadResponseLatencyInjector.java | 12 + .../response/stats/ResponseStatsUtilTest.java | 40 + .../stats/AggServerQuotaUsageStatsTest.java | 4 +- 61 files changed, 1986 insertions(+), 1177 deletions(-) create mode 100644 clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/NoOpReadResponseStats.java create mode 100644 clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/ReadResponseStats.java create mode 100644 internal/venice-client-common/src/main/java/com/linkedin/venice/utils/DoubleAndBooleanConsumer.java create mode 100644 internal/venice-test-common/src/main/java/com/linkedin/venice/utils/ValueSize.java create mode 100644 services/venice-server/src/main/java/com/linkedin/venice/listener/response/AbstractReadResponse.java create mode 100644 services/venice-server/src/main/java/com/linkedin/venice/listener/response/ParallelMultiKeyResponseWrapper.java rename services/venice-server/src/main/java/com/linkedin/venice/listener/response/{StorageResponseObject.java => SingleGetResponseWrapper.java} (53%) create mode 100644 services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/AbstractReadResponseStats.java create mode 100644 services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ComputeResponseStats.java create mode 100644 services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ComputeResponseStatsWithSizeProfiling.java create mode 100644 services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/MultiGetResponseStatsWithSizeProfiling.java create mode 100644 services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/MultiKeyResponseStats.java create mode 100644 services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ReadResponseStatsRecorder.java create mode 100644 services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ResponseStatsUtil.java create mode 100644 services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/SingleGetResponseStats.java create mode 100644 services/venice-server/src/test/java/com/linkedin/venice/listener/response/MultiKeyResponseWrapperTest.java create mode 100644 services/venice-server/src/test/java/com/linkedin/venice/listener/response/stats/ReadResponseLatencyInjector.java create mode 100644 services/venice-server/src/test/java/com/linkedin/venice/listener/response/stats/ResponseStatsUtilTest.java diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/DaVinciBackend.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/DaVinciBackend.java index 1ffce59d14..2d9819b251 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/DaVinciBackend.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/DaVinciBackend.java @@ -533,13 +533,8 @@ public synchronized void close() { } } - public synchronized StoreBackend getStoreOrThrow(String storeName) { - StoreBackend storeBackend = storeByNameMap.get(storeName); - if (storeBackend == null) { - storeBackend = new StoreBackend(this, storeName); - storeByNameMap.put(storeName, storeBackend); - } - return storeBackend; + public StoreBackend getStoreOrThrow(String storeName) { + return storeByNameMap.computeIfAbsent(storeName, s -> new StoreBackend(this, s)); } ScheduledExecutorService getExecutor() { diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/VersionBackend.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/VersionBackend.java index aa8685e20d..4c6718dec3 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/VersionBackend.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/VersionBackend.java @@ -5,6 +5,7 @@ import static com.linkedin.venice.ConfigKeys.SERVER_STOP_CONSUMPTION_TIMEOUT_IN_SECONDS; import com.linkedin.davinci.config.VeniceStoreVersionConfig; +import com.linkedin.davinci.listener.response.NoOpReadResponseStats; import com.linkedin.davinci.notifier.DaVinciPushStatusUpdateTask; import com.linkedin.davinci.storage.chunking.AbstractAvroChunkingAdapter; import com.linkedin.davinci.store.AbstractStorageEngine; @@ -215,7 +216,7 @@ public V read( reusableValue, binaryDecoder, version.isChunkingEnabled(), - null, + NoOpReadResponseStats.SINGLETON, readerSchemaId, storeDeserializerCache, compressor.get()); @@ -242,7 +243,7 @@ public GenericRecord compute( reusableValueRecord, binaryDecoder, version.isChunkingEnabled(), - null, + NoOpReadResponseStats.SINGLETON, readerSchemaId, storeDeserializerCache, compressor.get()); @@ -296,7 +297,6 @@ public void onCompletion(Optional exception) { reusableBinaryDecoder, keyRecordDeserializer, this.version.isChunkingEnabled(), - null, getSupersetOrLatestValueSchemaId(), this.storeDeserializerCache, this.compressor.get(), diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/utils/IsolatedIngestionUtils.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/utils/IsolatedIngestionUtils.java index 286db5009d..ef12ef6b8f 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/utils/IsolatedIngestionUtils.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/utils/IsolatedIngestionUtils.java @@ -343,10 +343,13 @@ public static String executeShellCommand(String command) { } catch (Exception e) { if (command.contains("kill")) { throw new VeniceException("Encountered exception when executing shell command: " + command, e); + } else if (e.getMessage().contains("No such file or directory")) { + // This is a common case, so we mute the full exception stacktrace. + LOGGER.info("Command not found. Exception message: {}", e.getMessage()); } else { LOGGER.info("Encounter exception when executing shell command: {}", command, e); - return ""; } + return ""; } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index 84ce40fbb8..59eafcd0e0 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -350,7 +350,7 @@ byte[] getRmdWithValueSchemaByteBufferFromStorage( long currentTimeForMetricsMs) { final long lookupStartTimeInNS = System.nanoTime(); ValueRecord result = SingleGetChunkingAdapter - .getReplicationMetadata(getStorageEngine(), partition, key, isChunked(), null, rmdManifestContainer); + .getReplicationMetadata(getStorageEngine(), partition, key, isChunked(), rmdManifestContainer); getHostLevelIngestionStats().recordIngestionReplicationMetadataLookUpLatency( LatencyUtils.getElapsedTimeFromNSToMS(lookupStartTimeInNS), currentTimeForMetricsMs); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 38222e2da7..b45066deba 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -19,6 +19,7 @@ import com.linkedin.davinci.config.VeniceStoreVersionConfig; import com.linkedin.davinci.helix.LeaderFollowerPartitionStateModel; import com.linkedin.davinci.ingestion.LagType; +import com.linkedin.davinci.listener.response.NoOpReadResponseStats; import com.linkedin.davinci.schema.merge.CollectionTimestampMergeRecordHelper; import com.linkedin.davinci.schema.merge.MergeRecordHelper; import com.linkedin.davinci.stats.ingestion.heartbeat.HeartbeatMonitoringService; @@ -3207,7 +3208,7 @@ private GenericRecord readStoredValueRecord( isChunked, null, null, - null, + NoOpReadResponseStats.SINGLETON, readerValueSchemaID, storeDeserializerCache, compressor.get(), 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 new file mode 100644 index 0000000000..22922a51c1 --- /dev/null +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/NoOpReadResponseStats.java @@ -0,0 +1,83 @@ +package com.linkedin.davinci.listener.response; + +public class NoOpReadResponseStats implements ReadResponseStats { + public static final NoOpReadResponseStats SINGLETON = new NoOpReadResponseStats(); + + private NoOpReadResponseStats() { + } + + @Override + public long getCurrentTimeInNanos() { + return 0; + } + + @Override + public void addDatabaseLookupLatency(long startTimeInNanos) { + + } + + @Override + public void addReadComputeLatency(double latency) { + + } + + @Override + public void addReadComputeDeserializationLatency(double latency) { + + } + + @Override + public void addReadComputeSerializationLatency(double latency) { + + } + + @Override + public void addKeySize(int size) { + + } + + @Override + public void addValueSize(int size) { + + } + + @Override + public void addReadComputeOutputSize(int size) { + + } + + @Override + public void incrementDotProductCount(int count) { + + } + + @Override + public void incrementCountOperatorCount(int count) { + + } + + @Override + public void incrementCosineSimilarityCount(int count) { + + } + + @Override + public void incrementHadamardProductCount(int count) { + + } + + @Override + public void setStorageExecutionSubmissionWaitTime(double storageExecutionSubmissionWaitTime) { + + } + + @Override + public void setStorageExecutionQueueLen(int storageExecutionQueueLen) { + + } + + @Override + public void incrementMultiChunkLargeValueCount() { + + } +} diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/ReadResponse.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/ReadResponse.java index b955943217..3d3703d80f 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/ReadResponse.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/ReadResponse.java @@ -2,207 +2,40 @@ import com.linkedin.venice.compression.CompressionStrategy; import io.netty.buffer.ByteBuf; -import it.unimi.dsi.fastutil.ints.IntList; /** - * This class is used to store common fields shared by various read responses. + * This is used to store common fields shared by various read responses. It is intended to store state required to + * fulfill the service's goals from the perspective of external callers (e.g. the Router or Fast Client). + * + * See also {@link #getStats()} which returns the container used to store state exclusively used by metrics. */ -public abstract class ReadResponse { - private double databaseLookupLatency = -1; - private double readComputeLatency = -1; - private double readComputeDeserializationLatency = -1; - private double readComputeSerializationLatency = -1; - private double storageExecutionSubmissionWaitTime; - private int storageExecutionQueueLen = -1; - private int multiChunkLargeValueCount = 0; - private CompressionStrategy compressionStrategy = CompressionStrategy.NO_OP; - private boolean isStreamingResponse = false; - private IntList keySizeList; - private IntList valueSizeList; - private int valueSize = 0; - private int readComputeOutputSize = 0; - private int dotProductCount = 0; - private int cosineSimilarityCount = 0; - private int hadamardProductCount = 0; - private int countOperatorCount = 0; - private int rcu = 0; +public interface ReadResponse { + ReadResponseStats getStats(); - public void setCompressionStrategy(CompressionStrategy compressionStrategy) { - this.compressionStrategy = compressionStrategy; - } + void setCompressionStrategy(CompressionStrategy compressionStrategy); - public void setStreamingResponse() { - this.isStreamingResponse = true; - } + void setStreamingResponse(); - public boolean isStreamingResponse() { - return this.isStreamingResponse; - } + boolean isStreamingResponse(); - public CompressionStrategy getCompressionStrategy() { - return compressionStrategy; - } - - public void setDatabaseLookupLatency(double latency) { - this.databaseLookupLatency = latency; - } - - public void addDatabaseLookupLatency(double latency) { - this.databaseLookupLatency += latency; - } - - public double getDatabaseLookupLatency() { - return this.databaseLookupLatency; - } - - public void setReadComputeLatency(double latency) { - this.readComputeLatency = latency; - } - - public void addReadComputeLatency(double latency) { - this.readComputeLatency += latency; - } - - public double getReadComputeLatency() { - return this.readComputeLatency; - } - - public void setReadComputeDeserializationLatency(double latency) { - this.readComputeDeserializationLatency = latency; - } - - public void addReadComputeDeserializationLatency(double latency) { - this.readComputeDeserializationLatency += latency; - } - - public void setKeySizeList(IntList keySizeList) { - this.keySizeList = keySizeList; - } - - public void setValueSizeList(IntList valueSizeList) { - this.valueSizeList = valueSizeList; - } - - public double getReadComputeDeserializationLatency() { - return this.readComputeDeserializationLatency; - } - - public void setReadComputeSerializationLatency(double latency) { - this.readComputeSerializationLatency = latency; - } - - public void addReadComputeSerializationLatency(double latency) { - this.readComputeSerializationLatency += latency; - } - - public void addValueSize(int size) { - this.valueSize += size; - } - - public int getValueSize() { - return valueSize; - } - - public void addReadComputeOutputSize(int size) { - this.readComputeOutputSize += size; - } - - public int getReadComputeOutputSize() { - return readComputeOutputSize; - } - - public void incrementDotProductCount(int count) { - dotProductCount += count; - } - - public void incrementCountOperatorCount(int count) { - countOperatorCount += count; - } - - public void incrementCosineSimilarityCount(int count) { - cosineSimilarityCount += count; - } - - public void incrementHadamardProductCount(int count) { - hadamardProductCount += count; - } - - public double getReadComputeSerializationLatency() { - return this.readComputeSerializationLatency; - } - - public double getStorageExecutionHandlerSubmissionWaitTime() { - return storageExecutionSubmissionWaitTime; - } - - public void setStorageExecutionSubmissionWaitTime(double storageExecutionSubmissionWaitTime) { - this.storageExecutionSubmissionWaitTime = storageExecutionSubmissionWaitTime; - } + CompressionStrategy getCompressionStrategy(); /** * Set the read compute unit (RCU) cost for this response's request * @param rcu */ - public void setRCU(int rcu) { - this.rcu = rcu; - } + void setRCU(int rcu); /** * Get the read compute unit (RCU) for this response's request * @return */ - public int getRCU() { - return this.rcu; - } - - public int getStorageExecutionQueueLen() { - return storageExecutionQueueLen; - } - - public void setStorageExecutionQueueLen(int storageExecutionQueueLen) { - this.storageExecutionQueueLen = storageExecutionQueueLen; - } - - public void incrementMultiChunkLargeValueCount() { - multiChunkLargeValueCount++; - } - - public int getMultiChunkLargeValueCount() { - return multiChunkLargeValueCount; - } - - public boolean isFound() { - return true; - } - - public IntList getKeySizeList() { - return keySizeList; - } - - public IntList getValueSizeList() { - return valueSizeList; - } - - public int getDotProductCount() { - return dotProductCount; - } - - public int getCosineSimilarityCount() { - return cosineSimilarityCount; - } - - public int getHadamardProductCount() { - return hadamardProductCount; - } - - public int getCountOperatorCount() { - return countOperatorCount; - } + int getRCU(); - public abstract int getRecordCount(); + boolean isFound(); - public abstract ByteBuf getResponseBody(); + ByteBuf getResponseBody(); - public abstract int getResponseSchemaIdHeader(); + int getResponseSchemaIdHeader(); } 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 new file mode 100644 index 0000000000..191d926be7 --- /dev/null +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/listener/response/ReadResponseStats.java @@ -0,0 +1,48 @@ +package com.linkedin.davinci.listener.response; + +/** + * This class is used to accumulate stats associated with a read response. + * + * This container is purely for metrics-related work, and should not be used to store any state which is functionally + * required to achieve the service's goal. Contrast with {@link ReadResponse}, which wraps this one. + * + * The reason to keep them separate is that the state used for metrics has a slightly longer lifespan than that used for + * functional purposes, since we record metrics at the very end, after the response is sent back. This allows us to free + * up the functional state sooner, even while we need to hang on to the metrics state. + */ +public interface ReadResponseStats { + long getCurrentTimeInNanos(); + + /** + * The implementer is responsible for doing the subtraction between the current time and the start time. + * + * @param startTimeInNanos the timestamp in nanoseconds of beginning of the period to measure. + */ + void addDatabaseLookupLatency(long startTimeInNanos); + + void addReadComputeLatency(double latency); + + void addReadComputeDeserializationLatency(double latency); + + void addReadComputeSerializationLatency(double latency); + + void addKeySize(int size); + + void addValueSize(int size); + + void addReadComputeOutputSize(int size); + + void incrementDotProductCount(int count); + + void incrementCountOperatorCount(int count); + + void incrementCosineSimilarityCount(int count); + + void incrementHadamardProductCount(int count); + + void setStorageExecutionSubmissionWaitTime(double storageExecutionSubmissionWaitTime); + + void setStorageExecutionQueueLen(int storageExecutionQueueLen); + + void incrementMultiChunkLargeValueCount(); +} diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/repository/ThinClientMetaStoreBasedRepository.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/repository/ThinClientMetaStoreBasedRepository.java index 0a393fd56b..190e163fc4 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/repository/ThinClientMetaStoreBasedRepository.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/repository/ThinClientMetaStoreBasedRepository.java @@ -9,7 +9,7 @@ import com.linkedin.venice.client.store.ClientFactory; import com.linkedin.venice.common.VeniceSystemStoreType; import com.linkedin.venice.exceptions.MissingKeyInStoreMetadataException; -import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.exceptions.VeniceRetriableException; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.StoreConfig; import com.linkedin.venice.meta.ZKStore; @@ -19,10 +19,16 @@ import com.linkedin.venice.systemstore.schemas.StoreMetaKey; import com.linkedin.venice.systemstore.schemas.StoreMetaValue; import com.linkedin.venice.systemstore.schemas.StoreProperties; +import com.linkedin.venice.utils.RetryUtils; import com.linkedin.venice.utils.VeniceProperties; import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; +import java.time.Duration; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; public class ThinClientMetaStoreBasedRepository extends NativeMetadataRepository { @@ -80,21 +86,21 @@ protected SchemaData getSchemaDataFromSystemStore(String storeName) { @Override protected StoreMetaValue getStoreMetaValue(String storeName, StoreMetaKey key) { - StoreMetaValue value; - try { - if (icProvider != null) { - value = - icProvider.call(getClass().getCanonicalName(), () -> getAvroClientForMetaStore(storeName).get(key)).get(); - } else { - value = getAvroClientForMetaStore(storeName).get(key).get(); + final Callable> supplier = () -> getAvroClientForMetaStore(storeName).get(key); + Callable> wrappedSupplier = + icProvider == null ? supplier : () -> icProvider.call(getClass().getCanonicalName(), supplier); + StoreMetaValue value = RetryUtils.executeWithMaxAttempt(() -> { + try { + return wrappedSupplier.call().get(1, TimeUnit.SECONDS); + } catch (ServiceDiscoveryException e) { + throw e; + } catch (Exception e) { + throw new VeniceRetriableException( + "Failed to get data from meta store using thin client for store: " + storeName + " with key: " + key, + e); } - } catch (ServiceDiscoveryException e) { - throw e; - } catch (Exception e) { - throw new VeniceException( - "Failed to get data from meta store using thin client for store: " + storeName + " with key: " + key, - e); - } + }, 5, Duration.ofSeconds(1), Arrays.asList(VeniceRetriableException.class)); + if (value == null) { throw new MissingKeyInStoreMetadataException(key.toString(), StoreMetaValue.class.getSimpleName()); } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/DiskHealthCheckService.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/DiskHealthCheckService.java index 38e9d5af70..8e81fe09d3 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/DiskHealthCheckService.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/DiskHealthCheckService.java @@ -220,7 +220,7 @@ public void run() { } } } catch (InterruptedException e) { - LOGGER.info("Disk check service thread shutting down", e); + LOGGER.info("Disk check service thread shutting down (interrupted)."); } catch (Exception ee) { LOGGER.error("Error while checking the disk health in server: ", ee); errorMessage = ee.getMessage(); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/AbstractAvroChunkingAdapter.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/AbstractAvroChunkingAdapter.java index 350a6d1221..8fd4ddd8ed 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/AbstractAvroChunkingAdapter.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/AbstractAvroChunkingAdapter.java @@ -1,6 +1,7 @@ package com.linkedin.davinci.storage.chunking; -import com.linkedin.davinci.listener.response.ReadResponse; +import com.linkedin.davinci.listener.response.NoOpReadResponseStats; +import com.linkedin.davinci.listener.response.ReadResponseStats; import com.linkedin.davinci.store.AbstractStorageEngine; import com.linkedin.davinci.store.record.ByteBufferValueRecord; import com.linkedin.davinci.store.record.ValueRecord; @@ -12,7 +13,6 @@ import com.linkedin.venice.serializer.RecordDeserializer; import com.linkedin.venice.storage.protocol.ChunkedValueManifest; import com.linkedin.venice.utils.ByteUtils; -import com.linkedin.venice.utils.LatencyUtils; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; @@ -32,18 +32,18 @@ public T constructValue( int bytesLength, T reusedValue, BinaryDecoder reusedDecoder, - ReadResponse response, + ReadResponseStats responseStats, int writerSchemaId, int readerSchemaId, StoreDeserializerCache storeDeserializerCache, VeniceCompressor compressor) { - return getByteArrayDecoder(compressor.getCompressionStrategy(), response).decode( + return getByteArrayDecoder(compressor.getCompressionStrategy()).decode( reusedDecoder, fullBytes, bytesLength, reusedValue, storeDeserializerCache.getDeserializer(writerSchemaId, readerSchemaId), - response, + responseStats, compressor); } @@ -55,8 +55,15 @@ public T constructValue( RecordDeserializer recordDeserializer, VeniceCompressor veniceCompressor) { try { - return byteArrayDecompressingDecoderValueOnly - .decode(null, valueOnlyBytes, offset, bytesLength, null, veniceCompressor, recordDeserializer, null); + return byteArrayDecompressingDecoderValueOnly.decode( + null, + valueOnlyBytes, + offset, + bytesLength, + null, + veniceCompressor, + recordDeserializer, + NoOpReadResponseStats.SINGLETON); } catch (IOException e) { throw new RuntimeException(e); } @@ -80,18 +87,18 @@ public T constructValue( ChunkedValueInputStream chunkedValueInputStream, T reusedValue, BinaryDecoder reusedDecoder, - ReadResponse response, + ReadResponseStats responseStats, int writerSchemaId, int readerSchemaId, StoreDeserializerCache storeDeserializerCache, VeniceCompressor compressor) { - return getInputStreamDecoder(response).decode( + return instrumentedDecompressingInputStreamDecoder.decode( reusedDecoder, chunkedValueInputStream, UNUSED_INPUT_BYTES_LENGTH, reusedValue, storeDeserializerCache.getDeserializer(writerSchemaId, readerSchemaId), - response, + responseStats, compressor); } @@ -102,7 +109,7 @@ public T get( boolean isChunked, T reusedValue, BinaryDecoder reusedDecoder, - ReadResponse response, + ReadResponseStats responseStats, int readerSchemaId, StoreDeserializerCache storeDeserializerCache, VeniceCompressor compressor, @@ -112,16 +119,16 @@ public T get( } return ChunkingUtils.getFromStorage( this, - store, + store::get, + store.getStoreVersionName(), partition, key, - response, + responseStats, reusedValue, reusedDecoder, readerSchemaId, storeDeserializerCache, compressor, - false, manifestContainer); } @@ -147,7 +154,6 @@ public ByteBufferValueRecord getWithSchemaId( reusedDecoder, storeDeserializerCache, compressor, - false, manifestContainer); } @@ -159,7 +165,7 @@ public T get( T reusedValue, BinaryDecoder reusedDecoder, boolean isChunked, - ReadResponse response, + ReadResponseStats responseStats, int readerSchemaId, StoreDeserializerCache storeDeserializerCache, VeniceCompressor compressor) { @@ -174,7 +180,7 @@ public T get( reusedRawValue, reusedValue, reusedDecoder, - response, + responseStats, readerSchemaId, storeDeserializerCache, compressor); @@ -188,7 +194,6 @@ public void getByPartialKey( BinaryDecoder reusedDecoder, RecordDeserializer keyRecordDeserializer, boolean isChunked, - ReadResponse response, int readerSchemaId, StoreDeserializerCache storeDeserializerCache, VeniceCompressor compressor, @@ -206,7 +211,6 @@ public void getByPartialKey( reusedValue, keyRecordDeserializer, reusedDecoder, - response, readerSchemaId, storeDeserializerCache, compressor, @@ -272,20 +276,14 @@ public void getByPartialKey( private final DecoderWrapper instrumentedDecompressingInputStreamDecoder = new InstrumentedDecoderWrapper<>(decompressingInputStreamDecoder); - private DecoderWrapper getByteArrayDecoder( - CompressionStrategy compressionStrategy, - ReadResponse response) { + private DecoderWrapper getByteArrayDecoder(CompressionStrategy compressionStrategy) { if (compressionStrategy == CompressionStrategy.NO_OP) { - return (response == null) ? byteArrayDecoder : instrumentedByteArrayDecoder; + return instrumentedByteArrayDecoder; } else { - return (response == null) ? decompressingByteArrayDecoder : instrumentedDecompressingByteArrayDecoder; + return instrumentedDecompressingByteArrayDecoder; } } - private DecoderWrapper getInputStreamDecoder(ReadResponse response) { - return (response == null) ? decompressingInputStreamDecoder : instrumentedDecompressingInputStreamDecoder; - } - private interface DecoderWrapper { OUTPUT decode( BinaryDecoder reusedDecoder, @@ -293,7 +291,7 @@ OUTPUT decode( int inputBytesLength, OUTPUT reusedValue, RecordDeserializer deserializer, - ReadResponse response, + ReadResponseStats responseStats, VeniceCompressor compressor); } @@ -306,7 +304,7 @@ OUTPUT decode( OUTPUT reusedValue, VeniceCompressor compressor, RecordDeserializer deserializer, - ReadResponse response) throws IOException; + ReadResponseStats response) throws IOException; } private static class InstrumentedDecoderWrapper implements DecoderWrapper { @@ -322,12 +320,12 @@ public OUTPUT decode( int inputBytesLength, OUTPUT reusedValue, RecordDeserializer deserializer, - ReadResponse response, + ReadResponseStats responseStats, VeniceCompressor compressor) { - long deserializeStartTimeInNS = System.nanoTime(); + long deserializeStartTimeInNS = responseStats.getCurrentTimeInNanos(); OUTPUT output = - delegate.decode(reusedDecoder, input, inputBytesLength, reusedValue, deserializer, response, compressor); - response.addReadComputeDeserializationLatency(LatencyUtils.getElapsedTimeFromNSToMS(deserializeStartTimeInNS)); + delegate.decode(reusedDecoder, input, inputBytesLength, reusedValue, deserializer, responseStats, compressor); + responseStats.addReadComputeDeserializationLatency(deserializeStartTimeInNS); return output; } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/BatchGetChunkingAdapter.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/BatchGetChunkingAdapter.java index 9dfd30a961..77c5fea521 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/BatchGetChunkingAdapter.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/BatchGetChunkingAdapter.java @@ -1,6 +1,6 @@ package com.linkedin.davinci.storage.chunking; -import com.linkedin.davinci.listener.response.ReadResponse; +import com.linkedin.davinci.listener.response.ReadResponseStats; import com.linkedin.davinci.store.AbstractStorageEngine; import com.linkedin.davinci.store.record.ValueRecord; import com.linkedin.venice.read.protocol.response.MultiGetResponseRecordV1; @@ -62,7 +62,7 @@ public static MultiGetResponseRecordV1 get( int partition, ByteBuffer key, boolean isChunked, - ReadResponse response) { + ReadResponseStats response) { if (isChunked) { key = ChunkingUtils.KEY_WITH_CHUNKING_SUFFIX_SERIALIZER.serializeNonChunkedKey(key); } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/ChunkingAdapter.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/ChunkingAdapter.java index 9af3631055..23c6aa240c 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/ChunkingAdapter.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/ChunkingAdapter.java @@ -1,10 +1,8 @@ package com.linkedin.davinci.storage.chunking; -import com.linkedin.davinci.listener.response.ReadResponse; -import com.linkedin.venice.compression.CompressionStrategy; +import com.linkedin.davinci.listener.response.ReadResponseStats; import com.linkedin.venice.compression.VeniceCompressor; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.meta.ReadOnlySchemaRepository; import com.linkedin.venice.serialization.StoreDeserializerCache; import com.linkedin.venice.serializer.RecordDeserializer; import com.linkedin.venice.storage.protocol.ChunkedValueManifest; @@ -31,8 +29,7 @@ public interface ChunkingAdapter { * @param bytesLength * @param reusedValue a previous instance of {@type VALUE} to be re-used in order to minimize GC * @param reusedDecoder a previous instance of {@link BinaryDecoder} to be re-used in order to minimize GC - * @param response the response returned by the query path, which carries certain metrics to be recorded at the - * end + * @param responseStats the {@link ReadResponseStats} which carries certain metrics to be recorded at the end * @param writerSchemaId schema used to serialize the value * @param readerSchemaId */ @@ -41,7 +38,7 @@ default VALUE constructValue( int bytesLength, VALUE reusedValue, BinaryDecoder reusedDecoder, - ReadResponse response, + ReadResponseStats responseStats, int writerSchemaId, int readerSchemaId, StoreDeserializerCache storeDeserializerCache, @@ -60,8 +57,6 @@ default VALUE constructValue( /** * This function can be implemented by the adapters which need fewer parameters. - * - * @see #constructValue(int, int, byte[], int, Object, BinaryDecoder, ReadResponse, CompressionStrategy, boolean, ReadOnlySchemaRepository, String, VeniceCompressor) */ default VALUE constructValue(int schemaId, byte[] fullBytes) { throw new VeniceException("Not implemented."); @@ -93,8 +88,7 @@ default VALUE constructValue(int schemaId, byte[] fullBytes) { * {@link #constructValue(int, Object)}: * @param reusedValue a previous instance of {@type VALUE} to be re-used in order to minimize GC * @param reusedDecoder a previous instance of {@link BinaryDecoder} to be re-used in order to minimize GC - * @param response the response returned by the query path, which carries certain metrics to be recorded at the - * end + * @param responseStats the {@link ReadResponseStats} which carries certain metrics to be recorded at the end * @param writerSchemaId of the user's value * @param readerSchemaId */ @@ -102,7 +96,7 @@ default VALUE constructValue( CHUNKS_CONTAINER chunksContainer, VALUE reusedValue, BinaryDecoder reusedDecoder, - ReadResponse response, + ReadResponseStats responseStats, int writerSchemaId, int readerSchemaId, StoreDeserializerCache storeDeserializerCache, @@ -112,8 +106,6 @@ default VALUE constructValue( /** * This function can be implemented by the adapters which need fewer parameters. - * - * @see #constructValue(int, Object, Object, BinaryDecoder, ReadResponse, CompressionStrategy, boolean, ReadOnlySchemaRepository, String, VeniceCompressor) */ default VALUE constructValue(int schemaId, CHUNKS_CONTAINER chunksContainer) { throw new VeniceException("Not implemented."); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/ChunkingUtils.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/ChunkingUtils.java index ad7c865a7a..dafe2b8855 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/ChunkingUtils.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/ChunkingUtils.java @@ -1,7 +1,8 @@ package com.linkedin.davinci.storage.chunking; import com.linkedin.davinci.callback.BytesStreamingCallback; -import com.linkedin.davinci.listener.response.ReadResponse; +import com.linkedin.davinci.listener.response.NoOpReadResponseStats; +import com.linkedin.davinci.listener.response.ReadResponseStats; import com.linkedin.davinci.store.AbstractStorageEngine; import com.linkedin.davinci.store.record.ByteBufferValueRecord; import com.linkedin.davinci.store.record.ValueRecord; @@ -16,9 +17,9 @@ import com.linkedin.venice.serializer.RecordDeserializer; import com.linkedin.venice.storage.protocol.ChunkedKeySuffix; import com.linkedin.venice.storage.protocol.ChunkedValueManifest; -import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.writer.VeniceWriter; import java.nio.ByteBuffer; +import java.util.Objects; import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.BinaryDecoder; @@ -65,6 +66,10 @@ public class ChunkingUtils { public static final KeyWithChunkingSuffixSerializer KEY_WITH_CHUNKING_SUFFIX_SERIALIZER = new KeyWithChunkingSuffixSerializer(); + interface StorageGetFunction { + byte[] apply(int partition, ByteBuffer key); + } + /** * Fills in default values for the unused parameters of the single get and batch get paths. */ @@ -73,8 +78,20 @@ static VALUE getFromStorage( AbstractStorageEngine store, int partition, ByteBuffer keyBuffer, - ReadResponse response) { - return getFromStorage(adapter, store, partition, keyBuffer, response, null, null, -1, null, null, false, null); + ReadResponseStats responseStats) { + return getFromStorage( + adapter, + store::get, + store.getStoreVersionName(), + partition, + keyBuffer, + responseStats, + null, + null, + -1, + null, + null, + null); } static VALUE getReplicationMetadataFromStorage( @@ -82,20 +99,19 @@ static VALUE getReplicationMetadataFromStorag AbstractStorageEngine store, int partition, ByteBuffer keyBuffer, - ReadResponse response, ChunkedValueManifestContainer manifestContainer) { return getFromStorage( adapter, - store, + store::getReplicationMetadata, + store.getStoreVersionName(), partition, keyBuffer, - response, + NoOpReadResponseStats.SINGLETON, null, null, -1, null, null, - true, manifestContainer); } @@ -107,11 +123,11 @@ static VALUE getFromStorage( ByteBuffer reusedRawValue, VALUE reusedValue, BinaryDecoder reusedDecoder, - ReadResponse response, + ReadResponseStats responseStats, int readerSchemaId, StoreDeserializerCache storeDeserializerCache, VeniceCompressor compressor) { - long databaseLookupStartTimeInNS = (response != null) ? System.nanoTime() : 0; + long databaseLookupStartTimeInNS = responseStats.getCurrentTimeInNanos(); reusedRawValue = store.get(partition, keyBuffer, reusedRawValue); if (reusedRawValue == null) { return null; @@ -121,15 +137,15 @@ static VALUE getFromStorage( reusedRawValue.limit(), databaseLookupStartTimeInNS, adapter, - store, + store::get, + store.getStoreVersionName(), partition, - response, + responseStats, reusedValue, reusedDecoder, readerSchemaId, storeDeserializerCache, compressor, - false, null); } @@ -141,14 +157,11 @@ static void getFromStorageByPartialKey( VALUE reusedValue, RecordDeserializer keyRecordDeserializer, BinaryDecoder reusedDecoder, - ReadResponse response, int readerSchemaId, StoreDeserializerCache storeDeserializerCache, VeniceCompressor compressor, StreamingCallback computingCallback) { - long databaseLookupStartTimeInNS = (response != null) ? System.nanoTime() : 0; - BytesStreamingCallback callback = new BytesStreamingCallback() { GenericRecord deserializedValueRecord; @@ -163,10 +176,6 @@ public void onRecordReceived(byte[] key, byte[] value) { if (writerSchemaId > 0) { // User-defined schema, thus not a chunked value. - if (response != null) { - response.addDatabaseLookupLatency(LatencyUtils.getElapsedTimeFromNSToMS(databaseLookupStartTimeInNS)); - } - GenericRecord deserializedKey = keyRecordDeserializer.deserialize(key); deserializedValueRecord = (GenericRecord) adapter.constructValue( @@ -174,7 +183,7 @@ public void onRecordReceived(byte[] key, byte[] value) { value.length, reusedValue, reusedDecoder, - response, + NoOpReadResponseStats.SINGLETON, writerSchemaId, readerSchemaId, storeDeserializerCache, @@ -206,40 +215,39 @@ public void onCompletion() { * not be called directly, from the query code, as it expects the key to be properly formatted * already. Use of one these simpler functions instead: * - * @see SingleGetChunkingAdapter#get(AbstractStorageEngine, int, byte[], boolean, ReadResponse) - * @see BatchGetChunkingAdapter#get(AbstractStorageEngine, int, ByteBuffer, boolean, ReadResponse) + * @see SingleGetChunkingAdapter#get(AbstractStorageEngine, int, byte[], boolean, ReadResponseStats) + * @see BatchGetChunkingAdapter#get(AbstractStorageEngine, int, ByteBuffer, boolean, ReadResponseStats) */ static VALUE getFromStorage( ChunkingAdapter adapter, - AbstractStorageEngine store, + StorageGetFunction storageGetFunction, + String storeVersionName, int partition, ByteBuffer keyBuffer, - ReadResponse response, + ReadResponseStats responseStats, VALUE reusedValue, BinaryDecoder reusedDecoder, int readerSchemaID, StoreDeserializerCache storeDeserializerCache, VeniceCompressor compressor, - boolean isRmdValue, ChunkedValueManifestContainer manifestContainer) { - long databaseLookupStartTimeInNS = (response != null) ? System.nanoTime() : 0; - byte[] value = - isRmdValue ? store.getReplicationMetadata(partition, keyBuffer.array()) : store.get(partition, keyBuffer); + long databaseLookupStartTimeInNS = responseStats.getCurrentTimeInNanos(); + byte[] value = storageGetFunction.apply(partition, keyBuffer); return getFromStorage( value, (value == null ? 0 : value.length), databaseLookupStartTimeInNS, adapter, - store, + storageGetFunction, + storeVersionName, partition, - response, + responseStats, reusedValue, reusedDecoder, readerSchemaID, storeDeserializerCache, compressor, - isRmdValue, manifestContainer); } @@ -252,28 +260,30 @@ static ByteBufferValueRecord getValueAndSchemaI BinaryDecoder reusedDecoder, StoreDeserializerCache storeDeserializerCache, VeniceCompressor compressor, - boolean isRmdValue, ChunkedValueManifestContainer manifestContainer) { - byte[] value = - isRmdValue ? store.getReplicationMetadata(partition, keyBuffer.array()) : store.get(partition, keyBuffer); + byte[] value = store.get(partition, keyBuffer); int writerSchemaId = value == null ? 0 : ValueRecord.parseSchemaId(value); - return new ByteBufferValueRecord<>( - getFromStorage( - value, - (value == null ? 0 : value.length), - 0, - adapter, - store, - partition, - null, - reusedValue, - reusedDecoder, - -1, - storeDeserializerCache, - compressor, - isRmdValue, - manifestContainer), - writerSchemaId); + VALUE object = getFromStorage( + value, + (value == null ? 0 : value.length), + 0, + adapter, + store::get, + store.getStoreVersionName(), + partition, + NoOpReadResponseStats.SINGLETON, + reusedValue, + reusedDecoder, + -1, + storeDeserializerCache, + compressor, + manifestContainer); + if (writerSchemaId == AvroProtocolDefinition.CHUNKED_VALUE_MANIFEST.getCurrentProtocolVersion()) { + writerSchemaId = Objects.requireNonNull(manifestContainer, "The ChunkedValueManifestContainer cannot be null.") + .getManifest() + .getSchemaId(); + } + return new ByteBufferValueRecord<>(object, writerSchemaId); } /** @@ -285,23 +295,23 @@ static ByteBufferValueRecord getValueAndSchemaI * not be called directly, from the query code, as it expects the key to be properly formatted * already. Use of one these simpler functions instead: * - * @see SingleGetChunkingAdapter#get(AbstractStorageEngine, int, byte[], boolean, ReadResponse) - * @see BatchGetChunkingAdapter#get(AbstractStorageEngine, int, ByteBuffer, boolean, ReadResponse) + * @see SingleGetChunkingAdapter#get(AbstractStorageEngine, int, byte[], boolean, ReadResponseStats) + * @see BatchGetChunkingAdapter#get(AbstractStorageEngine, int, ByteBuffer, boolean, ReadResponseStats) */ private static VALUE getFromStorage( byte[] value, int valueLength, long databaseLookupStartTimeInNS, ChunkingAdapter adapter, - AbstractStorageEngine store, + StorageGetFunction storageGetFunction, + String storeVersionName, int partition, - ReadResponse response, + ReadResponseStats responseStats, VALUE reusedValue, BinaryDecoder reusedDecoder, int readerSchemaId, StoreDeserializerCache storeDeserializerCache, VeniceCompressor compressor, - boolean isRmdValue, ChunkedValueManifestContainer manifestContainer) { if (value == null) { @@ -312,16 +322,14 @@ private static VALUE getFromStorage( if (writerSchemaId > 0) { // User-defined schema, thus not a chunked value. Early termination. - if (response != null) { - response.addDatabaseLookupLatency(LatencyUtils.getElapsedTimeFromNSToMS(databaseLookupStartTimeInNS)); - response.addValueSize(valueLength); - } + responseStats.addDatabaseLookupLatency(databaseLookupStartTimeInNS); + responseStats.addValueSize(valueLength); return adapter.constructValue( value, valueLength, reusedValue, reusedDecoder, - response, + responseStats, writerSchemaId, readerSchemaId, storeDeserializerCache, @@ -339,23 +347,23 @@ private static VALUE getFromStorage( CHUNKS_CONTAINER assembledValueContainer = adapter.constructChunksContainer(chunkedValueManifest); int actualSize = 0; + byte[] valueChunk; for (int chunkIndex = 0; chunkIndex < chunkedValueManifest.keysWithChunkIdSuffix.size(); chunkIndex++) { // N.B.: This is done sequentially. Originally, each chunk was fetched concurrently in the same executor // as the main queries, but this might cause deadlocks, so we are now doing it sequentially. If we want to // optimize large value retrieval in the future, it's unclear whether the concurrent retrieval approach // is optimal (as opposed to streaming the response out incrementally, for example). Since this is a // premature optimization, we are not addressing it right now. - byte[] chunkKey = chunkedValueManifest.keysWithChunkIdSuffix.get(chunkIndex).array(); - byte[] valueChunk = - isRmdValue ? store.getReplicationMetadata(partition, chunkKey) : store.get(partition, chunkKey); + valueChunk = storageGetFunction.apply(partition, chunkedValueManifest.keysWithChunkIdSuffix.get(chunkIndex)); if (valueChunk == null) { - throw new VeniceException("Chunk not found in " + getExceptionMessageDetails(store, partition, chunkIndex)); + throw new VeniceException( + "Chunk not found in " + getExceptionMessageDetails(storeVersionName, partition, chunkIndex)); } else if (ValueRecord.parseSchemaId(valueChunk) != AvroProtocolDefinition.CHUNK.getCurrentProtocolVersion()) { throw new VeniceException( "Did not get the chunk schema ID while attempting to retrieve a chunk! " + "Instead, got schema ID: " + ValueRecord.parseSchemaId(valueChunk) + " from " - + getExceptionMessageDetails(store, partition, chunkIndex)); + + getExceptionMessageDetails(storeVersionName, partition, chunkIndex)); } actualSize += valueChunk.length - ValueRecord.SCHEMA_HEADER_LENGTH; @@ -367,28 +375,26 @@ private static VALUE getFromStorage( throw new VeniceException( "The fully assembled large value does not have the expected size! " + "actualSize: " + actualSize + ", chunkedValueManifest.size: " + chunkedValueManifest.size + ", " - + getExceptionMessageDetails(store, partition, null)); + + getExceptionMessageDetails(storeVersionName, partition, null)); } - if (response != null) { - response.addDatabaseLookupLatency(LatencyUtils.getElapsedTimeFromNSToMS(databaseLookupStartTimeInNS)); - response.addValueSize(actualSize); - response.incrementMultiChunkLargeValueCount(); - } + responseStats.addDatabaseLookupLatency(databaseLookupStartTimeInNS); + responseStats.addValueSize(actualSize); + responseStats.incrementMultiChunkLargeValueCount(); return adapter.constructValue( assembledValueContainer, reusedValue, reusedDecoder, - response, + responseStats, chunkedValueManifest.schemaId, readerSchemaId, storeDeserializerCache, compressor); } - private static String getExceptionMessageDetails(AbstractStorageEngine store, int partition, Integer chunkIndex) { - String message = "store: " + store.getStoreVersionName() + ", partition: " + partition; + private static String getExceptionMessageDetails(String storeVersionName, int partition, Integer chunkIndex) { + String message = "store-version: " + storeVersionName + ", partition: " + partition; if (chunkIndex != null) { message += ", chunk index: " + chunkIndex; } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/SingleGetChunkingAdapter.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/SingleGetChunkingAdapter.java index 944c39416b..7f3d682055 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/SingleGetChunkingAdapter.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/storage/chunking/SingleGetChunkingAdapter.java @@ -1,6 +1,6 @@ package com.linkedin.davinci.storage.chunking; -import com.linkedin.davinci.listener.response.ReadResponse; +import com.linkedin.davinci.listener.response.ReadResponseStats; import com.linkedin.davinci.store.AbstractStorageEngine; import com.linkedin.davinci.store.record.ValueRecord; import com.linkedin.venice.storage.protocol.ChunkedValueManifest; @@ -44,7 +44,7 @@ public static ValueRecord get( int partition, byte[] key, boolean isChunked, - ReadResponse response) { + ReadResponseStats response) { ByteBuffer keyBuffer = isChunked ? ChunkingUtils.KEY_WITH_CHUNKING_SUFFIX_SERIALIZER.serializeNonChunkedKeyAsByteBuffer(key) : ByteBuffer.wrap(key); @@ -56,17 +56,11 @@ public static ValueRecord getReplicationMetadata( int partition, byte[] key, boolean isChunked, - ReadResponse response, ChunkedValueManifestContainer manifestContainer) { ByteBuffer keyBuffer = isChunked ? ChunkingUtils.KEY_WITH_CHUNKING_SUFFIX_SERIALIZER.serializeNonChunkedKeyAsByteBuffer(key) : ByteBuffer.wrap(key); - return ChunkingUtils.getReplicationMetadataFromStorage( - SINGLE_GET_CHUNKING_ADAPTER, - store, - partition, - keyBuffer, - response, - manifestContainer); + return ChunkingUtils + .getReplicationMetadataFromStorage(SINGLE_GET_CHUNKING_ADAPTER, store, partition, keyBuffer, manifestContainer); } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/AbstractStorageEngine.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/AbstractStorageEngine.java index 58db4d064d..62bac7d466 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/AbstractStorageEngine.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/AbstractStorageEngine.java @@ -538,7 +538,7 @@ public void deleteWithReplicationMetadata(int partitionId, byte[] key, byte[] re }); } - public byte[] getReplicationMetadata(int partitionId, byte[] key) { + public byte[] getReplicationMetadata(int partitionId, ByteBuffer key) { return executeWithSafeGuard(partitionId, () -> { AbstractStoragePartition partition = getPartitionOrThrow(partitionId); return partition.getReplicationMetadata(key); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/AbstractStoragePartition.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/AbstractStoragePartition.java index 3d6c410ea5..7292438a16 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/AbstractStoragePartition.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/AbstractStoragePartition.java @@ -159,7 +159,7 @@ public void putReplicationMetadata(byte[] key, byte[] metadata) { * Only {@link ReplicationMetadataRocksDBStoragePartition} will execute this method, * other storage partition implementation will VeniceUnsupportedOperationException. */ - public byte[] getReplicationMetadata(byte[] key) { + public byte[] getReplicationMetadata(ByteBuffer key) { throw new VeniceUnsupportedOperationException("getReplicationMetadata"); } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/rocksdb/ReplicationMetadataRocksDBStoragePartition.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/rocksdb/ReplicationMetadataRocksDBStoragePartition.java index a9023300be..982affdb88 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/rocksdb/ReplicationMetadataRocksDBStoragePartition.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/rocksdb/ReplicationMetadataRocksDBStoragePartition.java @@ -131,12 +131,16 @@ public synchronized void putWithReplicationMetadata(byte[] key, ByteBuffer value } @Override - public byte[] getReplicationMetadata(byte[] key) { + public byte[] getReplicationMetadata(ByteBuffer key) { readCloseRWLock.readLock().lock(); try { makeSureRocksDBIsStillOpen(); - return rocksDB - .get(columnFamilyHandleList.get(REPLICATION_METADATA_COLUMN_FAMILY_INDEX), READ_OPTIONS_DEFAULT, key); + return rocksDB.get( + columnFamilyHandleList.get(REPLICATION_METADATA_COLUMN_FAMILY_INDEX), + READ_OPTIONS_DEFAULT, + key.array(), + key.position(), + key.remaining()); } catch (RocksDBException e) { throw new VeniceException("Failed to get value from RocksDB: " + replicaId, e); } finally { diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/utils/ChunkAssembler.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/utils/ChunkAssembler.java index 6eaf89a3aa..02d72ac975 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/utils/ChunkAssembler.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/utils/ChunkAssembler.java @@ -1,5 +1,6 @@ package com.linkedin.davinci.utils; +import com.linkedin.davinci.listener.response.NoOpReadResponseStats; import com.linkedin.davinci.storage.chunking.RawBytesChunkingAdapter; import com.linkedin.davinci.store.memory.InMemoryStorageEngine; import com.linkedin.davinci.store.record.ValueRecord; @@ -16,7 +17,7 @@ /* - * This class serves as a utility to deserialize and assemble chunks consumed from a Kafka topic + * This class serves as a utility to deserialize and assemble chunks consumed from a Kafka topic */ public class ChunkAssembler { private static final Logger LOGGER = LogManager.getLogger(ChunkAssembler.class); @@ -81,7 +82,7 @@ public T bufferAndAssembleRecord( false, null, null, - null, + NoOpReadResponseStats.SINGLETON, readerSchemaId, RawBytesStoreDeserializerCache.getInstance(), compressor, diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java index 64ebed6d80..24bf0b6532 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java @@ -489,7 +489,8 @@ public void testReadingChunkedRmdFromStorage() { when(ingestionTask.isChunked()).thenReturn(true); when(ingestionTask.getHostLevelIngestionStats()).thenReturn(mock(HostLevelIngestionStats.class)); ChunkedValueManifestContainer container = new ChunkedValueManifestContainer(); - when(storageEngine.getReplicationMetadata(partition, topLevelKey1)).thenReturn(expectedNonChunkedValue); + when(storageEngine.getReplicationMetadata(partition, ByteBuffer.wrap(topLevelKey1))) + .thenReturn(expectedNonChunkedValue); byte[] result = ingestionTask.getRmdWithValueSchemaByteBufferFromStorage(partition, key1, container, 0L); Assert.assertNotNull(result); Assert.assertNull(container.getManifest()); @@ -517,8 +518,9 @@ public void testReadingChunkedRmdFromStorage() { chunkedManifestBytes = ByteUtils.prependIntHeaderToByteBuffer(chunkedManifestBytes, manifestSchemaId); byte[] topLevelKey2 = keyWithChunkingSuffixSerializer.serializeNonChunkedKey(key2); byte[] chunkedKey1InKey2 = chunkedKeyWithSuffix1.array(); - when(storageEngine.getReplicationMetadata(partition, topLevelKey2)).thenReturn(chunkedManifestBytes.array()); - when(storageEngine.getReplicationMetadata(partition, chunkedKey1InKey2)).thenReturn(chunkedValue1); + when(storageEngine.getReplicationMetadata(partition, ByteBuffer.wrap(topLevelKey2))) + .thenReturn(chunkedManifestBytes.array()); + when(storageEngine.getReplicationMetadata(partition, ByteBuffer.wrap(chunkedKey1InKey2))).thenReturn(chunkedValue1); byte[] result2 = ingestionTask.getRmdWithValueSchemaByteBufferFromStorage(partition, key2, container, 0L); Assert.assertNotNull(result2); Assert.assertNotNull(container.getManifest()); @@ -551,9 +553,10 @@ public void testReadingChunkedRmdFromStorage() { byte[] topLevelKey3 = keyWithChunkingSuffixSerializer.serializeNonChunkedKey(key3); byte[] chunkedKey1InKey3 = chunkedKeyWithSuffix1.array(); byte[] chunkedKey2InKey3 = chunkedKeyWithSuffix2.array(); - when(storageEngine.getReplicationMetadata(partition, topLevelKey3)).thenReturn(chunkedManifestBytes.array()); - when(storageEngine.getReplicationMetadata(partition, chunkedKey1InKey3)).thenReturn(chunkedValue1); - when(storageEngine.getReplicationMetadata(partition, chunkedKey2InKey3)).thenReturn(chunkedValue2); + when(storageEngine.getReplicationMetadata(partition, ByteBuffer.wrap(topLevelKey3))) + .thenReturn(chunkedManifestBytes.array()); + when(storageEngine.getReplicationMetadata(partition, ByteBuffer.wrap(chunkedKey1InKey3))).thenReturn(chunkedValue1); + when(storageEngine.getReplicationMetadata(partition, ByteBuffer.wrap(chunkedKey2InKey3))).thenReturn(chunkedValue2); byte[] result3 = ingestionTask.getRmdWithValueSchemaByteBufferFromStorage(partition, key3, container, 0L); Assert.assertNotNull(result3); Assert.assertNotNull(container.getManifest()); diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java index 5cac2fa635..3951d0d284 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java @@ -3766,9 +3766,9 @@ public void testResubscribeAfterRoleChange() throws Exception { doReturn(new ReentrantReadWriteLock()).when(mockAbstractStorageEngine).getRWLockForPartitionOrThrow(anyInt()); doReturn(putKeyFooReplicationMetadataWithValueSchemaIdBytesDefault).when(mockStoragePartition) - .getReplicationMetadata(putKeyFoo); + .getReplicationMetadata(ByteBuffer.wrap(putKeyFoo)); doReturn(deleteKeyFooReplicationMetadataWithValueSchemaIdBytes).when(mockStoragePartition) - .getReplicationMetadata(deleteKeyFoo); + .getReplicationMetadata(ByteBuffer.wrap(deleteKeyFoo)); VeniceWriter vtWriter = getVeniceWriter(topic, new MockInMemoryProducerAdapter(inMemoryLocalKafkaBroker)); VeniceWriter localRtWriter = getVeniceWriter( diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/storage/chunking/ChunkingTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/storage/chunking/ChunkingTest.java index 020e3e2958..d72a56c7b0 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/storage/chunking/ChunkingTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/storage/chunking/ChunkingTest.java @@ -5,6 +5,7 @@ import static org.mockito.Mockito.mock; import com.linkedin.davinci.compression.StorageEngineBackedCompressorFactory; +import com.linkedin.davinci.listener.response.NoOpReadResponseStats; import com.linkedin.davinci.storage.StorageMetadataService; import com.linkedin.davinci.store.AbstractStorageEngine; import com.linkedin.davinci.store.record.ByteBufferValueRecord; @@ -36,6 +37,8 @@ public class ChunkingTest { + private static final int SCHEMA_ID = 1; + @DataProvider(name = "recordProvider") public Object[][] recordProvider() { List params = new ArrayList<>(); @@ -182,22 +185,24 @@ private void runTest( ValueRecord.SCHEMA_HEADER_LENGTH, serializedRecord.length - cutOff); - SchemaEntry schemaEntry = new SchemaEntry(1, schema); + SchemaEntry schemaEntry = new SchemaEntry(SCHEMA_ID, schema); HelixReadOnlySchemaRepository schemaRepository = mock(HelixReadOnlySchemaRepository.class); - doReturn(schemaEntry).when(schemaRepository).getValueSchema(storeName, 1); + doReturn(schemaEntry).when(schemaRepository).getValueSchema(storeName, SCHEMA_ID); doReturn(schemaEntry).when(schemaRepository).getSupersetOrLatestValueSchema(storeName); AbstractStorageEngine storageEngine = mock(AbstractStorageEngine.class); - byte[] firstKey = ByteUtils.fromHexString( - "040647454FF4BAF2630A5449544C45440010494D504C494349540036EB0A5300374C6A9C5EEBB468C58E4300CE984E0001"); - byte[] secondKey = ByteUtils.fromHexString( - "040647454FF4BAF2630A5449544C45440010494D504C494349540036EB0A5300374C6A9C5EEBB468C58E4300CE984E0201"); + ByteBuffer firstKey = ByteBuffer.wrap( + ByteUtils.fromHexString( + "040647454FF4BAF2630A5449544C45440010494D504C494349540036EB0A5300374C6A9C5EEBB468C58E4300CE984E0001")); + ByteBuffer secondKey = ByteBuffer.wrap( + ByteUtils.fromHexString( + "040647454FF4BAF2630A5449544C45440010494D504C494349540036EB0A5300374C6A9C5EEBB468C58E4300CE984E0201")); ChunkedValueManifest chunkedValueManifest = new ChunkedValueManifest(); chunkedValueManifest.keysWithChunkIdSuffix = new ArrayList<>(2); - chunkedValueManifest.keysWithChunkIdSuffix.add(ByteBuffer.wrap(firstKey)); - chunkedValueManifest.keysWithChunkIdSuffix.add(ByteBuffer.wrap(secondKey)); - chunkedValueManifest.schemaId = 1; + chunkedValueManifest.keysWithChunkIdSuffix.add(firstKey); + chunkedValueManifest.keysWithChunkIdSuffix.add(secondKey); + chunkedValueManifest.schemaId = SCHEMA_ID; chunkedValueManifest.size = chunk1Bytes.length + chunk2Bytes.length - chunkedValueManifest.keysWithChunkIdSuffix.size() * ValueRecord.SCHEMA_HEADER_LENGTH; byte[] serializedCVM = SerializerDeserializerFactory.getAvroGenericSerializer(ChunkedValueManifest.SCHEMA$) @@ -233,7 +238,7 @@ private void runTest( null, storeDeserializerCache, compressor, - null); + new ChunkedValueManifestContainer()); } else { retrievedObject = chunkingAdapter.get( storageEngine, @@ -242,7 +247,7 @@ private void runTest( true, null, null, - null, + NoOpReadResponseStats.SINGLETON, readerSchemaId, storeDeserializerCache, compressor, @@ -278,9 +283,12 @@ public void testRawBytesChunkingAdapterWithSchemaId(GenericRecord record) { SerializerDeserializerFactory.getAvroGenericSerializer(record.getSchema()).serialize(record); runTest(record, RawBytesChunkingAdapter.INSTANCE, true, (valueFromStorageEngine) -> { Assert.assertTrue(valueFromStorageEngine instanceof ByteBufferValueRecord); - Object value = ((ByteBufferValueRecord) valueFromStorageEngine).value(); + ByteBufferValueRecord byteBufferValueRecord = (ByteBufferValueRecord) valueFromStorageEngine; + Object value = byteBufferValueRecord.value(); Assert.assertTrue(value instanceof ByteBuffer); - Assert.assertEquals(ByteUtils.extractByteArray((ByteBuffer) value), serializedRecord); + ByteBuffer bbValue = (ByteBuffer) value; + Assert.assertEquals(ByteUtils.extractByteArray(bbValue), serializedRecord); + Assert.assertEquals(byteBufferValueRecord.writerSchemaId(), SCHEMA_ID); return null; }, true); } diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/store/rocksdb/ReplicationMetadataRocksDBStoragePartitionTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/store/rocksdb/ReplicationMetadataRocksDBStoragePartitionTest.java index 84f83d2de3..5ad8076846 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/store/rocksdb/ReplicationMetadataRocksDBStoragePartitionTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/store/rocksdb/ReplicationMetadataRocksDBStoragePartitionTest.java @@ -202,7 +202,7 @@ public void testMetadataColumnFamily() { byte[] key = entry.getKey().getBytes(); byte[] value = storagePartition.get(key); Assert.assertEquals(value, entry.getValue().getFirst().getBytes()); - byte[] metadata = storagePartition.getReplicationMetadata(key); + byte[] metadata = storagePartition.getReplicationMetadata(ByteBuffer.wrap(key)); ByteBuffer replicationMetadataWithValueSchema = ByteBuffer.wrap(metadata); int replicationMetadataWithValueSchemaInt = replicationMetadataWithValueSchema.getInt(); @@ -222,7 +222,7 @@ public void testMetadataColumnFamily() { byte[] value = storagePartition.get(key); Assert.assertNull(value); - byte[] metadata = storagePartition.getReplicationMetadata(key); + byte[] metadata = storagePartition.getReplicationMetadata(ByteBuffer.wrap(key)); ByteBuffer replicationMetadataWithValueSchema = ByteBuffer.wrap(metadata); int replicationMetadataWithValueSchemaInt = replicationMetadataWithValueSchema.getInt(); @@ -248,7 +248,7 @@ public void testMetadataColumnFamily() { byte[] key = entry.getKey().getBytes(); byte[] value = storagePartition.get(key); Assert.assertEquals(value, entry.getValue().getFirst().getBytes()); - Assert.assertNull(storagePartition.getReplicationMetadata(key)); + Assert.assertNull(storagePartition.getReplicationMetadata(ByteBuffer.wrap(key))); } for (Map.Entry> entry: inputRecordsBatch.entrySet()) { @@ -263,7 +263,7 @@ public void testMetadataColumnFamily() { byte[] value = storagePartition.get(key); Assert.assertNull(value); - byte[] metadata = storagePartition.getReplicationMetadata(key); + byte[] metadata = storagePartition.getReplicationMetadata(ByteBuffer.wrap(key)); ByteBuffer replicationMetadataWithValueSchema = ByteBuffer.wrap(metadata); int replicationMetadataWithValueSchemaInt = replicationMetadataWithValueSchema.getInt(); @@ -426,7 +426,7 @@ public void testReplicationMetadataIngestion( Assert.assertEquals(storagePartition.get(entry.getKey().getBytes()), bytes); if (sorted) { Assert.assertEquals( - storagePartition.getReplicationMetadata(entry.getKey().getBytes()), + storagePartition.getReplicationMetadata(ByteBuffer.wrap(entry.getKey().getBytes())), entry.getValue().getSecond().getBytes()); } } diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/utils/IsolatedIngestionUtilsTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/utils/IsolatedIngestionUtilsTest.java index 98a50c8094..fa5cf01af2 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/utils/IsolatedIngestionUtilsTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/utils/IsolatedIngestionUtilsTest.java @@ -1,5 +1,10 @@ package com.linkedin.davinci.utils; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertThrows; +import static org.testng.Assert.assertTrue; + import com.linkedin.davinci.ingestion.utils.IsolatedIngestionUtils; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.ingestion.protocol.enums.IngestionAction; @@ -7,7 +12,6 @@ import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpVersion; -import org.testng.Assert; import org.testng.annotations.Test; @@ -17,22 +21,41 @@ public void testGetIngestionActionFromRequest() { HttpRequest request; // Validate REPORT request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, "http://127.0.0.1:27015/REPORT"); - Assert.assertEquals(IsolatedIngestionUtils.getIngestionActionFromRequest(request), IngestionAction.REPORT); + assertEquals(IsolatedIngestionUtils.getIngestionActionFromRequest(request), IngestionAction.REPORT); // Validate METRIC request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, "http://127.0.0.1:27015/METRIC"); - Assert.assertEquals(IsolatedIngestionUtils.getIngestionActionFromRequest(request), IngestionAction.METRIC); + assertEquals(IsolatedIngestionUtils.getIngestionActionFromRequest(request), IngestionAction.METRIC); // Validate COMMAND request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, "http://127.0.0.1:27015/COMMAND"); - Assert.assertEquals(IsolatedIngestionUtils.getIngestionActionFromRequest(request), IngestionAction.COMMAND); + assertEquals(IsolatedIngestionUtils.getIngestionActionFromRequest(request), IngestionAction.COMMAND); // Invalid URI style 1 request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, "http://127.0.0.1:27015"); HttpRequest finalRequest = request; - Assert - .assertThrows(VeniceException.class, () -> IsolatedIngestionUtils.getIngestionActionFromRequest(finalRequest)); + assertThrows(VeniceException.class, () -> IsolatedIngestionUtils.getIngestionActionFromRequest(finalRequest)); // Invalid URI style 2 request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, "http://127.0.0.1:27015/FAIL"); HttpRequest finalRequest2 = request; - Assert - .assertThrows(VeniceException.class, () -> IsolatedIngestionUtils.getIngestionActionFromRequest(finalRequest2)); + assertThrows(VeniceException.class, () -> IsolatedIngestionUtils.getIngestionActionFromRequest(finalRequest2)); + } + + @Test + public void testExecuteShellCommand() { + String pwdResult = IsolatedIngestionUtils.executeShellCommand("pwd"); + assertNotNull(pwdResult); + assertTrue(pwdResult.startsWith("/")); + + String nonExistingCommandResult = IsolatedIngestionUtils.executeShellCommand("blablabla"); + assertNotNull(nonExistingCommandResult); + assertEquals(nonExistingCommandResult, ""); + + String failedCommandResult = IsolatedIngestionUtils.executeShellCommand("cp bogus1 bogus2"); + assertNotNull(failedCommandResult); + assertEquals(failedCommandResult, ""); + + String emptyCommandResult = IsolatedIngestionUtils.executeShellCommand(""); + assertNotNull(emptyCommandResult); + assertEquals(emptyCommandResult, ""); + + assertThrows(VeniceException.class, () -> IsolatedIngestionUtils.executeShellCommand("kill xyz")); } } diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/DoubleAndBooleanConsumer.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/DoubleAndBooleanConsumer.java new file mode 100644 index 0000000000..d36e021de3 --- /dev/null +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/DoubleAndBooleanConsumer.java @@ -0,0 +1,5 @@ +package com.linkedin.venice.utils; + +public interface DoubleAndBooleanConsumer { + void accept(double doubleValue, boolean booleanValue); +} diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientTest.java index 5879211bb0..9580ed6b1b 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientTest.java @@ -111,6 +111,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Stream; @@ -167,8 +168,8 @@ public void cleanUp() { @Test(timeOut = TEST_TIMEOUT) public void testConcurrentGetAndStart() throws Exception { - String storeName1 = createStoreWithMetaSystemStoreAndPushStatusSystemStore(KEY_COUNT); - String storeName2 = createStoreWithMetaSystemStoreAndPushStatusSystemStore(KEY_COUNT); + String s1 = createStoreWithMetaSystemStoreAndPushStatusSystemStore(KEY_COUNT); + String s2 = createStoreWithMetaSystemStoreAndPushStatusSystemStore(KEY_COUNT); String baseDataPath = Utils.getTempDataDirectory().getAbsolutePath(); VeniceProperties backendConfig = new PropertyBuilder().put(CLIENT_USE_SYSTEM_STORE_REPOSITORY, true) @@ -177,22 +178,42 @@ public void testConcurrentGetAndStart() throws Exception { .put(PERSISTENCE_TYPE, ROCKS_DB) .build(); - for (int i = 0; i < 10; ++i) { + int totalIterations = 10; + for (int i = 0; i < totalIterations; ++i) { MetricsRepository metricsRepository = new MetricsRepository(); + final int iteration = i + 1; try (CachingDaVinciClientFactory factory = new CachingDaVinciClientFactory( d2Client, VeniceRouterWrapper.CLUSTER_DISCOVERY_D2_SERVICE_NAME, metricsRepository, backendConfig)) { + DaVinciConfig c1 = new DaVinciConfig(); + DaVinciConfig c2 = new DaVinciConfig().setIsolated(true); + BiFunction> starter = + (storeName, daVinciConfig) -> CompletableFuture.runAsync(() -> { + try { + factory.getGenericAvroClient(storeName, daVinciConfig).start(); + LOGGER.info( + "Successfully started DVC in iteration {}/{} for store '{}' with config: {}", + iteration, + totalIterations, + storeName, + daVinciConfig); + } catch (Exception e) { + LOGGER.warn( + "Caught exception while trying to start DVC in iteration {}/{} for store '{}' with config: {}", + iteration, + totalIterations, + storeName, + daVinciConfig); + throw e; + } + }); CompletableFuture - .allOf( - CompletableFuture.runAsync(() -> factory.getGenericAvroClient(storeName1, new DaVinciConfig()).start()), - CompletableFuture.runAsync(() -> factory.getGenericAvroClient(storeName2, new DaVinciConfig()).start()), - CompletableFuture.runAsync( - () -> factory.getGenericAvroClient(storeName1, new DaVinciConfig().setIsolated(true)).start()), - CompletableFuture.runAsync( - () -> factory.getGenericAvroClient(storeName2, new DaVinciConfig().setIsolated(true)).start())) + .allOf(starter.apply(s1, c1), starter.apply(s2, c1), starter.apply(s1, c2), starter.apply(s2, c2)) .get(); + } catch (Exception e) { + throw new VeniceException("Failed to instantiate DVCs in iteration " + iteration + "/" + totalIterations, e); } assertThrows(NullPointerException.class, AvroGenericDaVinciClient::getBackend); } @@ -207,8 +228,8 @@ public void testConcurrentGetAndStart() throws Exception { VeniceRouterWrapper.CLUSTER_DISCOVERY_D2_SERVICE_NAME, metricsRepository, backendConfig)) { - factory.getAndStartGenericAvroClient(storeName1, daVinciConfig); - factory.getAndStartGenericAvroClient(storeName1, daVinciConfig); + factory.getAndStartGenericAvroClient(s1, daVinciConfig); + factory.getAndStartGenericAvroClient(s1, daVinciConfig); } } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PartialUpdateTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PartialUpdateTest.java index 643e2046ce..79d542625f 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PartialUpdateTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PartialUpdateTest.java @@ -1592,7 +1592,7 @@ private void validateRmdData( serverWrapper.getVeniceServer().getStorageService().getStorageEngine(kafkaTopic); assertNotNull(storageEngine); ValueRecord result = SingleGetChunkingAdapter - .getReplicationMetadata(storageEngine, 0, serializeStringKeyToByteArray(key), true, null, null); + .getReplicationMetadata(storageEngine, 0, serializeStringKeyToByteArray(key), true, null); // Avoid assertion failure logging massive RMD record. boolean nullRmd = (result == null); assertFalse(nullRmd); @@ -2158,7 +2158,7 @@ private ChunkedValueManifest getChunkValueManifest( byte[] serializedKeyBytes = ChunkingUtils.KEY_WITH_CHUNKING_SUFFIX_SERIALIZER.serializeNonChunkedKey(serializeStringKeyToByteArray(key)); byte[] manifestValueBytes = isRmd - ? storageEngine.getReplicationMetadata(partition, serializedKeyBytes) + ? storageEngine.getReplicationMetadata(partition, ByteBuffer.wrap(serializedKeyBytes)) : storageEngine.get(partition, serializedKeyBytes); if (manifestValueBytes == null) { return null; @@ -2180,7 +2180,8 @@ private void validateChunkDataFromManifest( for (int i = 0; i < manifest.keysWithChunkIdSuffix.size(); i++) { byte[] chunkKeyBytes = manifest.keysWithChunkIdSuffix.get(i).array(); byte[] valueBytes = storageEngine.get(partition, chunkKeyBytes); - byte[] rmdBytes = isAAEnabled ? storageEngine.getReplicationMetadata(partition, chunkKeyBytes) : null; + byte[] rmdBytes = + isAAEnabled ? storageEngine.getReplicationMetadata(partition, ByteBuffer.wrap(chunkKeyBytes)) : null; validationFlow.accept(valueBytes, rmdBytes); } } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/fastclient/FastClientIndividualFeatureConfigurationTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/fastclient/FastClientIndividualFeatureConfigurationTest.java index 63a61ad1ed..425a7e5e04 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/fastclient/FastClientIndividualFeatureConfigurationTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/fastclient/FastClientIndividualFeatureConfigurationTest.java @@ -245,9 +245,11 @@ public void testServerRejectReadComputeRequest() throws Exception { .project(VALUE_FIELD_NAME) .execute(Collections.singleton(key)) .get(TIME_OUT, TimeUnit.MILLISECONDS); - fail(); + fail("The compute request should have thrown an exception."); } catch (Exception clientException) { - assertTrue(ExceptionUtils.recursiveMessageContains(clientException, "Read compute is not enabled for the store")); + if (!ExceptionUtils.recursiveMessageContains(clientException, "Read compute is not enabled for the store")) { + fail("The exception message did not contain the expected string.", clientException); + } } VeniceResponseMap responseMapWhenComputeDisabled = genericFastClient.compute() diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/ingestionHeartbeat/IngestionHeartBeatTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/ingestionHeartbeat/IngestionHeartBeatTest.java index 5a841d318f..bf1edcc0ee 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/ingestionHeartbeat/IngestionHeartBeatTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/ingestionHeartbeat/IngestionHeartBeatTest.java @@ -111,10 +111,12 @@ public void setUp() { @AfterTest(alwaysRun = true) public void cleanupStore() { - String parentControllerUrl = parentController.getControllerUrl(); - try (ControllerClient parentControllerClient = - new ControllerClient(multiRegionMultiClusterWrapper.getClusterNames()[0], parentControllerUrl)) { - parentControllerClient.disableAndDeleteStore(storeName); + if (this.parentController != null) { + String parentControllerUrl = parentController.getControllerUrl(); + try (ControllerClient parentControllerClient = + new ControllerClient(multiRegionMultiClusterWrapper.getClusterNames()[0], parentControllerUrl)) { + parentControllerClient.disableAndDeleteStore(storeName); + } } } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/router/TestStreaming.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/router/TestStreaming.java index 1da5fa2472..c793a65afc 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/router/TestStreaming.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/router/TestStreaming.java @@ -113,9 +113,11 @@ public void setUp() throws InterruptedException, ExecutionException, VeniceClien Utils.thisIsLocalhost(); veniceCluster = ServiceFactory.getVeniceCluster(1, 2, 0, 2, 100, true, false); - Properties serverProperties = new Properties(); Properties serverFeatureProperties = new Properties(); serverFeatureProperties.put(VeniceServerWrapper.SERVER_ENABLE_SSL, "true"); + Properties serverProperties = new Properties(); + serverProperties.put(ConfigKeys.SERVER_ENABLE_PARALLEL_BATCH_GET, true); + serverProperties.put(ConfigKeys.SERVER_PARALLEL_BATCH_GET_CHUNK_SIZE, 100); veniceCluster.addVeniceServer(serverFeatureProperties, serverProperties); // Create test store diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/storagenode/StorageNodeComputeTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/storagenode/StorageNodeComputeTest.java index 418cbdf16d..bf48f492a7 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/storagenode/StorageNodeComputeTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/storagenode/StorageNodeComputeTest.java @@ -31,6 +31,7 @@ import com.linkedin.venice.utils.IntegrationTestPushUtils; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Utils; +import com.linkedin.venice.utils.ValueSize; import com.linkedin.venice.writer.VeniceWriter; import com.linkedin.venice.writer.VeniceWriterFactory; import com.linkedin.venice.writer.VeniceWriterOptions; @@ -75,16 +76,6 @@ enum AvroImpl { } } - enum ValueSize { - SMALL_VALUE(false), LARGE_VALUE(true); - - final boolean config; - - ValueSize(boolean config) { - this.config = config; - } - } - private VeniceClusterWrapper veniceCluster; private AvroGenericStoreClient client; private int valueSchemaId; @@ -107,9 +98,11 @@ enum ValueSize { @BeforeClass(alwaysRun = true) public void setUp() throws InterruptedException, ExecutionException, VeniceClientException { veniceCluster = ServiceFactory.getVeniceCluster(1, 1, 0, 2, 100, false, false); - // Add one more server with fast-avro enabled + // Add one more server with all the bells and whistles: fast-avro, parallel batch get Properties serverProperties = new Properties(); serverProperties.put(ConfigKeys.SERVER_COMPUTE_FAST_AVRO_ENABLED, true); + serverProperties.put(ConfigKeys.SERVER_ENABLE_PARALLEL_BATCH_GET, true); + serverProperties.put(ConfigKeys.SERVER_PARALLEL_BATCH_GET_CHUNK_SIZE, 100); veniceCluster.addVeniceServer(new Properties(), serverProperties); // To trigger long-tail retry diff --git a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/ValueSize.java b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/ValueSize.java new file mode 100644 index 0000000000..b0d027b80e --- /dev/null +++ b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/ValueSize.java @@ -0,0 +1,11 @@ +package com.linkedin.venice.utils; + +public enum ValueSize { + SMALL_VALUE(false), LARGE_VALUE(true); + + public final boolean config; + + ValueSize(boolean config) { + this.config = config; + } +} diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/OutboundHttpWrapperHandler.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/OutboundHttpWrapperHandler.java index 3ed3cb8ee6..84b413ee2c 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/OutboundHttpWrapperHandler.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/OutboundHttpWrapperHandler.java @@ -10,11 +10,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.linkedin.davinci.listener.response.AdminResponse; import com.linkedin.davinci.listener.response.MetadataResponse; -import com.linkedin.davinci.listener.response.ReadResponse; import com.linkedin.davinci.listener.response.ServerCurrentVersionResponse; import com.linkedin.davinci.listener.response.TopicPartitionIngestionContextResponse; import com.linkedin.venice.HttpConstants; import com.linkedin.venice.compression.CompressionStrategy; +import com.linkedin.venice.listener.response.AbstractReadResponse; import com.linkedin.venice.listener.response.BinaryResponse; import com.linkedin.venice.listener.response.HttpShortcutResponse; import com.linkedin.venice.utils.ExceptionUtils; @@ -53,8 +53,8 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) CompressionStrategy compressionStrategy = CompressionStrategy.NO_OP; boolean isStreamingResponse = false; try { - if (msg instanceof ReadResponse) { - ReadResponse obj = (ReadResponse) msg; + if (msg instanceof AbstractReadResponse) { + AbstractReadResponse obj = (AbstractReadResponse) msg; ServerStatsContext statsContext = statsHandler.getServerStatsContext(); setStats(statsContext, obj); @@ -186,22 +186,7 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) ctx.writeAndFlush(response); } - public void setStats(ServerStatsContext statsContext, ReadResponse obj) { - statsContext.setDatabaseLookupLatency(obj.getDatabaseLookupLatency()); - statsContext.setStorageExecutionHandlerSubmissionWaitTime(obj.getStorageExecutionHandlerSubmissionWaitTime()); - statsContext.setStorageExecutionQueueLen(obj.getStorageExecutionQueueLen()); - statsContext.setSuccessRequestKeyCount(obj.getRecordCount()); - statsContext.setMultiChunkLargeValueCount(obj.getMultiChunkLargeValueCount()); - statsContext.setReadComputeLatency(obj.getReadComputeLatency()); - statsContext.setReadComputeDeserializationLatency(obj.getReadComputeDeserializationLatency()); - statsContext.setReadComputeSerializationLatency(obj.getReadComputeSerializationLatency()); - statsContext.setDotProductCount(obj.getDotProductCount()); - statsContext.setCosineSimilarityCount(obj.getCosineSimilarityCount()); - statsContext.setHadamardProductCount(obj.getHadamardProductCount()); - statsContext.setCountOperatorCount(obj.getCountOperatorCount()); - statsContext.setKeySizeList(obj.getKeySizeList()); - statsContext.setValueSizeList(obj.getValueSizeList()); - statsContext.setValueSize(obj.getValueSize()); - statsContext.setReadComputeOutputSize(obj.getReadComputeOutputSize()); + public void setStats(ServerStatsContext statsContext, AbstractReadResponse obj) { + statsContext.setReadResponseStats(obj.getStatsRecorder()); } } diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/ServerStatsContext.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/ServerStatsContext.java index f69c35192d..b990d4ef91 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/ServerStatsContext.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/ServerStatsContext.java @@ -1,13 +1,16 @@ package com.linkedin.venice.listener; +import static com.linkedin.venice.listener.response.stats.ResponseStatsUtil.consumeDoubleIfAbove; +import static com.linkedin.venice.listener.response.stats.ResponseStatsUtil.consumeIntIfAbove; + import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.listener.request.RouterRequest; +import com.linkedin.venice.listener.response.stats.ReadResponseStatsRecorder; import com.linkedin.venice.read.RequestType; import com.linkedin.venice.stats.AggServerHttpRequestStats; import com.linkedin.venice.stats.ServerHttpRequestStats; import io.netty.channel.ChannelHandlerContext; import io.netty.handler.codec.http.HttpResponseStatus; -import it.unimi.dsi.fastutil.ints.IntList; /** @@ -19,30 +22,14 @@ * direct copy of StatsHandler, without Netty Channel Read/Write logic. */ public class ServerStatsContext { + private ReadResponseStatsRecorder responseStatsRecorder; private long startTimeInNS; private HttpResponseStatus responseStatus; private String storeName = null; private boolean isMetadataRequest; - private double databaseLookupLatency = -1; - private int multiChunkLargeValueCount = -1; private int requestKeyCount = -1; - private int successRequestKeyCount = -1; private int requestSizeInBytes = -1; - private double readComputeLatency = -1; - private double readComputeDeserializationLatency = -1; - private double readComputeSerializationLatency = -1; - private int dotProductCount = 0; - private int cosineSimilarityCount = 0; - private int hadamardProductCount = 0; - private int countOperatorCount = 0; private boolean isRequestTerminatedEarly = false; - - private IntList keySizeList; - private IntList valueSizeList; - - private int valueSize = 0; - private int readComputeOutputSize = 0; - private final AggServerHttpRequestStats singleGetStats; private final AggServerHttpRequestStats multiGetStats; private final AggServerHttpRequestStats computeStats; @@ -56,8 +43,6 @@ public class ServerStatsContext { * This is mostly to bypass the issue that stat callback could be triggered multiple times for one single request. */ private boolean statCallbackExecuted = false; - private double storageExecutionSubmissionWaitTime; - private int storageExecutionQueueLen; /** * Normally, one multi-get request will be split into two parts, and it means @@ -65,16 +50,16 @@ public class ServerStatsContext { * * 'firstPartLatency' will measure the time took by: * {@link StatsHandler} - * {@link HttpServerCodec} - * {@link HttpObjectAggregator} + * {@link io.netty.handler.codec.http.HttpServerCodec} + * {@link io.netty.handler.codec.http.HttpObjectAggregator} * * 'partsInvokeDelayLatency' will measure the delay between the invocation of part1 * and the invocation of part2; * * 'secondPartLatency' will measure the time took by: * {@link StatsHandler} - * {@link HttpServerCodec} - * {@link HttpObjectAggregator} + * {@link io.netty.handler.codec.http.HttpServerCodec} + * {@link io.netty.handler.codec.http.HttpObjectAggregator} * {@link VerifySslHandler} * {@link ServerAclHandler} * {@link RouterRequestHttpHandler} @@ -85,8 +70,6 @@ public class ServerStatsContext { private double secondPartLatency = -1; private double partsInvokeDelayLatency = -1; private int requestPartCount = -1; - private boolean isComplete; - private boolean isMisroutedStoreVersion = false; private double flushLatency = -1; private int responseSize = -1; @@ -95,30 +78,14 @@ public boolean isNewRequest() { return newRequest; } - public double getSecondPartLatency() { - return secondPartLatency; - } - public void setSecondPartLatency(double secondPartLatency) { this.secondPartLatency = secondPartLatency; } - public double getPartsInvokeDelayLatency() { - return partsInvokeDelayLatency; - } - public void setPartsInvokeDelayLatency(double partsInvokeDelayLatency) { this.partsInvokeDelayLatency = partsInvokeDelayLatency; } - public int getRequestPartCount() { - return requestPartCount; - } - - public void setRequestPartCount(int requestPartCount) { - this.requestPartCount = requestPartCount; - } - public void incrementRequestPartCount() { this.requestPartCount++; } @@ -135,6 +102,7 @@ public ServerStatsContext( } public void resetContext() { + this.responseStatsRecorder = null; storeName = null; startTimeInNS = System.nanoTime(); partsInvokeDelayLatency = -1; @@ -143,21 +111,9 @@ public void resetContext() { isMetadataRequest = false; responseStatus = null; statCallbackExecuted = false; - databaseLookupLatency = -1; - storageExecutionSubmissionWaitTime = -1; - storageExecutionQueueLen = -1; requestKeyCount = -1; - successRequestKeyCount = -1; requestSizeInBytes = -1; - multiChunkLargeValueCount = -1; - readComputeLatency = -1; - readComputeDeserializationLatency = -1; - readComputeSerializationLatency = -1; - dotProductCount = 0; - cosineSimilarityCount = 0; - hadamardProductCount = 0; isRequestTerminatedEarly = false; - isComplete = false; isMisroutedStoreVersion = false; flushLatency = -1; responseSize = -1; @@ -165,6 +121,10 @@ public void resetContext() { newRequest = false; } + public void setReadResponseStats(ReadResponseStatsRecorder responseStatsRecorder) { + this.responseStatsRecorder = responseStatsRecorder; + } + public void setFirstPartLatency(double firstPartLatency) { this.firstPartLatency = firstPartLatency; } @@ -242,66 +202,6 @@ public void setRequestSize(int requestSizeInBytes) { this.requestSizeInBytes = requestSizeInBytes; } - public void setSuccessRequestKeyCount(int successKeyCount) { - this.successRequestKeyCount = successKeyCount; - } - - public void setDatabaseLookupLatency(double latency) { - this.databaseLookupLatency = latency; - } - - public void setReadComputeLatency(double latency) { - this.readComputeLatency = latency; - } - - public void setReadComputeDeserializationLatency(double latency) { - this.readComputeDeserializationLatency = latency; - } - - public void setReadComputeSerializationLatency(double latency) { - this.readComputeSerializationLatency = latency; - } - - public void setDotProductCount(int count) { - this.dotProductCount = count; - } - - public void setCosineSimilarityCount(int count) { - this.cosineSimilarityCount = count; - } - - public void setHadamardProductCount(int count) { - this.hadamardProductCount = count; - } - - public void setCountOperatorCount(int count) { - this.countOperatorCount = count; - } - - public void setStorageExecutionHandlerSubmissionWaitTime(double storageExecutionSubmissionWaitTime) { - this.storageExecutionSubmissionWaitTime = storageExecutionSubmissionWaitTime; - } - - public void setStorageExecutionQueueLen(int storageExecutionQueueLen) { - this.storageExecutionQueueLen = storageExecutionQueueLen; - } - - public boolean isAssembledMultiChunkLargeValue() { - return multiChunkLargeValueCount > 0; - } - - public void setMultiChunkLargeValueCount(int multiChunkLargeValueCount) { - this.multiChunkLargeValueCount = multiChunkLargeValueCount; - } - - public void setKeySizeList(IntList keySizeList) { - this.keySizeList = keySizeList; - } - - public void setValueSizeList(IntList valueSizeList) { - this.valueSizeList = valueSizeList; - } - public long getRequestStartTimeInNS() { return this.startTimeInNS; } @@ -316,82 +216,23 @@ public void setResponseSize(int size) { public void recordBasicMetrics(ServerHttpRequestStats serverHttpRequestStats) { if (serverHttpRequestStats != null) { - if (databaseLookupLatency >= 0) { - serverHttpRequestStats.recordDatabaseLookupLatency(databaseLookupLatency, isAssembledMultiChunkLargeValue()); - } - if (storageExecutionSubmissionWaitTime >= 0) { - currentStats.recordStorageExecutionHandlerSubmissionWaitTime(storageExecutionSubmissionWaitTime); - } - if (storageExecutionQueueLen >= 0) { - currentStats.recordStorageExecutionQueueLen(storageExecutionQueueLen); - } - if (multiChunkLargeValueCount > 0) { - // We only record this metric for requests where large values occurred - serverHttpRequestStats.recordMultiChunkLargeValueCount(multiChunkLargeValueCount); - } - if (requestKeyCount > 0) { - serverHttpRequestStats.recordRequestKeyCount(requestKeyCount); - } - if (successRequestKeyCount > 0) { - serverHttpRequestStats.recordSuccessRequestKeyCount(successRequestKeyCount); - } - if (requestSizeInBytes > 0) { - serverHttpRequestStats.recordRequestSizeInBytes(requestSizeInBytes); - } - if (firstPartLatency > 0) { - serverHttpRequestStats.recordRequestFirstPartLatency(firstPartLatency); - } - if (partsInvokeDelayLatency > 0) { - serverHttpRequestStats.recordRequestPartsInvokeDelayLatency(partsInvokeDelayLatency); - } - if (secondPartLatency > 0) { - serverHttpRequestStats.recordRequestSecondPartLatency(secondPartLatency); - } - if (requestPartCount > 0) { - serverHttpRequestStats.recordRequestPartCount(requestPartCount); - } - if (readComputeLatency >= 0) { - serverHttpRequestStats.recordReadComputeLatency(readComputeLatency, isAssembledMultiChunkLargeValue()); + if (this.responseStatsRecorder != null) { + this.responseStatsRecorder.recordMetrics(serverHttpRequestStats); } - if (readComputeDeserializationLatency >= 0) { - serverHttpRequestStats.recordReadComputeDeserializationLatency( - readComputeDeserializationLatency, - isAssembledMultiChunkLargeValue()); - } - if (readComputeSerializationLatency >= 0) { - serverHttpRequestStats - .recordReadComputeSerializationLatency(readComputeSerializationLatency, isAssembledMultiChunkLargeValue()); - } - if (dotProductCount > 0) { - serverHttpRequestStats.recordDotProductCount(dotProductCount); - } - if (cosineSimilarityCount > 0) { - serverHttpRequestStats.recordCosineSimilarityCount(cosineSimilarityCount); - } - if (hadamardProductCount > 0) { - serverHttpRequestStats.recordHadamardProduct(hadamardProductCount); - } - if (countOperatorCount > 0) { - serverHttpRequestStats.recordCountOperator(countOperatorCount); - } - if (isRequestTerminatedEarly) { + + consumeIntIfAbove(serverHttpRequestStats::recordRequestKeyCount, this.requestKeyCount, 0); + consumeIntIfAbove(serverHttpRequestStats::recordRequestSizeInBytes, this.requestSizeInBytes, 0); + consumeDoubleIfAbove(serverHttpRequestStats::recordRequestFirstPartLatency, this.firstPartLatency, 0); + consumeDoubleIfAbove( + serverHttpRequestStats::recordRequestPartsInvokeDelayLatency, + this.partsInvokeDelayLatency, + 0); + consumeDoubleIfAbove(serverHttpRequestStats::recordRequestSecondPartLatency, this.secondPartLatency, 0); + consumeIntIfAbove(serverHttpRequestStats::recordRequestPartCount, this.requestPartCount, 0); + + if (this.isRequestTerminatedEarly) { serverHttpRequestStats.recordEarlyTerminatedEarlyRequest(); } - if (keySizeList != null) { - for (int i = 0; i < keySizeList.size(); i++) { - serverHttpRequestStats.recordKeySizeInByte(keySizeList.getInt(i)); - } - } - if (valueSizeList != null) { - for (int i = 0; i < valueSizeList.size(); i++) { - if (valueSizeList.getInt(i) != -1) { - serverHttpRequestStats.recordValueSizeInByte(valueSizeList.getInt(i)); - } - } - } - if (readComputeOutputSize > 0) { - serverHttpRequestStats.recordReadComputeEfficiency((double) valueSize / readComputeOutputSize); - } if (flushLatency >= 0) { serverHttpRequestStats.recordFlushLatency(flushLatency); } @@ -404,7 +245,6 @@ public void recordBasicMetrics(ServerHttpRequestStats serverHttpRequestStats) { // This method does not have to be synchronized since operations in Tehuti are already synchronized. // Please re-consider the race condition if new logic is added. public void successRequest(ServerHttpRequestStats stats, double elapsedTime) { - isComplete = true; if (stats != null) { stats.recordSuccessRequest(); stats.recordSuccessRequestLatency(elapsedTime); @@ -414,7 +254,6 @@ public void successRequest(ServerHttpRequestStats stats, double elapsedTime) { } public void errorRequest(ServerHttpRequestStats stats, double elapsedTime) { - isComplete = true; if (stats == null) { currentStats.recordErrorRequest(); currentStats.recordErrorRequestLatency(elapsedTime); @@ -430,27 +269,11 @@ public void errorRequest(ServerHttpRequestStats stats, double elapsedTime) { } } - public void setValueSize(int size) { - this.valueSize = size; - } - - public void setReadComputeOutputSize(int size) { - this.readComputeOutputSize = size; - } - public int getRequestKeyCount() { return requestKeyCount; } - public boolean isComplete() { - return isComplete; - } - public void setMisroutedStoreVersion(boolean misroutedStoreVersion) { isMisroutedStoreVersion = misroutedStoreVersion; } - - public boolean isMisroutedStoreVersion() { - return isMisroutedStoreVersion; - } } diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/StatsHandler.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/StatsHandler.java index 01d8f26b35..f21bc3b32b 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/StatsHandler.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/StatsHandler.java @@ -6,7 +6,6 @@ import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.listener.request.RouterRequest; -import com.linkedin.venice.read.RequestType; import com.linkedin.venice.stats.AggServerHttpRequestStats; import com.linkedin.venice.stats.ServerHttpRequestStats; import com.linkedin.venice.utils.LatencyUtils; @@ -15,7 +14,6 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPromise; import io.netty.handler.codec.http.HttpResponseStatus; -import it.unimi.dsi.fastutil.ints.IntList; public class StatsHandler extends ChannelDuplexHandler { @@ -55,14 +53,6 @@ public void setRequestTerminatedEarly() { serverStatsContext.setRequestTerminatedEarly(); } - public void setRequestType(RequestType requestType) { - serverStatsContext.setRequestType(requestType); - } - - public void setRequestKeyCount(int keyCount) { - serverStatsContext.setRequestKeyCount(keyCount); - } - public void setRequestInfo(RouterRequest request) { serverStatsContext.setRequestInfo(request); } @@ -71,74 +61,6 @@ public void setRequestSize(int requestSizeInBytes) { serverStatsContext.setRequestSize(requestSizeInBytes); } - public void setSuccessRequestKeyCount(int successKeyCount) { - serverStatsContext.setSuccessRequestKeyCount(successKeyCount); - } - - public void setDatabaseLookupLatency(double latency) { - serverStatsContext.setDatabaseLookupLatency(latency); - } - - public void setReadComputeLatency(double latency) { - serverStatsContext.setReadComputeLatency(latency); - } - - public void setReadComputeDeserializationLatency(double latency) { - serverStatsContext.setReadComputeDeserializationLatency(latency); - } - - public void setReadComputeSerializationLatency(double latency) { - serverStatsContext.setReadComputeSerializationLatency(latency); - } - - public void setDotProductCount(int count) { - serverStatsContext.setDotProductCount(count); - } - - public void setCosineSimilarityCount(int count) { - serverStatsContext.setCosineSimilarityCount(count); - } - - public void setHadamardProductCount(int count) { - serverStatsContext.setHadamardProductCount(count); - } - - public void setCountOperatorCount(int count) { - serverStatsContext.setCountOperatorCount(count); - } - - public void setStorageExecutionHandlerSubmissionWaitTime(double storageExecutionSubmissionWaitTime) { - serverStatsContext.setStorageExecutionHandlerSubmissionWaitTime(storageExecutionSubmissionWaitTime); - } - - public void setStorageExecutionQueueLen(int storageExecutionQueueLen) { - serverStatsContext.setStorageExecutionQueueLen(storageExecutionQueueLen); - } - - public boolean isAssembledMultiChunkLargeValue() { - return serverStatsContext.isAssembledMultiChunkLargeValue(); - } - - public void setMultiChunkLargeValueCount(int multiChunkLargeValueCount) { - serverStatsContext.setMultiChunkLargeValueCount(multiChunkLargeValueCount); - } - - public void setKeySizeList(IntList keySizeList) { - serverStatsContext.setKeySizeList(keySizeList); - } - - public void setValueSizeList(IntList valueSizeList) { - serverStatsContext.setValueSizeList(valueSizeList); - } - - public void setValueSize(int valueSize) { - serverStatsContext.setValueSize(valueSize); - } - - public void setReadComputeOutputSize(int readComputeOutputSize) { - serverStatsContext.setReadComputeOutputSize(readComputeOutputSize); - } - public long getRequestStartTimeInNS() { return serverStatsContext.getRequestStartTimeInNS(); } 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 ed9f7bac36..4e3b1de9f6 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 @@ -6,6 +6,7 @@ import com.linkedin.davinci.listener.response.AdminResponse; import com.linkedin.davinci.listener.response.MetadataResponse; import com.linkedin.davinci.listener.response.ReadResponse; +import com.linkedin.davinci.listener.response.ReadResponseStats; import com.linkedin.davinci.listener.response.ServerCurrentVersionResponse; import com.linkedin.davinci.listener.response.TopicPartitionIngestionContextResponse; import com.linkedin.davinci.storage.DiskHealthCheckService; @@ -18,6 +19,7 @@ import com.linkedin.davinci.store.AbstractStorageEngine; import com.linkedin.davinci.store.record.ValueRecord; import com.linkedin.venice.cleaner.ResourceReadUsageTracker; +import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.compression.VeniceCompressor; import com.linkedin.venice.compute.ComputeUtils; import com.linkedin.venice.compute.protocol.request.ComputeOperation; @@ -36,18 +38,22 @@ import com.linkedin.venice.listener.request.HealthCheckRequest; import com.linkedin.venice.listener.request.MetadataFetchRequest; import com.linkedin.venice.listener.request.MultiGetRouterRequestWrapper; +import com.linkedin.venice.listener.request.MultiKeyRouterRequestWrapper; import com.linkedin.venice.listener.request.RouterRequest; import com.linkedin.venice.listener.request.TopicPartitionIngestionContextRequest; import com.linkedin.venice.listener.response.BinaryResponse; 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.StorageResponseObject; +import com.linkedin.venice.listener.response.MultiKeyResponseWrapper; +import com.linkedin.venice.listener.response.ParallelMultiKeyResponseWrapper; +import com.linkedin.venice.listener.response.SingleGetResponseWrapper; +import com.linkedin.venice.listener.response.stats.ComputeResponseStatsWithSizeProfiling; +import com.linkedin.venice.listener.response.stats.MultiGetResponseStatsWithSizeProfiling; import com.linkedin.venice.meta.ReadOnlySchemaRepository; import com.linkedin.venice.meta.ReadOnlyStoreRepository; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.Version; -import com.linkedin.venice.read.RequestType; import com.linkedin.venice.read.protocol.request.router.MultiGetRouterRequestKeyV1; import com.linkedin.venice.read.protocol.response.MultiGetResponseRecordV1; import com.linkedin.venice.schema.SchemaData; @@ -69,21 +75,18 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.handler.codec.http.HttpResponseStatus; -import it.unimi.dsi.fastutil.ints.IntArrayList; -import it.unimi.dsi.fastutil.ints.IntList; -import it.unimi.dsi.fastutil.ints.IntLists; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.CompletionException; import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Consumer; import java.util.function.Function; +import java.util.function.IntFunction; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; @@ -114,15 +117,23 @@ public class StorageReadRequestHandler extends ChannelInboundHandlerAdapter { private final Map computeResultSchemaCache; private final boolean fastAvroEnabled; private final Function> genericSerializerGetter; - private final boolean parallelBatchGetEnabled; private final int parallelBatchGetChunkSize; - private final boolean keyValueProfilingEnabled; private final VeniceServerConfig serverConfig; private final Map perStoreVersionStateMap = new VeniceConcurrentHashMap<>(); private final Map> storeDeserializerCacheMap = new VeniceConcurrentHashMap<>(); private final StorageEngineBackedCompressorFactory compressorFactory; - private final Optional resourceReadUsageTracker; + private final Consumer resourceReadUsageTracker; + + /** + * The function handles below are used to drive the K/V size profiling, which is enabled (or not) by an immutable + * config determined at the time we construct the {@link StorageReadRequestHandler}. This way, we don't need to + * evaluate the config flag during every request. + */ + private final IntFunction multiGetResponseProvider; + private final IntFunction computeResponseProvider; + private final Function> multiGetHandler; + private final Function> computeHandler; private static class PerStoreVersionState { final StoreDeserializerCache storeDeserializerCache; @@ -187,7 +198,7 @@ public StorageReadRequestHandler( int parallelBatchGetChunkSize, VeniceServerConfig serverConfig, StorageEngineBackedCompressorFactory compressorFactory, - Optional resourceReadUsageTracker) { + Optional optionalResourceReadUsageTracker) { this.executor = executor; this.computeExecutor = computeExecutor; this.storageEngineRepository = storageEngineRepository; @@ -201,139 +212,101 @@ public StorageReadRequestHandler( ? FastSerializerDeserializerFactory::getFastAvroGenericSerializer : SerializerDeserializerFactory::getAvroGenericSerializer; this.computeResultSchemaCache = new VeniceConcurrentHashMap<>(); - this.parallelBatchGetEnabled = parallelBatchGetEnabled; this.parallelBatchGetChunkSize = parallelBatchGetChunkSize; - this.keyValueProfilingEnabled = serverConfig.isKeyValueProfilingEnabled(); + if (parallelBatchGetEnabled) { + this.multiGetHandler = this::handleMultiGetRequestInParallel; + this.computeHandler = this::handleComputeRequestInParallel; + } else { + this.multiGetHandler = this::handleMultiGetRequest; + this.computeHandler = this::handleComputeRequest; + } + boolean keyValueProfilingEnabled = serverConfig.isKeyValueProfilingEnabled(); + if (keyValueProfilingEnabled) { + this.multiGetResponseProvider = + s -> new MultiGetResponseWrapper(s, new MultiGetResponseStatsWithSizeProfiling(s)); + this.computeResponseProvider = s -> new ComputeResponseWrapper(s, new ComputeResponseStatsWithSizeProfiling(s)); + } else { + this.multiGetResponseProvider = MultiGetResponseWrapper::new; + this.computeResponseProvider = ComputeResponseWrapper::new; + } this.serverConfig = serverConfig; this.compressorFactory = compressorFactory; - this.resourceReadUsageTracker = resourceReadUsageTracker; + if (optionalResourceReadUsageTracker.isPresent()) { + ResourceReadUsageTracker tracker = optionalResourceReadUsageTracker.get(); + this.resourceReadUsageTracker = tracker::recordReadUsage; + } else { + this.resourceReadUsageTracker = ignored -> {}; + } } @Override public void channelRead(ChannelHandlerContext context, Object message) throws Exception { - final long preSubmissionTimeNs = System.nanoTime(); - - /** - * N.B.: This is the only place in the entire class where we submit things into the {@link executor}. - * - * The reason for this is two-fold: - * - * 1. We want to make the {@link StorageReadRequestHandler} fully non-blocking as far as Netty (which - * is the one calling this function) is concerned. Therefore, it is beneficial to fork off the - * work into the executor from the very beginning. - * 2. By making the execution asynchronous from the beginning, we can simplify the rest of the class - * by making every other function a blocking one. If there is a desire to introduce additional - * concurrency in the rest of the class (i.e.: to make batch gets or large value re-assembly - * parallel), then it would be good to carefully consider whether this is a premature optimization, - * and if not, whether these additional operations should be performed in the same executor or in - * a secondary one, so as to not starve the primary requests. Furthermore, it should be considered - * whether it might be more beneficial to do streaming of these large response use cases, rather - * than parallel operations gated behind a synchronization barrier before any of the response can - * be sent out. - */ - if (message instanceof RouterRequest) { RouterRequest request = (RouterRequest) message; - resourceReadUsageTracker.ifPresent(tracker -> tracker.recordReadUsage(request.getResourceName())); + this.resourceReadUsageTracker.accept(request.getResourceName()); // Check before putting the request to the intermediate queue if (request.shouldRequestBeTerminatedEarly()) { // Try to make the response short - VeniceRequestEarlyTerminationException earlyTerminationException = - new VeniceRequestEarlyTerminationException(request.getStoreName()); context.writeAndFlush( new HttpShortcutResponse( - earlyTerminationException.getMessage(), - earlyTerminationException.getHttpResponseStatus())); + VeniceRequestEarlyTerminationException.getMessage(request.getStoreName()), + VeniceRequestEarlyTerminationException.getHttpResponseStatus())); return; } - /** - * For now, we are evaluating whether parallel lookup is good overall or not. - * Eventually, we either pick up the new parallel implementation or keep the original one, so it is fine - * to have some duplicate code for the time-being. - */ - if (parallelBatchGetEnabled && request.getRequestType().equals(RequestType.MULTI_GET)) { - handleMultiGetRequestInParallel((MultiGetRouterRequestWrapper) request, parallelBatchGetChunkSize) - .whenComplete((v, e) -> { - if (e != null) { - if (e instanceof VeniceRequestEarlyTerminationException) { - VeniceRequestEarlyTerminationException earlyTerminationException = - (VeniceRequestEarlyTerminationException) e; - context.writeAndFlush( - new HttpShortcutResponse( - earlyTerminationException.getMessage(), - earlyTerminationException.getHttpResponseStatus())); - } else if (e instanceof VeniceNoStoreException) { - HttpResponseStatus status = getHttpResponseStatus((VeniceNoStoreException) e); - context.writeAndFlush( - new HttpShortcutResponse( - "No storage exists for: " + ((VeniceNoStoreException) e).getStoreName(), - status)); - } else { - LOGGER.error("Exception thrown in parallel batch get for {}", request.getResourceName(), e); - HttpShortcutResponse shortcutResponse = - new HttpShortcutResponse(e.getMessage(), HttpResponseStatus.INTERNAL_SERVER_ERROR); - shortcutResponse.setMisroutedStoreVersion(checkMisroutedStoreVersionRequest(request)); - context.writeAndFlush(shortcutResponse); - } - } else { - context.writeAndFlush(v); - } - }); - return; + + CompletableFuture responseFuture; + switch (request.getRequestType()) { + case SINGLE_GET: + responseFuture = handleSingleGetRequest((GetRouterRequest) request); + break; + case MULTI_GET: + responseFuture = this.multiGetHandler.apply((MultiGetRouterRequestWrapper) request); + break; + case COMPUTE: + responseFuture = this.computeHandler.apply((ComputeRouterRequestWrapper) request); + break; + default: + throw new VeniceException("Unknown request type: " + request.getRequestType()); } - final ThreadPoolExecutor executor = getExecutor(request.getRequestType()); - executor.submit(() -> { - try { - if (request.shouldRequestBeTerminatedEarly()) { - throw new VeniceRequestEarlyTerminationException(request.getStoreName()); - } - double submissionWaitTime = LatencyUtils.getElapsedTimeFromNSToMS(preSubmissionTimeNs); - int queueLen = executor.getQueue().size(); - ReadResponse response; - switch (request.getRequestType()) { - case SINGLE_GET: - response = handleSingleGetRequest((GetRouterRequest) request); - break; - case MULTI_GET: - response = handleMultiGetRequest((MultiGetRouterRequestWrapper) request); - break; - case COMPUTE: - response = handleComputeRequest((ComputeRouterRequestWrapper) message); - break; - default: - throw new VeniceException("Unknown request type: " + request.getRequestType()); - } - response.setStorageExecutionSubmissionWaitTime(submissionWaitTime); - response.setStorageExecutionQueueLen(queueLen); + responseFuture.whenComplete((response, throwable) -> { + if (throwable == null) { response.setRCU(ReadQuotaEnforcementHandler.getRcu(request)); if (request.isStreamingRequest()) { response.setStreamingResponse(); } context.writeAndFlush(response); - } catch (VeniceNoStoreException e) { + return; + } + if (throwable instanceof CompletionException && throwable.getCause() != null) { + throwable = throwable.getCause(); + } + if (throwable instanceof VeniceNoStoreException) { + VeniceNoStoreException e = (VeniceNoStoreException) throwable; String msg = "No storage exists for store: " + e.getStoreName(); if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { LOGGER.error(msg, e); } HttpResponseStatus status = getHttpResponseStatus(e); context.writeAndFlush(new HttpShortcutResponse("No storage exists for: " + e.getStoreName(), status)); - } catch (VeniceRequestEarlyTerminationException e) { + } else if (throwable instanceof VeniceRequestEarlyTerminationException) { + VeniceRequestEarlyTerminationException e = (VeniceRequestEarlyTerminationException) throwable; String msg = "Request timed out for store: " + e.getStoreName(); if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { LOGGER.error(msg, e); } context.writeAndFlush(new HttpShortcutResponse(e.getMessage(), HttpResponseStatus.REQUEST_TIMEOUT)); - } catch (OperationNotAllowedException e) { + } else if (throwable instanceof OperationNotAllowedException) { + OperationNotAllowedException e = (OperationNotAllowedException) throwable; String msg = "METHOD_NOT_ALLOWED: " + e.getMessage(); if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { LOGGER.error(msg, e); } context.writeAndFlush(new HttpShortcutResponse(e.getMessage(), HttpResponseStatus.METHOD_NOT_ALLOWED)); - } catch (Exception e) { - LOGGER.error("Exception thrown for {}", request.getResourceName(), e); + } else { + LOGGER.error("Exception thrown for {}", request.getResourceName(), throwable); HttpShortcutResponse shortcutResponse = - new HttpShortcutResponse(e.getMessage(), HttpResponseStatus.INTERNAL_SERVER_ERROR); + new HttpShortcutResponse(throwable.getMessage(), HttpResponseStatus.INTERNAL_SERVER_ERROR); shortcutResponse.setMisroutedStoreVersion(checkMisroutedStoreVersionRequest(request)); context.writeAndFlush(shortcutResponse); } @@ -411,18 +384,6 @@ private boolean checkMisroutedStoreVersionRequest(RouterRequest request) { return misrouted; } - private ThreadPoolExecutor getExecutor(RequestType requestType) { - switch (requestType) { - case SINGLE_GET: - case MULTI_GET: - return executor; - case COMPUTE: - return computeExecutor; - default: - throw new VeniceException("Request type " + requestType + " is not supported."); - } - } - private PerStoreVersionState getPerStoreVersionState(String storeVersion) { PerStoreVersionState s = perStoreVersionStateMap.computeIfAbsent(storeVersion, this::generatePerStoreVersionState); if (s.storageEngine.isClosed()) { @@ -454,129 +415,126 @@ private AbstractStorageEngine getStorageEngineOrThrow(String storeVersion) { return storageEngine; } - public ReadResponse handleSingleGetRequest(GetRouterRequest request) { - String topic = request.getResourceName(); - PerStoreVersionState perStoreVersionState = getPerStoreVersionState(topic); - byte[] key = request.getKeyBytes(); + public CompletableFuture handleSingleGetRequest(GetRouterRequest request) { + final int queueLen = this.executor.getQueue().size(); + final long preSubmissionTimeNs = System.nanoTime(); + return CompletableFuture.supplyAsync(() -> { + if (request.shouldRequestBeTerminatedEarly()) { + throw new VeniceRequestEarlyTerminationException(request.getStoreName()); + } - AbstractStorageEngine storageEngine = perStoreVersionState.storageEngine; - boolean isChunked = storageEngine.isChunked(); - StorageResponseObject response = new StorageResponseObject(); - response.setCompressionStrategy(storageEngine.getCompressionStrategy()); - response.setDatabaseLookupLatency(0); + double submissionWaitTime = LatencyUtils.getElapsedTimeFromNSToMS(preSubmissionTimeNs); - ValueRecord valueRecord = - SingleGetChunkingAdapter.get(storageEngine, request.getPartition(), key, isChunked, response); - response.setValueRecord(valueRecord); + String topic = request.getResourceName(); + PerStoreVersionState perStoreVersionState = getPerStoreVersionState(topic); + byte[] key = request.getKeyBytes(); - if (keyValueProfilingEnabled) { - response.setKeySizeList(IntLists.singleton(key.length)); - response.setValueSizeList(IntLists.singleton(response.isFound() ? valueRecord.getDataSize() : -1)); - } + AbstractStorageEngine storageEngine = perStoreVersionState.storageEngine; + boolean isChunked = storageEngine.isChunked(); + SingleGetResponseWrapper response = new SingleGetResponseWrapper(); + response.setCompressionStrategy(storageEngine.getCompressionStrategy()); - return response; + ValueRecord valueRecord = + SingleGetChunkingAdapter.get(storageEngine, request.getPartition(), key, isChunked, response.getStats()); + response.setValueRecord(valueRecord); + + response.getStats().addKeySize(key.length); + response.getStats().setStorageExecutionSubmissionWaitTime(submissionWaitTime); + response.getStats().setStorageExecutionQueueLen(queueLen); + + return response; + }); } - private CompletableFuture handleMultiGetRequestInParallel( - MultiGetRouterRequestWrapper request, - int parallelChunkSize) { - String topic = request.getResourceName(); - Iterable keys = request.getKeys(); - PerStoreVersionState perStoreVersionState = getPerStoreVersionState(topic); - AbstractStorageEngine storageEngine = perStoreVersionState.storageEngine; - - MultiGetResponseWrapper responseWrapper = new MultiGetResponseWrapper(request.getKeyCount()); - responseWrapper.setCompressionStrategy(storageEngine.getCompressionStrategy()); - responseWrapper.setDatabaseLookupLatency(0); - boolean isChunked = storageEngine.isChunked(); - - ExecutorService executorService = getExecutor(RequestType.MULTI_GET); - if (!(keys instanceof ArrayList)) { - throw new VeniceException("'keys' in MultiGetResponseWrapper should be an ArrayList"); - } - final ArrayList keyList = (ArrayList) keys; - int totalKeyNum = keyList.size(); - int splitSize = (int) Math.ceil((double) totalKeyNum / parallelChunkSize); + private CompletableFuture handleMultiGetRequestInParallel(MultiGetRouterRequestWrapper request) { + List keys = request.getKeys(); + RequestContext requestContext = new RequestContext(request, this); + + return processBatchInParallel( + keys, + requestContext.storeVersion.storageEngine.getCompressionStrategy(), + request, + ParallelMultiKeyResponseWrapper::multiGet, + this.multiGetResponseProvider, + this.executor, + requestContext, + this::processMultiGet); + } - ReentrantLock requestLock = new ReentrantLock(); - CompletableFuture[] chunkFutures = new CompletableFuture[splitSize]; + private interface ParallelResponseProvider { + ParallelMultiKeyResponseWrapper get(int chunkCount, int chunkSize, IntFunction responseProvider); + } - IntList responseKeySizeList = keyValueProfilingEnabled ? new IntArrayList(totalKeyNum) : null; - IntList responseValueSizeList = keyValueProfilingEnabled ? new IntArrayList(totalKeyNum) : null; + private interface SingleBatchProcessor { + void process(int startPos, int endPos, List keys, C requestContext, R chunkOfResponse); + } - for (int cur = 0; cur < splitSize; ++cur) { + private CompletableFuture processBatchInParallel( + List keys, + CompressionStrategy compressionStrategy, + MultiKeyRouterRequestWrapper request, + ParallelResponseProvider parallelResponseProvider, + IntFunction individualResponseProvider, + ThreadPoolExecutor threadPoolExecutor, + C requestContext, + SingleBatchProcessor batchProcessor) { + int totalKeyNum = keys.size(); + int chunkCount = (int) Math.ceil((double) totalKeyNum / this.parallelBatchGetChunkSize); + ParallelMultiKeyResponseWrapper responseWrapper = + parallelResponseProvider.get(chunkCount, this.parallelBatchGetChunkSize, individualResponseProvider); + responseWrapper.setCompressionStrategy(compressionStrategy); + + CompletableFuture[] chunkFutures = new CompletableFuture[chunkCount]; + + final int queueLen = threadPoolExecutor.getQueue().size(); + final long preSubmissionTimeNs = System.nanoTime(); + for (int cur = 0; cur < chunkCount; ++cur) { final int finalCur = cur; chunkFutures[cur] = CompletableFuture.runAsync(() -> { + double submissionWaitTime = LatencyUtils.getElapsedTimeFromNSToMS(preSubmissionTimeNs); + if (request.shouldRequestBeTerminatedEarly()) { throw new VeniceRequestEarlyTerminationException(request.getStoreName()); } - int startPos = finalCur * parallelChunkSize; - int endPos = Math.min((finalCur + 1) * parallelChunkSize, totalKeyNum); - for (int subChunkCur = startPos; subChunkCur < endPos; ++subChunkCur) { - final MultiGetRouterRequestKeyV1 key = keyList.get(subChunkCur); - if (responseKeySizeList != null) { - responseKeySizeList.set(subChunkCur, key.keyBytes.remaining()); - } - MultiGetResponseRecordV1 record = - BatchGetChunkingAdapter.get(storageEngine, key.partitionId, key.keyBytes, isChunked, responseWrapper); - if (record == null) { - if (request.isStreamingRequest()) { - // 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. - record = new MultiGetResponseRecordV1(); - // Negative key index to indicate the non-existing keys - record.keyIndex = Math.negateExact(key.keyIndex); - record.schemaId = StreamingConstants.NON_EXISTING_KEY_SCHEMA_ID; - record.value = StreamingUtils.EMPTY_BYTE_BUFFER; - } - } else { - record.keyIndex = key.keyIndex; - } - if (record != null) { - if (responseValueSizeList != null) { - responseValueSizeList.set(subChunkCur, record.value.remaining()); - } - // TODO: streaming support in storage node - requestLock.lock(); - try { - responseWrapper.addRecord(record); - } finally { - requestLock.unlock(); - } - } else { - if (responseValueSizeList != null) { - responseValueSizeList.set(subChunkCur, -1); - } - } - } - }, executorService); + int startPos = finalCur * this.parallelBatchGetChunkSize; + int endPos = Math.min((finalCur + 1) * this.parallelBatchGetChunkSize, totalKeyNum); + R chunkOfResponse = responseWrapper.getChunk(finalCur); + batchProcessor.process(startPos, endPos, keys, requestContext, chunkOfResponse); + + chunkOfResponse.getStats().setStorageExecutionSubmissionWaitTime(submissionWaitTime); + }, threadPoolExecutor); } return CompletableFuture.allOf(chunkFutures).handle((v, e) -> { if (e != null) { throw new VeniceException(e); } - responseWrapper.setKeySizeList(responseKeySizeList); - responseWrapper.setValueSizeList(responseValueSizeList); + + responseWrapper.getChunk(0).getStats().setStorageExecutionQueueLen(queueLen); return responseWrapper; }); } - public ReadResponse handleMultiGetRequest(MultiGetRouterRequestWrapper request) { - Iterable keys = request.getKeys(); - PerStoreVersionState perStoreVersionState = getPerStoreVersionState(request.getResourceName()); - AbstractStorageEngine storageEngine = perStoreVersionState.storageEngine; - - MultiGetResponseWrapper responseWrapper = new MultiGetResponseWrapper(request.getKeyCount()); - responseWrapper.setCompressionStrategy(storageEngine.getCompressionStrategy()); - responseWrapper.setDatabaseLookupLatency(0); - boolean isChunked = storageEngine.isChunked(); - for (MultiGetRouterRequestKeyV1 key: keys) { - MultiGetResponseRecordV1 record = - BatchGetChunkingAdapter.get(storageEngine, key.partitionId, key.keyBytes, isChunked, responseWrapper); + private void processMultiGet( + int startPos, + int endPos, + List keys, + RequestContext requestContext, + MultiGetResponseWrapper response) { + MultiGetRouterRequestKeyV1 key; + MultiGetResponseRecordV1 record; + for (int subChunkCur = startPos; subChunkCur < endPos; ++subChunkCur) { + key = keys.get(subChunkCur); + response.getStats().addKeySize(key.getKeyBytes().remaining()); + record = BatchGetChunkingAdapter.get( + requestContext.storeVersion.storageEngine, + key.partitionId, + key.keyBytes, + requestContext.isChunked, + response.getStats()); if (record == null) { - if (request.isStreamingRequest()) { + 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. record = new MultiGetResponseRecordV1(); @@ -584,63 +542,203 @@ record = new MultiGetResponseRecordV1(); record.keyIndex = Math.negateExact(key.keyIndex); record.schemaId = StreamingConstants.NON_EXISTING_KEY_SCHEMA_ID; record.value = StreamingUtils.EMPTY_BYTE_BUFFER; + response.addRecord(record); } } else { record.keyIndex = key.keyIndex; + response.addRecord(record); } + } + + // Trigger serialization + response.getResponseBody(); + } - if (record != null) { - // TODO: streaming support in storage node - responseWrapper.addRecord(record); + public CompletableFuture handleMultiGetRequest(MultiGetRouterRequestWrapper request) { + final int queueLen = this.executor.getQueue().size(); + final long preSubmissionTimeNs = System.nanoTime(); + return CompletableFuture.supplyAsync(() -> { + double submissionWaitTime = LatencyUtils.getElapsedTimeFromNSToMS(preSubmissionTimeNs); + + if (request.shouldRequestBeTerminatedEarly()) { + throw new VeniceRequestEarlyTerminationException(request.getStoreName()); } + + List keys = request.getKeys(); + MultiGetResponseWrapper responseWrapper = this.multiGetResponseProvider.apply(request.getKeyCount()); + RequestContext requestContext = new RequestContext(request, this); + responseWrapper.setCompressionStrategy(requestContext.storeVersion.storageEngine.getCompressionStrategy()); + + processMultiGet(0, request.getKeyCount(), keys, requestContext, responseWrapper); + + responseWrapper.getStats().setStorageExecutionSubmissionWaitTime(submissionWaitTime); + responseWrapper.getStats().setStorageExecutionQueueLen(queueLen); + return responseWrapper; + }, executor); + } + + private CompletableFuture handleComputeRequest(ComputeRouterRequestWrapper request) { + if (!metadataRepository.isReadComputationEnabled(request.getStoreName())) { + CompletableFuture failFast = new CompletableFuture(); + failFast.completeExceptionally( + new OperationNotAllowedException( + "Read compute is not enabled for the store. Please contact Venice team to enable the feature.")); + return failFast; } - return responseWrapper; + + final int queueLen = this.computeExecutor.getQueue().size(); + final long preSubmissionTimeNs = System.nanoTime(); + return CompletableFuture.supplyAsync(() -> { + if (request.shouldRequestBeTerminatedEarly()) { + throw new VeniceRequestEarlyTerminationException(request.getStoreName()); + } + + double submissionWaitTime = LatencyUtils.getElapsedTimeFromNSToMS(preSubmissionTimeNs); + + ComputeRequestContext computeRequestContext = new ComputeRequestContext(request, this); + int keyCount = request.getKeyCount(); + ComputeResponseWrapper response = this.computeResponseProvider.apply(keyCount); + + processCompute(0, keyCount, request.getKeys(), computeRequestContext, response); + + response.getStats().setStorageExecutionSubmissionWaitTime(submissionWaitTime); + response.getStats().setStorageExecutionQueueLen(queueLen); + return response; + }, computeExecutor); } - private ReadResponse handleComputeRequest(ComputeRouterRequestWrapper request) { + private CompletableFuture handleComputeRequestInParallel(ComputeRouterRequestWrapper request) { if (!metadataRepository.isReadComputationEnabled(request.getStoreName())) { - throw new OperationNotAllowedException( - "Read compute is not enabled for the store. Please contact Venice team to enable the feature."); + CompletableFuture failFast = new CompletableFuture(); + failFast.completeExceptionally( + new OperationNotAllowedException( + "Read compute is not enabled for the store. Please contact Venice team to enable the feature.")); + return failFast; } - SchemaEntry superSetOrLatestValueSchema = schemaRepository.getSupersetOrLatestValueSchema(request.getStoreName()); - SchemaEntry valueSchemaEntry = getComputeValueSchema(request, superSetOrLatestValueSchema); - Schema resultSchema = getComputeResultSchema(request.getComputeRequest(), valueSchemaEntry.getSchema()); - RecordSerializer resultSerializer = genericSerializerGetter.apply(resultSchema); - PerStoreVersionState storeVersion = getPerStoreVersionState(request.getResourceName()); - VeniceCompressor compressor = - compressorFactory.getCompressor(storeVersion.storageEngine.getCompressionStrategy(), request.getResourceName()); - - // Reuse the same value record and result record instances for all values + + List keys = request.getKeys(); + ComputeRequestContext requestContext = new ComputeRequestContext(request, this); + + return processBatchInParallel( + keys, + CompressionStrategy.NO_OP, + request, + ParallelMultiKeyResponseWrapper::compute, + this.computeResponseProvider, + this.computeExecutor, + requestContext, + this::processCompute); + } + + /** + * The request context holds state which the server needs to compute once per query, and which is safe to share across + * subtasks of the same query, as is the case when executing batch get and compute requests in parallel chunks. + */ + private static class RequestContext { + final PerStoreVersionState storeVersion; + final boolean isChunked; + final boolean isStreaming; + + RequestContext(MultiKeyRouterRequestWrapper request, StorageReadRequestHandler handler) { + this.storeVersion = handler.getPerStoreVersionState(request.getResourceName()); + this.isChunked = storeVersion.storageEngine.isChunked(); + this.isStreaming = request.isStreamingRequest(); + } + } + + private static class ComputeRequestContext extends RequestContext { + final SchemaEntry valueSchemaEntry; + final Schema resultSchema; + final VeniceCompressor compressor; + final RecordSerializer resultSerializer; + final List operations; + final List operationResultFields; + + ComputeRequestContext(ComputeRouterRequestWrapper request, StorageReadRequestHandler handler) { + super(request, handler); + this.valueSchemaEntry = handler.getComputeValueSchema(request); + this.resultSchema = handler.getComputeResultSchema(request.getComputeRequest(), valueSchemaEntry.getSchema()); + this.resultSerializer = handler.genericSerializerGetter.apply(resultSchema); + this.compressor = handler.compressorFactory + .getCompressor(storeVersion.storageEngine.getCompressionStrategy(), request.getResourceName()); + this.operations = request.getComputeRequest().getOperations(); + this.operationResultFields = ComputeUtils.getOperationResultFields(operations, resultSchema); + } + } + + private void processCompute( + int startPos, + int endPos, + List keys, + ComputeRequestContext requestContext, + ComputeResponseWrapper response) { + /** + * Reuse the same value record and result record instances for all values. This cannot be part of the + * {@link ComputeRequestContext}, otherwise it could get contaminated across threads. + */ ReusableObjects reusableObjects = threadLocalReusableObjects.get(); - GenericRecord reusableValueRecord = - reusableObjects.valueRecordMap.computeIfAbsent(valueSchemaEntry.getSchema(), GenericData.Record::new); + GenericRecord reusableValueRecord = reusableObjects.valueRecordMap + .computeIfAbsent(requestContext.valueSchemaEntry.getSchema(), GenericData.Record::new); GenericRecord reusableResultRecord = - reusableObjects.resultRecordMap.computeIfAbsent(resultSchema, GenericData.Record::new); + reusableObjects.resultRecordMap.computeIfAbsent(requestContext.resultSchema, GenericData.Record::new); reusableObjects.computeContext.clear(); - ComputeResponseWrapper response = new ComputeResponseWrapper(request.getKeyCount()); - List operations = request.getComputeRequest().getOperations(); - List operationResultFields = ComputeUtils.getOperationResultFields(operations, resultSchema); int hits = 0; - for (ComputeRouterRequestKeyV1 key: request.getKeys()) { + long serializeStartTimeInNS, computeStartTimeInNS; + ComputeRouterRequestKeyV1 key; + ComputeResponseRecordV1 record; + for (int subChunkCur = startPos; subChunkCur < endPos; ++subChunkCur) { + key = keys.get(subChunkCur); + response.getStats().addKeySize(key.getKeyBytes().remaining()); AvroRecordUtils.clearRecord(reusableResultRecord); - GenericRecord result = computeResult( - operations, - operationResultFields, - storeVersion, - key, + reusableValueRecord = GenericRecordChunkingAdapter.INSTANCE.get( + requestContext.storeVersion.storageEngine, + key.getPartitionId(), + ByteUtils.extractByteArray(key.getKeyBytes()), + reusableObjects.byteBuffer, reusableValueRecord, - valueSchemaEntry.getId(), - compressor, - response, - reusableObjects, - reusableResultRecord); - if (addComputationResult(response, key, result, resultSerializer, request.isStreamingRequest())) { + reusableObjects.binaryDecoder, + requestContext.isChunked, + response.getStats(), + requestContext.valueSchemaEntry.getId(), + requestContext.storeVersion.storeDeserializerCache, + requestContext.compressor); + if (reusableValueRecord != null) { + computeStartTimeInNS = System.nanoTime(); + reusableResultRecord = ComputeUtils.computeResult( + requestContext.operations, + requestContext.operationResultFields, + reusableObjects.computeContext, + reusableValueRecord, + reusableResultRecord); + + serializeStartTimeInNS = System.nanoTime(); // N.B. This clock call is also used as the end of the compute time + record = new ComputeResponseRecordV1(); + record.keyIndex = key.getKeyIndex(); + record.value = ByteBuffer.wrap(requestContext.resultSerializer.serialize(reusableResultRecord)); + + response.getStats() + .addReadComputeSerializationLatency(LatencyUtils.getElapsedTimeFromNSToMS(serializeStartTimeInNS)); + response.getStats() + .addReadComputeLatency(LatencyUtils.convertNSToMS(serializeStartTimeInNS - computeStartTimeInNS)); + response.getStats().addReadComputeOutputSize(record.value.remaining()); + + 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); } } - incrementOperatorCounters(response, operations, hits); - return response; + + // Trigger serialization + response.getResponseBody(); + + incrementOperatorCounters(response.getStats(), requestContext.operations, hits); } private BinaryResponse handleDictionaryFetchRequest(DictionaryFetchRequest request) { @@ -668,99 +766,19 @@ private Schema getComputeResultSchema(ComputeRequest computeRequest, Schema valu return resultSchema; } - private SchemaEntry getComputeValueSchema( - ComputeRouterRequestWrapper request, - SchemaEntry superSetOrLatestValueSchema) { + private SchemaEntry getComputeValueSchema(ComputeRouterRequestWrapper request) { + SchemaEntry superSetOrLatestValueSchema = schemaRepository.getSupersetOrLatestValueSchema(request.getStoreName()); return request.getValueSchemaId() != SchemaData.INVALID_VALUE_SCHEMA_ID ? schemaRepository.getValueSchema(request.getStoreName(), request.getValueSchemaId()) : superSetOrLatestValueSchema; } - /** - * @return true if the result is not null, false otherwise - */ - private boolean addComputationResult( - ComputeResponseWrapper response, - ComputeRouterRequestKeyV1 key, - GenericRecord result, - RecordSerializer resultSerializer, - boolean isStreaming) { - if (result != null) { - long serializeStartTimeInNS = System.nanoTime(); - ComputeResponseRecordV1 record = new ComputeResponseRecordV1(); - record.keyIndex = key.getKeyIndex(); - record.value = ByteBuffer.wrap(resultSerializer.serialize(result)); - response.addReadComputeSerializationLatency(LatencyUtils.getElapsedTimeFromNSToMS(serializeStartTimeInNS)); - response.addReadComputeOutputSize(record.value.remaining()); - response.addRecord(record); - return true; - } else if (isStreaming) { - // For streaming, we need to send back non-existing keys - ComputeResponseRecordV1 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); - } - return false; - } - - private GenericRecord computeResult( - List operations, - List operationResultFields, - PerStoreVersionState storeVersion, - ComputeRouterRequestKeyV1 key, - GenericRecord reusableValueRecord, - int readerSchemaId, - VeniceCompressor compressor, - ComputeResponseWrapper response, - ReusableObjects reusableObjects, - GenericRecord reusableResultRecord) { - reusableValueRecord = - readValueRecord(key, storeVersion, readerSchemaId, compressor, response, reusableObjects, reusableValueRecord); - if (reusableValueRecord == null) { - return null; - } - - long computeStartTimeInNS = System.nanoTime(); - reusableResultRecord = ComputeUtils.computeResult( - operations, - operationResultFields, - reusableObjects.computeContext, - reusableValueRecord, - reusableResultRecord); - response.addReadComputeLatency(LatencyUtils.getElapsedTimeFromNSToMS(computeStartTimeInNS)); - return reusableResultRecord; - } - - private GenericRecord readValueRecord( - ComputeRouterRequestKeyV1 key, - PerStoreVersionState storeVersion, - int readerSchemaId, - VeniceCompressor compressor, - ReadResponse response, - ReusableObjects reusableObjects, - GenericRecord reusableValueRecord) { - return GenericRecordChunkingAdapter.INSTANCE.get( - storeVersion.storageEngine, - key.getPartitionId(), - ByteUtils.extractByteArray(key.getKeyBytes()), - reusableObjects.byteBuffer, - reusableValueRecord, - reusableObjects.binaryDecoder, - storeVersion.storageEngine.isChunked(), - response, - readerSchemaId, - storeVersion.storeDeserializerCache, - compressor); - } - private static void incrementOperatorCounters( - ComputeResponseWrapper response, - Iterable operations, + ReadResponseStats response, + List operations, int hits) { - for (ComputeOperation operation: operations) { - switch (ComputeOperationType.valueOf(operation)) { + for (int i = 0; i < operations.size(); i++) { + switch (ComputeOperationType.valueOf(operations.get(i))) { case DOT_PRODUCT: response.incrementDotProductCount(hits); break; diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/VeniceRequestEarlyTerminationException.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/VeniceRequestEarlyTerminationException.java index 6b071390ca..86bcc81a48 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/VeniceRequestEarlyTerminationException.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/VeniceRequestEarlyTerminationException.java @@ -8,7 +8,7 @@ public class VeniceRequestEarlyTerminationException extends VeniceException { private final String storeName; public VeniceRequestEarlyTerminationException(String storeName) { - super("The request to store: " + storeName + " is terminated because of early termination setup"); + super(getMessage(storeName)); this.storeName = storeName; } @@ -21,6 +21,10 @@ public static HttpResponseStatus getHttpResponseStatus() { return HttpResponseStatus.REQUEST_TIMEOUT; } + public static String getMessage(String storeName) { + return "The request to store: " + storeName + " is terminated because of early termination setup"; + } + public String getStoreName() { return storeName; } diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/grpc/handlers/GrpcStorageReadRequestHandler.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/grpc/handlers/GrpcStorageReadRequestHandler.java index a929a71783..833ecddcac 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/grpc/handlers/GrpcStorageReadRequestHandler.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/grpc/handlers/GrpcStorageReadRequestHandler.java @@ -10,9 +10,19 @@ import com.linkedin.venice.listener.request.MultiGetRouterRequestWrapper; import com.linkedin.venice.listener.request.RouterRequest; import com.linkedin.venice.response.VeniceReadResponseStatus; -import com.linkedin.venice.utils.LatencyUtils; +import io.netty.channel.ChannelHandlerContext; +/** + * This class is an incomplete copypasta of the logic in {@link StorageReadRequestHandler#channelRead(ChannelHandlerContext, Object)}. + * + * Besides the maintenance issue of the repeated code, and the incomplete functionality support, another potentially big + * issue is that the threading model seems to be significantly different. This class does all the work in-line, in a + * blocking fashion. All of these disparities are likely to cause significant issues in terms of trying to ramp the gRPC + * path. + * + * TODO: Refactor with better abstractions so that gRPC and legacy endpoints have better code reuse and behavior parity. + */ public class GrpcStorageReadRequestHandler extends VeniceServerGrpcHandler { private final StorageReadRequestHandler storage; @@ -23,22 +33,21 @@ public GrpcStorageReadRequestHandler(StorageReadRequestHandler storage) { @Override public void processRequest(GrpcRequestContext ctx) { RouterRequest request = ctx.getRouterRequest(); - final long preSubmissionTimeNs = System.nanoTime(); ReadResponse response = null; - double submissionWaitTime = -1; try { if (request.shouldRequestBeTerminatedEarly()) { throw new VeniceRequestEarlyTerminationException(request.getStoreName()); } - submissionWaitTime = LatencyUtils.getElapsedTimeFromNSToMS(preSubmissionTimeNs); switch (request.getRequestType()) { case SINGLE_GET: - response = storage.handleSingleGetRequest((GetRouterRequest) request); + // TODO: get rid of blocking here + response = storage.handleSingleGetRequest((GetRouterRequest) request).get(); break; case MULTI_GET: - response = storage.handleMultiGetRequest((MultiGetRouterRequestWrapper) request); + // TODO: get rid of blocking here + response = storage.handleMultiGetRequest((MultiGetRouterRequestWrapper) request).get(); break; default: ctx.setError(); @@ -59,7 +68,6 @@ public void processRequest(GrpcRequestContext ctx) { } if (!ctx.hasError() && response != null) { - response.setStorageExecutionSubmissionWaitTime(submissionWaitTime); response.setRCU(ReadQuotaEnforcementHandler.getRcu(request)); if (request.isStreamingRequest()) { response.setStreamingResponse(); diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/request/MultiKeyRouterRequestWrapper.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/request/MultiKeyRouterRequestWrapper.java index 67ef89a0f0..c4ee1b1a85 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/request/MultiKeyRouterRequestWrapper.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/request/MultiKeyRouterRequestWrapper.java @@ -24,7 +24,7 @@ protected MultiKeyRouterRequestWrapper( this.keys = keys; } - public Iterable getKeys() { + public List getKeys() { return this.keys; } diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/AbstractReadResponse.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/AbstractReadResponse.java new file mode 100644 index 0000000000..5088114248 --- /dev/null +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/AbstractReadResponse.java @@ -0,0 +1,58 @@ +package com.linkedin.venice.listener.response; + +import com.linkedin.davinci.listener.response.ReadResponse; +import com.linkedin.venice.compression.CompressionStrategy; +import com.linkedin.venice.listener.response.stats.ReadResponseStatsRecorder; +import io.netty.buffer.ByteBuf; + + +/** + * This class is used to store common fields shared by various read responses. + */ +public abstract class AbstractReadResponse implements ReadResponse { + private CompressionStrategy compressionStrategy = CompressionStrategy.NO_OP; + private boolean isStreamingResponse = false; + private int rcu = 0; + + public void setCompressionStrategy(CompressionStrategy compressionStrategy) { + this.compressionStrategy = compressionStrategy; + } + + public void setStreamingResponse() { + this.isStreamingResponse = true; + } + + public boolean isStreamingResponse() { + return this.isStreamingResponse; + } + + public CompressionStrategy getCompressionStrategy() { + return compressionStrategy; + } + + /** + * Set the read compute unit (RCU) cost for this response's request + * @param rcu + */ + public void setRCU(int rcu) { + this.rcu = rcu; + } + + /** + * Get the read compute unit (RCU) for this response's request + * @return + */ + public int getRCU() { + return this.rcu; + } + + public boolean isFound() { + return true; + } + + public abstract ByteBuf getResponseBody(); + + public abstract int getResponseSchemaIdHeader(); + + public abstract ReadResponseStatsRecorder getStatsRecorder(); +} diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/ComputeResponseWrapper.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/ComputeResponseWrapper.java index 303d02c590..d28b6e9411 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/ComputeResponseWrapper.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/ComputeResponseWrapper.java @@ -1,31 +1,22 @@ package com.linkedin.venice.listener.response; -import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.compute.protocol.response.ComputeResponseRecordV1; +import com.linkedin.venice.listener.response.stats.ComputeResponseStats; import com.linkedin.venice.schema.avro.ReadAvroProtocolDefinition; import com.linkedin.venice.serializer.FastSerializerDeserializerFactory; import com.linkedin.venice.serializer.RecordSerializer; public class ComputeResponseWrapper extends MultiKeyResponseWrapper { - private static final RecordSerializer SERIALIZER = + static final RecordSerializer SERIALIZER = FastSerializerDeserializerFactory.getFastAvroGenericSerializer(ComputeResponseRecordV1.getClassSchema()); public ComputeResponseWrapper(int maxKeyCount) { - super(maxKeyCount); - // The following metrics will get incremented for each record processed in computeResult() - setReadComputeDeserializationLatency(0.0); - setDatabaseLookupLatency(0.0); - setReadComputeSerializationLatency(0.0); - setReadComputeLatency(0.0); - - // Compute responses are never compressed - setCompressionStrategy(CompressionStrategy.NO_OP); + this(maxKeyCount, new ComputeResponseStats()); } - @Override - protected RecordSerializer getResponseSerializer() { - return SERIALIZER; + public ComputeResponseWrapper(int maxKeyCount, ComputeResponseStats responseStats) { + super(maxKeyCount, responseStats, SERIALIZER); } @Override diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/MultiGetResponseWrapper.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/MultiGetResponseWrapper.java index 3331cb850b..1e9db7ab59 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/MultiGetResponseWrapper.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/MultiGetResponseWrapper.java @@ -1,5 +1,6 @@ package com.linkedin.venice.listener.response; +import com.linkedin.venice.listener.response.stats.MultiKeyResponseStats; import com.linkedin.venice.read.protocol.response.MultiGetResponseRecordV1; import com.linkedin.venice.schema.avro.ReadAvroProtocolDefinition; import com.linkedin.venice.serializer.FastSerializerDeserializerFactory; @@ -7,16 +8,15 @@ public class MultiGetResponseWrapper extends MultiKeyResponseWrapper { - private static final RecordSerializer SERIALIZER = + static final RecordSerializer SERIALIZER = FastSerializerDeserializerFactory.getFastAvroGenericSerializer(MultiGetResponseRecordV1.getClassSchema()); public MultiGetResponseWrapper(int maxKeyCount) { - super(maxKeyCount); + this(maxKeyCount, new MultiKeyResponseStats()); } - @Override - protected RecordSerializer getResponseSerializer() { - return SERIALIZER; + public MultiGetResponseWrapper(int maxKeyCount, MultiKeyResponseStats responseStats) { + super(maxKeyCount, responseStats, SERIALIZER); } @Override diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/MultiKeyResponseWrapper.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/MultiKeyResponseWrapper.java index dbbcf077c7..50bde34bb7 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/MultiKeyResponseWrapper.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/MultiKeyResponseWrapper.java @@ -1,43 +1,83 @@ package com.linkedin.venice.listener.response; -import com.linkedin.davinci.listener.response.ReadResponse; +import com.linkedin.davinci.listener.response.ReadResponseStats; +import com.linkedin.venice.listener.response.stats.MultiKeyResponseStats; +import com.linkedin.venice.listener.response.stats.ReadResponseStatsRecorder; import com.linkedin.venice.serializer.RecordSerializer; +import com.linkedin.venice.utils.lazy.Lazy; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import java.util.ArrayList; +import java.util.Collections; import java.util.List; -public abstract class MultiKeyResponseWrapper extends ReadResponse { - protected final List records; +public abstract class MultiKeyResponseWrapper extends AbstractReadResponse { + private final MultiKeyResponseStats responseStats; + private final Lazy responseBody; - public MultiKeyResponseWrapper(int maxKeyCount) { + /** + * Mutable because we want {@link #getResponseBody()} to "freeze" this list. + */ + private List records; + + MultiKeyResponseWrapper(int maxKeyCount, MultiKeyResponseStats responseStats, RecordSerializer recordSerializer) { this.records = new ArrayList<>(maxKeyCount); + this.responseStats = responseStats; + this.responseBody = Lazy.of(() -> { + ByteBuf responseBodyByteBuf = Unpooled.wrappedBuffer(recordSerializer.serializeObjects(records)); + this.responseStats.setRecordCount(this.records.size()); + + /** + * This re-assignment prevents additional records from being added after the response body is generated. + * It can also potentially help GC by allowing these references which are no longer needed to be collected sooner. + */ + this.records = Collections.emptyList(); + return responseBodyByteBuf; + }); } + /** + * @param record to be added into the container. + * @throws IllegalStateException if called after {@link #getResponseBody()} + */ public void addRecord(K record) { - records.add(record); + try { + records.add(record); + } catch (UnsupportedOperationException e) { + // Defensive code, should never happen unless we have a regression. + throw new IllegalStateException( + this.getClass().getSimpleName() + ".addRecord() cannot be called after getResponseBody()."); + } } - protected abstract RecordSerializer getResponseSerializer(); - public abstract int getResponseSchemaIdHeader(); - public int getRecordCount() { - return records.size(); + @Override + public ReadResponseStats getStats() { + return this.responseStats; } @Override - public ByteBuf getResponseBody() { - return Unpooled.wrappedBuffer(getResponseSerializer().serializeObjects(records)); + public ReadResponseStatsRecorder getStatsRecorder() { + return this.responseStats; } /** - * This function needs to be synchronized because during batch gets, there could be several - * threads incrementing this all at once. + * N.B.: This function is backed by a {@link Lazy} property. The first time it is called, the response is recorded and + * will therefore not be regenerated even if the more records are added. To make this behavior unambiguous, it is no + * longer possible to add records after calling this function, as the {@link #records} property then becomes an empty + * list. + * + * @return the serialized response as a {@link ByteBuf} */ @Override - public void incrementMultiChunkLargeValueCount() { - super.incrementMultiChunkLargeValueCount(); + public ByteBuf getResponseBody() { + return responseBody.get(); + } + + @Override + public String toString() { + return this.getClass().getSimpleName() + "(" + this.records.size() + " records)"; } } diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/ParallelMultiKeyResponseWrapper.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/ParallelMultiKeyResponseWrapper.java new file mode 100644 index 0000000000..96a3e283e2 --- /dev/null +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/ParallelMultiKeyResponseWrapper.java @@ -0,0 +1,134 @@ +package com.linkedin.venice.listener.response; + +import com.linkedin.davinci.listener.response.ReadResponseStats; +import com.linkedin.venice.listener.response.stats.ReadResponseStatsRecorder; +import com.linkedin.venice.stats.ServerHttpRequestStats; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import java.util.function.IntFunction; + + +public class ParallelMultiKeyResponseWrapper extends AbstractReadResponse { + private final T[] chunks; + + private ParallelMultiKeyResponseWrapper( + int chunkCount, + int chunkSize, + T[] chunks, + IntFunction multiGetResponseProvider) { + this.chunks = chunks; + for (int i = 0; i < chunkCount; i++) { + this.chunks[i] = multiGetResponseProvider.apply(chunkSize); + } + } + + public static ParallelMultiKeyResponseWrapper multiGet( + int chunkCount, + int chunkSize, + IntFunction responseProvider) { + return new ParallelMultiKeyResponseWrapper<>( + chunkCount, + chunkSize, + new MultiGetResponseWrapper[chunkCount], + responseProvider); + } + + public static ParallelMultiKeyResponseWrapper compute( + int chunkCount, + int chunkSize, + IntFunction responseProvider) { + return new ParallelMultiKeyResponseWrapper<>( + chunkCount, + chunkSize, + new ComputeResponseWrapper[chunkCount], + responseProvider); + } + + public T getChunk(int chunkIndex) { + return this.chunks[chunkIndex]; + } + + /** + * N.B.: Only the individual chunks should be passed into code paths that require this API. If some refactoring causes + * that assumption to change, then we have a regression. + */ + @Override + public ReadResponseStats getStats() { + throw new IllegalStateException(this.getClass().getSimpleName() + " does not support accumulating metrics."); + } + + @Override + public ReadResponseStatsRecorder getStatsRecorder() { + return new CompositeReadResponseStatsRecorder(this.chunks); + } + + @Override + public ByteBuf getResponseBody() { + ByteBuf[] byteBufChunks = new ByteBuf[chunks.length]; + for (int i = 0; i < chunks.length; i++) { + byteBufChunks[i] = this.chunks[i].getResponseBody(); + } + return Unpooled.wrappedBuffer(byteBufChunks); + } + + @Override + public int getResponseSchemaIdHeader() { + return this.chunks[0].getResponseSchemaIdHeader(); + } + + @Override + public String toString() { + return this.getClass().getSimpleName() + "(" + this.chunks.length + " " + this.chunks[0].getClass().getSimpleName() + + " chunks)"; + } + + private static final class CompositeReadResponseStatsRecorder implements ReadResponseStatsRecorder { + /** The aggregated stats of all the chunks (for stats which can be aggregated). */ + private final ReadResponseStatsRecorder mergedStats; + + /** + * This array references the {@link ReadResponseStatsRecorder} of all response chunks beyond the 1st one (for stats + * which cannot be aggregated). + */ + private final ReadResponseStatsRecorder[] statsRecorders; + + CompositeReadResponseStatsRecorder(MultiKeyResponseWrapper[] responseChunks) { + this.mergedStats = responseChunks[0].getStatsRecorder(); + + /** + * This array can be one element shorter than {@param responseChunks} because the first chunk's storage exec sub + * wait time will be recorded as part of the {@link mergedStats}. + */ + this.statsRecorders = new ReadResponseStatsRecorder[responseChunks.length - 1]; + ReadResponseStatsRecorder statsRecorder; + for (int i = 1; i < responseChunks.length; i++) { + statsRecorder = responseChunks[i].getStatsRecorder(); + // We merge the stats of all chunks from the 2nd one to the last one into the stats of the 1st chunk. + this.mergedStats.merge(statsRecorder); + + /** + * We also keep a handle to each {@link ReadResponseStatsRecorder} beyond the 1st so that later on we can call + * {@link #recordUnmergedMetrics(ServerHttpRequestStats)} on them. */ + this.statsRecorders[i - 1] = statsRecorder; + } + } + + @Override + public void recordMetrics(ServerHttpRequestStats stats) { + this.mergedStats.recordMetrics(stats); + recordUnmergedMetrics(stats); + } + + @Override + public void recordUnmergedMetrics(ServerHttpRequestStats stats) { + for (int i = 0; i < this.statsRecorders.length; i++) { + this.statsRecorders[i].recordUnmergedMetrics(stats); + } + } + + @Override + public void merge(ReadResponseStatsRecorder other) { + throw new UnsupportedOperationException("merge should never be called on " + this.getClass().getSimpleName()); + } + } +} diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/StorageResponseObject.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/SingleGetResponseWrapper.java similarity index 53% rename from services/venice-server/src/main/java/com/linkedin/venice/listener/response/StorageResponseObject.java rename to services/venice-server/src/main/java/com/linkedin/venice/listener/response/SingleGetResponseWrapper.java index bb006054c1..14e3b65e55 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/StorageResponseObject.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/SingleGetResponseWrapper.java @@ -1,15 +1,18 @@ package com.linkedin.venice.listener.response; -import com.linkedin.davinci.listener.response.ReadResponse; +import com.linkedin.davinci.listener.response.ReadResponseStats; import com.linkedin.davinci.store.record.ValueRecord; +import com.linkedin.venice.listener.response.stats.ReadResponseStatsRecorder; +import com.linkedin.venice.listener.response.stats.SingleGetResponseStats; import io.netty.buffer.ByteBuf; -public class StorageResponseObject extends ReadResponse { +public class SingleGetResponseWrapper extends AbstractReadResponse { // Value record storing both schema id and the real data private ValueRecord valueRecord; + private final SingleGetResponseStats responseStats = new SingleGetResponseStats(); - public StorageResponseObject() { + public SingleGetResponseWrapper() { } public void setValueRecord(ValueRecord valueRecord) { @@ -21,16 +24,18 @@ public ValueRecord getValueRecord() { } @Override - public boolean isFound() { - return this.valueRecord != null; + public ReadResponseStats getStats() { + return this.responseStats; + } + + @Override + public ReadResponseStatsRecorder getStatsRecorder() { + return this.responseStats; } - public int getRecordCount() { - if (isFound()) { - return 1; - } else { - return 0; - } + @Override + public boolean isFound() { + return this.valueRecord != null; } @Override 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 new file mode 100644 index 0000000000..d39ee79c1a --- /dev/null +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/AbstractReadResponseStats.java @@ -0,0 +1,165 @@ +package com.linkedin.venice.listener.response.stats; + +import static com.linkedin.venice.listener.response.stats.ResponseStatsUtil.consumeDoubleAndBooleanIfAbove; +import static com.linkedin.venice.listener.response.stats.ResponseStatsUtil.consumeDoubleIfAbove; +import static com.linkedin.venice.listener.response.stats.ResponseStatsUtil.consumeIntIfAbove; + +import com.linkedin.davinci.listener.response.ReadResponseStats; +import com.linkedin.venice.stats.ServerHttpRequestStats; +import com.linkedin.venice.utils.LatencyUtils; + + +/** + * This abstract class is the container for response stats. The stats can be accumulated via the APIs provided by + * {@link ReadResponseStats}, and then recorded using the API from {@link ReadResponseStatsRecorder}. + * + * The class hierarchy aims to minimize the amount of state required for any given response, based on its type and + * relevant server configs: + * + * - {@link AbstractReadResponseStats} + * +-- {@link SingleGetResponseStats} + * +-- {@link MultiKeyResponseStats} + * +-- {@link MultiGetResponseStatsWithSizeProfiling} + * +-- {@link ComputeResponseStats} + * +-- {@link ComputeResponseStatsWithSizeProfiling} + */ +public abstract class AbstractReadResponseStats implements ReadResponseStats, ReadResponseStatsRecorder { + /** + * Package-private on purpose. Only intended for use in tests. + */ + static boolean TEST_ONLY_INJECT_SLEEP_DURING_INCREMENT_TO_SIMULATE_RACE_CONDITION = false; + private static final int UNINITIALIZED = -1; + + private double databaseLookupLatency = 0; + private double storageExecutionSubmissionWaitTime = UNINITIALIZED; + private int storageExecutionQueueLen = UNINITIALIZED; + private int multiChunkLargeValueCount = 0; + + protected abstract int getRecordCount(); + + @Override + public long getCurrentTimeInNanos() { + return System.nanoTime(); + } + + @Override + public void addDatabaseLookupLatency(long startTimeInNanos) { + this.databaseLookupLatency += LatencyUtils.getElapsedTimeFromNSToMS(startTimeInNanos); + } + + @Override + public void setStorageExecutionSubmissionWaitTime(double storageExecutionSubmissionWaitTime) { + this.storageExecutionSubmissionWaitTime = storageExecutionSubmissionWaitTime; + } + + @Override + public void setStorageExecutionQueueLen(int storageExecutionQueueLen) { + this.storageExecutionQueueLen = storageExecutionQueueLen; + } + + public void incrementMultiChunkLargeValueCount() { + int currentValue = multiChunkLargeValueCount; + if (TEST_ONLY_INJECT_SLEEP_DURING_INCREMENT_TO_SIMULATE_RACE_CONDITION) { + /** + * The code below is to reliably trigger a race condition in parallel batch get metrics. Unfortunately, it is not + * easy to cleanly inject this delay, so it is left here as a global variable. The race can still happen without + * the sleep (assuming the stats handling code regressed to a buggy state), but it is less likely. + * + * See: StorageReadRequestHandlerTest.testMultiGetNotUsingKeyBytes + */ + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + multiChunkLargeValueCount = currentValue + 1; + } + + @Override + public void recordMetrics(ServerHttpRequestStats stats) { + consumeDoubleAndBooleanIfAbove( + stats::recordDatabaseLookupLatency, + this.databaseLookupLatency, + isAssembledMultiChunkLargeValue(), + 0); + consumeIntIfAbove(stats::recordMultiChunkLargeValueCount, this.multiChunkLargeValueCount, 0); + consumeIntIfAbove(stats::recordSuccessRequestKeyCount, getRecordCount(), 0); + consumeIntIfAbove(stats::recordStorageExecutionQueueLen, this.storageExecutionQueueLen, UNINITIALIZED); + + recordUnmergedMetrics(stats); + + // Other metrics can be recorded by subclasses + } + + @Override + public void recordUnmergedMetrics(ServerHttpRequestStats stats) { + consumeDoubleIfAbove( + stats::recordStorageExecutionHandlerSubmissionWaitTime, + this.storageExecutionSubmissionWaitTime, + UNINITIALIZED); + } + + @Override + public void merge(ReadResponseStatsRecorder other) { + if (other instanceof AbstractReadResponseStats) { + AbstractReadResponseStats otherStats = (AbstractReadResponseStats) other; + this.databaseLookupLatency += otherStats.databaseLookupLatency; + this.multiChunkLargeValueCount += otherStats.multiChunkLargeValueCount; + } + } + + protected boolean isAssembledMultiChunkLargeValue() { + return this.multiChunkLargeValueCount > 0; + } + + // Below are the read compute functions which are part of the API but should only be overridden and called when + // appropriate + + /** + * This defensive code should never be called. If it is, then some refactoring caused a regression. + */ + private void throwUnsupportedMetric() { + throw new IllegalStateException(this.getClass().getSimpleName() + " does not support compute metrics."); + } + + @Override + public void addReadComputeLatency(double latency) { + throwUnsupportedMetric(); + } + + @Override + public void addReadComputeDeserializationLatency(double latency) { + throwUnsupportedMetric(); + } + + @Override + public void addReadComputeSerializationLatency(double latency) { + throwUnsupportedMetric(); + } + + @Override + public void addReadComputeOutputSize(int size) { + throwUnsupportedMetric(); + } + + @Override + public void incrementDotProductCount(int count) { + throwUnsupportedMetric(); + } + + @Override + public void incrementCountOperatorCount(int count) { + throwUnsupportedMetric(); + } + + @Override + public void incrementCosineSimilarityCount(int count) { + throwUnsupportedMetric(); + } + + @Override + public void incrementHadamardProductCount(int count) { + throwUnsupportedMetric(); + } +} diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ComputeResponseStats.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ComputeResponseStats.java new file mode 100644 index 0000000000..863a5df3aa --- /dev/null +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ComputeResponseStats.java @@ -0,0 +1,110 @@ +package com.linkedin.venice.listener.response.stats; + +import static com.linkedin.venice.listener.response.stats.ResponseStatsUtil.consumeDoubleAndBooleanIfAbove; +import static com.linkedin.venice.listener.response.stats.ResponseStatsUtil.consumeIntIfAbove; + +import com.linkedin.venice.stats.ServerHttpRequestStats; + + +public class ComputeResponseStats extends MultiKeyResponseStats { + private double readComputeLatency = 0; + private double readComputeDeserializationLatency = 0; + private double readComputeSerializationLatency = 0; + private int readComputeOutputSize = 0; + private int totalValueSize = 0; + private int dotProductCount = 0; + private int cosineSimilarityCount = 0; + private int hadamardProductCount = 0; + private int countOperatorCount = 0; + + @Override + public void addValueSize(int size) { + this.totalValueSize += size; + } + + @Override + public void addReadComputeLatency(double latency) { + this.readComputeLatency += latency; + } + + @Override + public void addReadComputeDeserializationLatency(double latency) { + this.readComputeDeserializationLatency += latency; + } + + @Override + public void addReadComputeSerializationLatency(double latency) { + this.readComputeSerializationLatency += latency; + } + + @Override + public void addReadComputeOutputSize(int size) { + this.readComputeOutputSize += size; + } + + @Override + public void incrementDotProductCount(int count) { + this.dotProductCount += count; + } + + @Override + public void incrementCountOperatorCount(int count) { + this.countOperatorCount += count; + } + + @Override + public void incrementCosineSimilarityCount(int count) { + this.cosineSimilarityCount += count; + } + + @Override + public void incrementHadamardProductCount(int count) { + this.hadamardProductCount += count; + } + + @Override + public void recordMetrics(ServerHttpRequestStats stats) { + super.recordMetrics(stats); + + consumeIntIfAbove(stats::recordCosineSimilarityCount, this.cosineSimilarityCount, 0); + consumeIntIfAbove(stats::recordCountOperator, this.countOperatorCount, 0); + consumeIntIfAbove(stats::recordDotProductCount, this.dotProductCount, 0); + consumeIntIfAbove(stats::recordHadamardProduct, this.hadamardProductCount, 0); + boolean isAssembledMultiChunkLargeValue = isAssembledMultiChunkLargeValue(); + consumeDoubleAndBooleanIfAbove( + stats::recordReadComputeDeserializationLatency, + this.readComputeDeserializationLatency, + isAssembledMultiChunkLargeValue, + 0); + consumeDoubleAndBooleanIfAbove( + stats::recordReadComputeLatency, + this.readComputeLatency, + isAssembledMultiChunkLargeValue, + 0); + consumeDoubleAndBooleanIfAbove( + stats::recordReadComputeSerializationLatency, + this.readComputeSerializationLatency, + isAssembledMultiChunkLargeValue, + 0); + if (this.readComputeOutputSize > 0) { + stats.recordReadComputeEfficiency((double) this.totalValueSize / readComputeOutputSize); + } + } + + @Override + public void merge(ReadResponseStatsRecorder other) { + super.merge(other); + if (other instanceof ComputeResponseStats) { + ComputeResponseStats otherStats = (ComputeResponseStats) other; + this.readComputeLatency += otherStats.readComputeLatency; + this.readComputeDeserializationLatency += otherStats.readComputeDeserializationLatency; + this.readComputeSerializationLatency += otherStats.readComputeSerializationLatency; + this.readComputeOutputSize += otherStats.readComputeOutputSize; + this.totalValueSize += otherStats.totalValueSize; + this.dotProductCount += otherStats.dotProductCount; + this.cosineSimilarityCount += otherStats.cosineSimilarityCount; + this.hadamardProductCount += otherStats.hadamardProductCount; + this.countOperatorCount += otherStats.countOperatorCount; + } + } +} diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ComputeResponseStatsWithSizeProfiling.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ComputeResponseStatsWithSizeProfiling.java new file mode 100644 index 0000000000..8e4103b152 --- /dev/null +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ComputeResponseStatsWithSizeProfiling.java @@ -0,0 +1,41 @@ +package com.linkedin.venice.listener.response.stats; + +import com.linkedin.venice.stats.ServerHttpRequestStats; +import it.unimi.dsi.fastutil.ints.IntArrayList; +import it.unimi.dsi.fastutil.ints.IntList; + + +public class ComputeResponseStatsWithSizeProfiling extends ComputeResponseStats { + private final IntList keySizes; + private final IntList valueSizes; + + public ComputeResponseStatsWithSizeProfiling(int maxKeyCount) { + this.keySizes = new IntArrayList(maxKeyCount); + this.valueSizes = new IntArrayList(maxKeyCount); + } + + @Override + public void addKeySize(int size) { + this.keySizes.add(size); + } + + @Override + public void addValueSize(int size) { + /** N.B.: {@link ComputeResponseStats} does have some logic to execute, so it is necessary to call the super. */ + super.addValueSize(size); + this.valueSizes.add(size); + } + + /** + * N.B.: We prefer treating the K/V sizes as non-mergeable, even though we could technically merge these lists into a + * bigger list, because doing so would trigger list resizes and copying, which is less efficient. Furthermore, there + * is no benefit from the merging since we still need to do one record call per item. + * + * @param stats the {@link ServerHttpRequestStats} object to record stats into. + */ + @Override + public void recordUnmergedMetrics(ServerHttpRequestStats stats) { + super.recordUnmergedMetrics(stats); + ResponseStatsUtil.recordKeyValueSizes(stats, this.keySizes, this.valueSizes); + } +} diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/MultiGetResponseStatsWithSizeProfiling.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/MultiGetResponseStatsWithSizeProfiling.java new file mode 100644 index 0000000000..bf43c4e6d8 --- /dev/null +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/MultiGetResponseStatsWithSizeProfiling.java @@ -0,0 +1,39 @@ +package com.linkedin.venice.listener.response.stats; + +import com.linkedin.venice.stats.ServerHttpRequestStats; +import it.unimi.dsi.fastutil.ints.IntArrayList; +import it.unimi.dsi.fastutil.ints.IntList; + + +public class MultiGetResponseStatsWithSizeProfiling extends MultiKeyResponseStats { + private final IntList keySizes; + private final IntList valueSizes; + + public MultiGetResponseStatsWithSizeProfiling(int maxKeyCount) { + this.keySizes = new IntArrayList(maxKeyCount); + this.valueSizes = new IntArrayList(maxKeyCount); + } + + @Override + public void addKeySize(int size) { + this.keySizes.add(size); + } + + @Override + public void addValueSize(int size) { + this.valueSizes.add(size); + } + + /** + * N.B.: We prefer treating the K/V sizes as non-mergeable, even though we could technically merge these lists into a + * bigger list, because doing so would trigger list resizes and copying, which is less efficient. Furthermore, there + * is no benefit from the merging since we still need to do one record call per item. + * + * @param stats the {@link ServerHttpRequestStats} object to record stats into. + */ + @Override + public void recordUnmergedMetrics(ServerHttpRequestStats stats) { + super.recordUnmergedMetrics(stats); + ResponseStatsUtil.recordKeyValueSizes(stats, this.keySizes, this.valueSizes); + } +} diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/MultiKeyResponseStats.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/MultiKeyResponseStats.java new file mode 100644 index 0000000000..534112f78a --- /dev/null +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/MultiKeyResponseStats.java @@ -0,0 +1,31 @@ +package com.linkedin.venice.listener.response.stats; + +public class MultiKeyResponseStats extends AbstractReadResponseStats { + private int recordCount = -1; + + @Override + public void addKeySize(int size) { + } + + @Override + public void addValueSize(int size) { + } + + public void setRecordCount(int count) { + this.recordCount = count; + } + + @Override + protected int getRecordCount() { + return this.recordCount; + } + + @Override + public void merge(ReadResponseStatsRecorder other) { + super.merge(other); + if (other instanceof MultiKeyResponseStats) { + MultiKeyResponseStats otherStats = (MultiKeyResponseStats) other; + this.recordCount += otherStats.recordCount; + } + } +} diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ReadResponseStatsRecorder.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ReadResponseStatsRecorder.java new file mode 100644 index 0000000000..91a104cc42 --- /dev/null +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ReadResponseStatsRecorder.java @@ -0,0 +1,40 @@ +package com.linkedin.venice.listener.response.stats; + +import com.linkedin.venice.stats.ServerHttpRequestStats; + + +/** + * This class is used to record stats associated with a read response. This container may hold two types of stats: + * + * Some stats are "mergeable", which means that we can aggregate multiple instances of {@link ReadResponseStatsRecorder} + * into one via {@link #merge(ReadResponseStatsRecorder)}, and call {@link #recordMetrics(ServerHttpRequestStats)} onto + * the merged instance. Doing so achieves the same outcome as calling the {@link #recordMetrics(ServerHttpRequestStats)} + * API on all individual instances, but from a performance standpoint, it may be preferable to do fewer calls to the + * stats object, since it needs to correctly handle high concurrency. + * + * Some other stats are not mergeable, and must be recorded individually. Those stats are not going to be merged when + * calling {@link #merge(ReadResponseStatsRecorder)}, and the code path which makes use of the merge is responsible for + * calling the {@link #recordUnmergedMetrics(ServerHttpRequestStats)} API on the remaining objects. + */ +public interface ReadResponseStatsRecorder { + /** + * Record all metrics, including those which can and cannot be merged. + * + * @param stats the {@link ServerHttpRequestStats} object to record stats into. + */ + void recordMetrics(ServerHttpRequestStats stats); + + /** + * Record metrics which are not mergeable by the {@link #merge(ReadResponseStatsRecorder)} function. + * + * @param stats the {@link ServerHttpRequestStats} object to record stats into. + */ + void recordUnmergedMetrics(ServerHttpRequestStats stats); + + /** + * Merge the stats contained in this instance with those contained in the {@param other} instance. + * + * @param other instance to merge with. + */ + void merge(ReadResponseStatsRecorder other); +} diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ResponseStatsUtil.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ResponseStatsUtil.java new file mode 100644 index 0000000000..7f2c50810c --- /dev/null +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/ResponseStatsUtil.java @@ -0,0 +1,41 @@ +package com.linkedin.venice.listener.response.stats; + +import com.linkedin.venice.stats.ServerHttpRequestStats; +import com.linkedin.venice.utils.DoubleAndBooleanConsumer; +import it.unimi.dsi.fastutil.ints.IntList; +import java.util.function.DoubleConsumer; +import java.util.function.IntConsumer; + + +public class ResponseStatsUtil { + public static void recordKeyValueSizes(ServerHttpRequestStats stats, IntList keySizes, IntList valueSizes) { + for (int i = 0; i < valueSizes.size(); i++) { + consumeIntIfAbove(stats::recordValueSizeInByte, valueSizes.getInt(i), 0); + } + for (int i = 0; i < keySizes.size(); i++) { + stats.recordKeySizeInByte(keySizes.getInt(i)); + } + } + + public static void consumeIntIfAbove(IntConsumer consumer, int value, int threshold) { + if (value > threshold) { + consumer.accept(value); + } + } + + public static void consumeDoubleIfAbove(DoubleConsumer consumer, double value, double threshold) { + if (value > threshold) { + consumer.accept(value); + } + } + + public static void consumeDoubleAndBooleanIfAbove( + DoubleAndBooleanConsumer consumer, + double value, + boolean b, + double threshold) { + if (value > threshold) { + consumer.accept(value, b); + } + } +} diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/SingleGetResponseStats.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/SingleGetResponseStats.java new file mode 100644 index 0000000000..97dbc6fc15 --- /dev/null +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/response/stats/SingleGetResponseStats.java @@ -0,0 +1,33 @@ +package com.linkedin.venice.listener.response.stats; + +import com.linkedin.venice.stats.ServerHttpRequestStats; + + +public class SingleGetResponseStats extends AbstractReadResponseStats { + private int keySize = 0; + private int valueSize = 0; + + @Override + public void addKeySize(int size) { + this.keySize += size; + } + + @Override + public void addValueSize(int size) { + // N.B.: In the case of single get, this should only called once, so it is effectively a setter. + this.valueSize = size; + } + + @Override + protected int getRecordCount() { + return this.valueSize > 0 ? 1 : 0; + } + + @Override + public void recordMetrics(ServerHttpRequestStats stats) { + super.recordMetrics(stats); + + ResponseStatsUtil.consumeIntIfAbove(stats::recordValueSizeInByte, this.valueSize, 0); + stats.recordKeySizeInByte(this.keySize); + } +} 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 81e3cd80f6..8305d622d6 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 @@ -68,14 +68,6 @@ public void recordErrorRequestLatency(double latency) { totalStats.recordErrorRequestLatency(latency); } - public void recordStorageExecutionHandlerSubmissionWaitTime(double submissionWaitTime) { - totalStats.recordStorageExecutionHandlerSubmissionWaitTime(submissionWaitTime); - } - - public void recordStorageExecutionQueueLen(int len) { - totalStats.recordStorageExecutionQueueLen(len); - } - public void recordMisroutedStoreVersionRequest() { totalStats.recordMisroutedStoreVersionRequest(); } 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 1da08ae563..89fb4d32b3 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 @@ -295,7 +295,8 @@ public ServerHttpRequestStats( () -> totalStats.earlyTerminatedEarlyRequestCountSensor, new OccurrenceRate()); - if (isKeyValueProfilingEnabled) { + if (isKeyValueProfilingEnabled || requestType == RequestType.SINGLE_GET) { + // size profiling is only expensive for requests with lots of keys, but we keep it always on for single gets... String requestValueSizeSensorName = "request_value_size"; requestValueSizeSensor = registerPerStoreAndTotal( requestValueSizeSensorName, @@ -443,11 +444,11 @@ public void recordEarlyTerminatedEarlyRequest() { earlyTerminatedEarlyRequestCountSensor.record(); } - public void recordKeySizeInByte(long keySize) { + public void recordKeySizeInByte(int keySize) { requestKeySizeSensor.record(keySize); } - public void recordValueSizeInByte(long valueSize) { + public void recordValueSizeInByte(int valueSize) { requestValueSizeSensor.record(valueSize); } diff --git a/services/venice-server/src/test/java/com/linkedin/venice/grpc/ServerStatsContextTest.java b/services/venice-server/src/test/java/com/linkedin/venice/grpc/ServerStatsContextTest.java index de2672252b..0c1d0922e2 100644 --- a/services/venice-server/src/test/java/com/linkedin/venice/grpc/ServerStatsContextTest.java +++ b/services/venice-server/src/test/java/com/linkedin/venice/grpc/ServerStatsContextTest.java @@ -7,6 +7,7 @@ import com.linkedin.venice.listener.ServerStatsContext; import com.linkedin.venice.listener.request.RouterRequest; +import com.linkedin.venice.listener.response.stats.ComputeResponseStats; import com.linkedin.venice.read.RequestType; import com.linkedin.venice.stats.AggServerHttpRequestStats; import com.linkedin.venice.stats.ServerHttpRequestStats; @@ -114,28 +115,32 @@ public void testRecordBasicMetrics() { ServerStatsContext context = new ServerStatsContext(singleGetStats, multiGetStats, computeStats); ServerHttpRequestStats stats = mock(ServerHttpRequestStats.class); context.setStoreName("testStore"); + context.setRequestType(RequestType.COMPUTE); - context.setRequestType(RequestType.MULTI_GET); - context.setDatabaseLookupLatency(10.5); - context.setStorageExecutionHandlerSubmissionWaitTime(20.5); - context.setMultiChunkLargeValueCount(10); context.setRequestKeyCount(105); - context.setSuccessRequestKeyCount(100); context.setRequestSize(1000); - context.setRequestPartCount(11); - context.setReadComputeLatency(1000); - context.setReadComputeDeserializationLatency(100); - context.setReadComputeSerializationLatency(200); - context.setDotProductCount(300); - context.setCosineSimilarityCount(13); - context.setHadamardProductCount(132); - context.setCountOperatorCount(432); + context.incrementRequestPartCount(); + context.incrementRequestPartCount(); + + ComputeResponseStats responseStats = new ComputeResponseStats(); + responseStats.setRecordCount(100); + responseStats.addDatabaseLookupLatency(10); + responseStats.setStorageExecutionSubmissionWaitTime(20.5); + responseStats.incrementMultiChunkLargeValueCount(); + responseStats.addReadComputeLatency(1000); + responseStats.addReadComputeDeserializationLatency(100); + responseStats.addReadComputeSerializationLatency(200); + responseStats.incrementDotProductCount(300); + responseStats.incrementCosineSimilarityCount(13); + responseStats.incrementHadamardProductCount(132); + responseStats.incrementCountOperatorCount(432); + context.setReadResponseStats(responseStats); context.recordBasicMetrics(stats); - // verify that 13 interactions are recorded with the stats object, only 13 record metrics to the stats object + // verify that 14 interactions are recorded with the stats object, only 14 record metrics to the stats object MockingDetails details = org.mockito.Mockito.mockingDetails(stats); int invocations = details.getInvocations().size(); - assertEquals(invocations, 13); + assertEquals(invocations, 14); } } 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 fd9858384d..73baa88328 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 @@ -12,8 +12,10 @@ import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.timeout; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -45,6 +47,8 @@ import com.linkedin.venice.compute.protocol.response.ComputeResponseRecordV1; import com.linkedin.venice.exceptions.PersistenceFailureException; import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.guid.JavaUtilGuidV4Generator; +import com.linkedin.venice.kafka.protocol.GUID; import com.linkedin.venice.listener.grpc.GrpcRequestContext; import com.linkedin.venice.listener.grpc.handlers.GrpcStorageReadRequestHandler; import com.linkedin.venice.listener.grpc.handlers.VeniceServerGrpcHandler; @@ -56,10 +60,11 @@ import com.linkedin.venice.listener.request.MultiGetRouterRequestWrapper; import com.linkedin.venice.listener.request.RouterRequest; import com.linkedin.venice.listener.request.TopicPartitionIngestionContextRequest; +import com.linkedin.venice.listener.response.AbstractReadResponse; 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.StorageResponseObject; +import com.linkedin.venice.listener.response.SingleGetResponseWrapper; +import com.linkedin.venice.listener.response.stats.ReadResponseLatencyInjector; import com.linkedin.venice.meta.PartitionerConfig; import com.linkedin.venice.meta.PartitionerConfigImpl; import com.linkedin.venice.meta.QueryAction; @@ -82,17 +87,27 @@ import com.linkedin.venice.schema.SchemaEntry; import com.linkedin.venice.schema.SchemaReader; import com.linkedin.venice.schema.avro.ReadAvroProtocolDefinition; +import com.linkedin.venice.serialization.KeyWithChunkingSuffixSerializer; import com.linkedin.venice.serialization.VeniceKafkaSerializer; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; +import com.linkedin.venice.serialization.avro.ChunkedValueManifestSerializer; import com.linkedin.venice.serialization.avro.VeniceAvroKafkaSerializer; import com.linkedin.venice.serializer.AvroSerializer; import com.linkedin.venice.serializer.RecordDeserializer; import com.linkedin.venice.serializer.RecordSerializer; import com.linkedin.venice.serializer.SerializerDeserializerFactory; +import com.linkedin.venice.stats.ServerHttpRequestStats; +import com.linkedin.venice.storage.protocol.ChunkId; +import com.linkedin.venice.storage.protocol.ChunkedKeySuffix; +import com.linkedin.venice.storage.protocol.ChunkedValueManifest; import com.linkedin.venice.streaming.StreamingUtils; import com.linkedin.venice.unit.kafka.SimplePartitioner; import com.linkedin.venice.utils.DataProviderUtils; +import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Utils; +import com.linkedin.venice.utils.ValueSize; +import com.linkedin.venice.utils.concurrent.BlockingQueueType; +import com.linkedin.venice.utils.concurrent.ThreadPoolFactory; import io.netty.buffer.Unpooled; import io.netty.channel.ChannelHandlerContext; import io.netty.handler.codec.http.DefaultFullHttpRequest; @@ -129,6 +144,7 @@ import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -147,6 +163,12 @@ public void execute(Runnable runnable) { private final ChannelHandlerContext context = mock(ChannelHandlerContext.class); private final ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(Object.class); private final ThreadPoolExecutor executor = new InlineExecutor(); + private final int numberOfExecutionThreads = Runtime.getRuntime().availableProcessors(); + private final ThreadPoolExecutor parallelExecutor = ThreadPoolFactory.createThreadPool( + numberOfExecutionThreads, + this.getClass().getSimpleName(), + 4096, + BlockingQueueType.LINKED_BLOCKING_QUEUE); private final Store store = mock(Store.class); private final Version version = mock(Version.class); private final AbstractStorageEngine storageEngine = mock(AbstractStorageEngine.class); @@ -160,6 +182,9 @@ public void execute(Runnable runnable) { private final ReadMetadataRetriever readMetadataRetriever = mock(ReadMetadataRetriever.class); private final VeniceServerConfig serverConfig = mock(VeniceServerConfig.class); private final VenicePartitioner partitioner = new SimplePartitioner(); + private final ChunkedValueManifestSerializer chunkedValueManifestSerializer = + new ChunkedValueManifestSerializer(true); + private final KeyWithChunkingSuffixSerializer keyWithChunkingSuffixSerializer = new KeyWithChunkingSuffixSerializer(); @BeforeMethod public void setUp() { @@ -192,6 +217,31 @@ public void cleanUp() { readMetadataRetriever, serverConfig, context); + ReadResponseLatencyInjector.removeExtraLatency(); + } + + private enum ParallelQueryProcessing { + PARALLEL(true), SEQUENTIAL(false); + + final boolean configValue; + + ParallelQueryProcessing(boolean configValue) { + this.configValue = configValue; + } + } + + @DataProvider(name = "storageReadRequestHandlerParams") + public Object[][] storageReadRequestHandlerParams() { + int smallRecordCount = numberOfExecutionThreads * 10; + int largeRecordCount = numberOfExecutionThreads * 100; + return new Object[][] { { ParallelQueryProcessing.SEQUENTIAL, smallRecordCount, ValueSize.SMALL_VALUE }, + { ParallelQueryProcessing.SEQUENTIAL, smallRecordCount, ValueSize.LARGE_VALUE }, + { ParallelQueryProcessing.SEQUENTIAL, largeRecordCount, ValueSize.SMALL_VALUE }, + { ParallelQueryProcessing.SEQUENTIAL, largeRecordCount, ValueSize.LARGE_VALUE }, + { ParallelQueryProcessing.PARALLEL, smallRecordCount, ValueSize.SMALL_VALUE }, + { ParallelQueryProcessing.PARALLEL, smallRecordCount, ValueSize.LARGE_VALUE }, + { ParallelQueryProcessing.PARALLEL, largeRecordCount, ValueSize.SMALL_VALUE }, + { ParallelQueryProcessing.PARALLEL, largeRecordCount, ValueSize.LARGE_VALUE } }; } private StorageReadRequestHandler createStorageReadRequestHandler() { @@ -202,8 +252,8 @@ private StorageReadRequestHandler createStorageReadRequestHandler( boolean parallelBatchGetEnabled, int parallelBatchGetChunkSize) { return new StorageReadRequestHandler( - executor, - executor, + parallelBatchGetEnabled ? parallelExecutor : executor, + parallelBatchGetEnabled ? parallelExecutor : executor, storageEngineRepository, storeRepository, schemaRepository, @@ -236,8 +286,11 @@ public void storageExecutionHandlerPassesRequestsAndGeneratesResponses() throws StorageReadRequestHandler requestHandler = createStorageReadRequestHandler(); requestHandler.channelRead(context, request); - verify(context, times(1)).writeAndFlush(argumentCaptor.capture()); - StorageResponseObject responseObject = (StorageResponseObject) argumentCaptor.getValue(); + TestUtils.waitForNonDeterministicAssertion( + 1, + TimeUnit.SECONDS, + () -> verify(context, times(1)).writeAndFlush(argumentCaptor.capture())); + SingleGetResponseWrapper responseObject = (SingleGetResponseWrapper) argumentCaptor.getValue(); assertEquals(responseObject.getValueRecord().getDataInBytes(), valueString.getBytes()); assertEquals(responseObject.getValueRecord().getSchemaId(), schemaId); } @@ -254,8 +307,14 @@ public void testDiskHealthCheckService() throws Exception { assertEquals(healthCheckResponse.getStatus(), HttpResponseStatus.OK); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testMultiGetNotUsingKeyBytes(Boolean isParallel) throws Exception { + @Test(dataProvider = "storageReadRequestHandlerParams") + public void testMultiGetNotUsingKeyBytes(ParallelQueryProcessing parallel, int recordCount, ValueSize largeValue) + throws Exception { + + ReadResponseLatencyInjector.injectExtraLatency(); + doReturn(largeValue.config).when(version).isChunkingEnabled(); + doReturn(largeValue.config).when(storageEngine).isChunked(); + int schemaId = 1; // [0]""/[1]"storage"/[2]{$resourceName} @@ -270,7 +329,9 @@ public void testMultiGetNotUsingKeyBytes(Boolean isParallel) throws Exception { String valuePrefix = "value_"; Map allValueStrings = new HashMap<>(); - int recordCount = 10; + int chunkSize = recordCount / numberOfExecutionThreads; + GUID guid = new JavaUtilGuidV4Generator().getGuid(); + int sequenceNumber = 0; // Prepare multiGet records belong to specific sub-partitions, if the router does not have the right logic to figure // out @@ -278,13 +339,42 @@ public void testMultiGetNotUsingKeyBytes(Boolean isParallel) throws Exception { // considers the first byte of the key. for (int i = 0; i < recordCount; ++i) { MultiGetRouterRequestKeyV1 requestKey = new MultiGetRouterRequestKeyV1(); - byte[] keyBytes = keySerializer.serialize(null, keyPrefix + i); + String keyString = keyPrefix + i; + byte[] keyBytes = keySerializer.serialize(null, keyString); requestKey.keyBytes = ByteBuffer.wrap(keyBytes); requestKey.keyIndex = i; requestKey.partitionId = 0; String valueString = valuePrefix + i; - byte[] valueBytes = ValueRecord.create(schemaId, valueString.getBytes()).serialize(); - doReturn(valueBytes).when(storageEngine).get(0, ByteBuffer.wrap(keyBytes)); + byte[] valueBytes; + if (largeValue.config) { + byte[] chunk1 = ValueRecord + .create(AvroProtocolDefinition.CHUNK.getCurrentProtocolVersion(), valueString.substring(0, 3).getBytes()) + .serialize(); + byte[] chunk2 = ValueRecord + .create(AvroProtocolDefinition.CHUNK.getCurrentProtocolVersion(), valueString.substring(3).getBytes()) + .serialize(); + List keysWithChunkingSuffix = new ArrayList<>(2); + ByteBuffer chunk1KeyBytes = keyWithChunkingSuffixSerializer + .serializeChunkedKey(keyBytes, new ChunkedKeySuffix(new ChunkId(guid, 0, sequenceNumber++, 0), true)); + ByteBuffer chunk2KeyBytes = keyWithChunkingSuffixSerializer + .serializeChunkedKey(keyBytes, new ChunkedKeySuffix(new ChunkId(guid, 0, sequenceNumber++, 1), true)); + keysWithChunkingSuffix.add(chunk1KeyBytes); + keysWithChunkingSuffix.add(chunk2KeyBytes); + doReturn(chunk1).when(storageEngine).get(0, chunk1KeyBytes); + doReturn(chunk2).when(storageEngine).get(0, chunk2KeyBytes); + ChunkedValueManifest chunkedValueManifest = + new ChunkedValueManifest(keysWithChunkingSuffix, schemaId, valueString.length()); + valueBytes = chunkedValueManifestSerializer.serialize("", chunkedValueManifest); + } else { + valueBytes = valueString.getBytes(); + } + byte[] valueRecordContainerBytes = ValueRecord.create( + largeValue.config ? AvroProtocolDefinition.CHUNKED_VALUE_MANIFEST.getCurrentProtocolVersion() : schemaId, + valueBytes).serialize(); + if (largeValue.config) { + keyBytes = keyWithChunkingSuffixSerializer.serializeNonChunkedKey(keyBytes); + } + doReturn(valueRecordContainerBytes).when(storageEngine).get(0, ByteBuffer.wrap(keyBytes)); allValueStrings.put(i, valueString); keys.add(requestKey); } @@ -300,15 +390,21 @@ public void testMultiGetNotUsingKeyBytes(Boolean isParallel) throws Exception { MultiGetRouterRequestWrapper request = MultiGetRouterRequestWrapper .parseMultiGetHttpRequest(httpRequest, RequestHelper.getRequestParts(URI.create(httpRequest.uri()))); - StorageReadRequestHandler requestHandler = createStorageReadRequestHandler(isParallel, 10); + StorageReadRequestHandler requestHandler = createStorageReadRequestHandler(parallel.configValue, chunkSize); + long startTime = System.currentTimeMillis(); requestHandler.channelRead(context, request); + verify(context, timeout(5000)).writeAndFlush(argumentCaptor.capture()); + long timeSpent = System.currentTimeMillis() - startTime; + System.out.println("Time spent: " + timeSpent + " ms for " + recordCount + " records."); - verify(context, times(1)).writeAndFlush(argumentCaptor.capture()); - MultiGetResponseWrapper multiGetResponseWrapper = (MultiGetResponseWrapper) argumentCaptor.getValue(); + Object response = argumentCaptor.getValue(); + assertTrue(response instanceof AbstractReadResponse, "The response should be castable to AbstractReadResponse."); + AbstractReadResponse multiGetResponseWrapper = (AbstractReadResponse) response; RecordDeserializer deserializer = SerializerDeserializerFactory.getAvroSpecificDeserializer(MultiGetResponseRecordV1.class); - Iterable values = - deserializer.deserializeObjects(multiGetResponseWrapper.getResponseBody().array()); + byte[] responseBytes = new byte[multiGetResponseWrapper.getResponseBody().readableBytes()]; + multiGetResponseWrapper.getResponseBody().getBytes(0, responseBytes); + Iterable values = deserializer.deserializeObjects(responseBytes); Map results = new HashMap<>(); values.forEach(K -> { String valueString = new String(K.value.array(), StandardCharsets.UTF_8); @@ -318,6 +414,23 @@ public void testMultiGetNotUsingKeyBytes(Boolean isParallel) throws Exception { for (int i = 0; i < recordCount; i++) { assertEquals(results.get(i), allValueStrings.get(i)); } + + ServerHttpRequestStats stats = mock(ServerHttpRequestStats.class); + multiGetResponseWrapper.getStatsRecorder().recordMetrics(stats); + verify(stats).recordSuccessRequestKeyCount(recordCount); + if (largeValue.config) { + /** + * The assertion below can catch an issue where metrics are inaccurate during parallel batch gets. This was due to + * a race condition which is now fixed. Unfortunately, the race is hard to hit, even on the buggy code, so the test + * succeeding is not a guarantee of the absence of a regression (but the test failing certainly indicates that). + * + * We are making the test fail reliably (in the presence of buggy stats handling) via a backdoor available through + * {@link ReadResponseLatencyInjector}. + */ + verify(stats).recordMultiChunkLargeValueCount(recordCount); + } else { + verify(stats, never()).recordMultiChunkLargeValueCount(anyInt()); + } } @Test @@ -357,10 +470,14 @@ public void storageExecutionHandlerLogsExceptions() throws Exception { StorageReadRequestHandler requestHandler = createStorageReadRequestHandler(); requestHandler.channelRead(context, request); + TestUtils.waitForNonDeterministicAssertion( + 1, + TimeUnit.SECONDS, + () -> verify(context, times(1)).writeAndFlush(argumentCaptor.capture())); verify(context, times(1)).writeAndFlush(argumentCaptor.capture()); HttpShortcutResponse shortcutResponse = (HttpShortcutResponse) argumentCaptor.getValue(); assertEquals(shortcutResponse.getStatus(), HttpResponseStatus.INTERNAL_SERVER_ERROR); - assertEquals(shortcutResponse.getMessage(), exceptionMessage); + assertTrue(shortcutResponse.getMessage().contains(exceptionMessage)); // Asserting that the exception got logged Assert.assertTrue(errorLogCount.get() > 0); @@ -537,6 +654,7 @@ public void testHandleComputeRequest(boolean readComputationEnabled) throws Exce ComputeRouterRequestKeyV1 missingKey = new ComputeRouterRequestKeyV1(1, ByteBuffer.wrap(missingKeyString.getBytes()), partition); doReturn(Arrays.asList(key, missingKey)).when(request).getKeys(); + doReturn(2).when(request).getKeyCount(); StorageReadRequestHandler requestHandler = createStorageReadRequestHandler(); requestHandler.channelRead(context, request); @@ -548,17 +666,8 @@ public void testHandleComputeRequest(boolean readComputationEnabled) throws Exce } else { ComputeResponseWrapper computeResponse = (ComputeResponseWrapper) argumentCaptor.getValue(); assertEquals(computeResponse.isStreamingResponse(), request.isStreamingRequest()); - assertEquals(computeResponse.getRecordCount(), keySet.size()); - assertEquals(computeResponse.getMultiChunkLargeValueCount(), 0); assertEquals(computeResponse.getCompressionStrategy(), CompressionStrategy.NO_OP); - assertEquals(computeResponse.getDotProductCount(), 1); - assertEquals(computeResponse.getHadamardProductCount(), 1); - assertEquals(computeResponse.getCountOperatorCount(), 0); - assertEquals(computeResponse.getCosineSimilarityCount(), 0); - - assertEquals(computeResponse.getValueSize(), valueBytes.length); - int expectedReadComputeOutputSize = 0; RecordDeserializer responseDeserializer = SerializerDeserializerFactory.getAvroSpecificDeserializer(ComputeResponseRecordV1.class); @@ -569,10 +678,21 @@ public void testHandleComputeRequest(boolean readComputationEnabled) throws Exce } else { assertEquals(record.getKeyIndex(), 0); Assert.assertNotEquals(record.getValue(), StreamingUtils.EMPTY_BYTE_BUFFER); - expectedReadComputeOutputSize += record.getValue().limit(); + expectedReadComputeOutputSize += record.getValue().remaining(); } } - assertEquals(computeResponse.getReadComputeOutputSize(), expectedReadComputeOutputSize); + + double expectedReadComputeEfficiency = (double) valueBytes.length / (double) expectedReadComputeOutputSize; + + ServerHttpRequestStats stats = mock(ServerHttpRequestStats.class); + computeResponse.getStatsRecorder().recordMetrics(stats); + verify(stats).recordSuccessRequestKeyCount(keySet.size()); + verify(stats).recordDotProductCount(1); + verify(stats).recordHadamardProduct(1); + verify(stats).recordReadComputeEfficiency(expectedReadComputeEfficiency); + verify(stats, never()).recordMultiChunkLargeValueCount(anyInt()); + verify(stats, never()).recordCountOperator(anyInt()); + verify(stats, never()).recordCosineSimilarityCount(anyInt()); } } @@ -609,9 +729,12 @@ public void testMissingStorageEngine() throws Exception { /** This first request will prime the "perStoreVersionStateMap" inside {@link StorageReadRequestHandler} */ requestHandler.channelRead(context, request); - verify(storageEngine, times(1)).get(anyInt(), any(ByteBuffer.class)); + TestUtils.waitForNonDeterministicAssertion( + 1, + TimeUnit.SECONDS, + () -> verify(storageEngine, times(1)).get(anyInt(), any(ByteBuffer.class))); verify(context, times(1)).writeAndFlush(argumentCaptor.capture()); - StorageResponseObject responseObject = (StorageResponseObject) argumentCaptor.getValue(); + SingleGetResponseWrapper responseObject = (SingleGetResponseWrapper) argumentCaptor.getValue(); assertTrue(responseObject.isFound()); assertEquals(responseObject.getValueRecord().getDataInBytes(), valueString.getBytes()); @@ -625,9 +748,12 @@ public void testMissingStorageEngine() throws Exception { // Second request should not use the stale storage engine requestHandler.channelRead(context, request); verify(storageEngine, times(1)).get(anyInt(), any(ByteBuffer.class)); // No extra invocation - verify(storageEngine2, times(1)).get(anyInt(), any(ByteBuffer.class)); // Good one + TestUtils.waitForNonDeterministicAssertion( + 1, + TimeUnit.SECONDS, + () -> verify(storageEngine2, times(1)).get(anyInt(), any(ByteBuffer.class))); // Good one verify(context, times(2)).writeAndFlush(argumentCaptor.capture()); - responseObject = (StorageResponseObject) argumentCaptor.getValue(); + responseObject = (SingleGetResponseWrapper) argumentCaptor.getValue(); assertTrue(responseObject.isFound()); assertEquals(responseObject.getValueRecord().getDataInBytes(), valueString.getBytes()); } @@ -665,7 +791,10 @@ public void testMisRoutedStoreVersion() throws Exception { requestHandler.channelRead(context, request); ArgumentCaptor shortcutResponseArgumentCaptor = ArgumentCaptor.forClass(HttpShortcutResponse.class); - verify(context).writeAndFlush(shortcutResponseArgumentCaptor.capture()); + TestUtils.waitForNonDeterministicAssertion( + 1, + TimeUnit.SECONDS, + () -> verify(context).writeAndFlush(shortcutResponseArgumentCaptor.capture())); Assert.assertTrue(shortcutResponseArgumentCaptor.getValue().isMisroutedStoreVersion()); } @@ -689,13 +818,19 @@ public void testNoStorageEngineReturn503() throws Exception { requestHandler.channelRead(context, request); ArgumentCaptor shortcutResponseArgumentCaptor = ArgumentCaptor.forClass(HttpShortcutResponse.class); - verify(context).writeAndFlush(shortcutResponseArgumentCaptor.capture()); + TestUtils.waitForNonDeterministicAssertion( + 1, + TimeUnit.SECONDS, + () -> verify(context).writeAndFlush(shortcutResponseArgumentCaptor.capture())); Assert.assertEquals(shortcutResponseArgumentCaptor.getValue().getStatus(), SERVICE_UNAVAILABLE); // when current version different from resource, return 400 doReturn(10).when(store).getCurrentVersion(); requestHandler.channelRead(context, request); - verify(context, times(2)).writeAndFlush(shortcutResponseArgumentCaptor.capture()); + TestUtils.waitForNonDeterministicAssertion( + 1, + TimeUnit.SECONDS, + () -> verify(context, times(2)).writeAndFlush(shortcutResponseArgumentCaptor.capture())); Assert.assertEquals(shortcutResponseArgumentCaptor.getValue().getStatus(), BAD_REQUEST); } diff --git a/services/venice-server/src/test/java/com/linkedin/venice/listener/response/MultiKeyResponseWrapperTest.java b/services/venice-server/src/test/java/com/linkedin/venice/listener/response/MultiKeyResponseWrapperTest.java new file mode 100644 index 0000000000..9ee0ba7717 --- /dev/null +++ b/services/venice-server/src/test/java/com/linkedin/venice/listener/response/MultiKeyResponseWrapperTest.java @@ -0,0 +1,101 @@ +package com.linkedin.venice.listener.response; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; + +import com.linkedin.venice.compute.protocol.response.ComputeResponseRecordV1; +import com.linkedin.venice.listener.response.stats.ComputeResponseStatsWithSizeProfiling; +import com.linkedin.venice.listener.response.stats.MultiGetResponseStatsWithSizeProfiling; +import com.linkedin.venice.read.protocol.response.MultiGetResponseRecordV1; +import io.netty.buffer.ByteBuf; +import java.nio.ByteBuffer; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + + +public class MultiKeyResponseWrapperTest { + private static final int RECORD_COUNT = 10; + private static final int CHUNK_COUNT = 5; + private static final int CHUNK_SIZE = RECORD_COUNT / CHUNK_COUNT; + /** We always use the same value but it does not matter for the sake of this test... */ + private static final ByteBuffer SERIALIZED_VALUE = ByteBuffer.wrap(new byte[] { 0, 1, 2, 3 }); + + /** + * There are 3 dimensions of 2 possibilities each: + * - Batch get and compute + * - Sequential and parallel + * - With and without K/V size profiling + * + * This leads to 2^3 = 8 permutations. + */ + @DataProvider(name = "responseWrapperProvider") + public static Object[][] responseWrapperProvider() { + MultiGetResponseWrapper multiGetResponseWrapper = new MultiGetResponseWrapper(RECORD_COUNT); + ComputeResponseWrapper computeResponseWrapper = new ComputeResponseWrapper(RECORD_COUNT); + MultiGetResponseWrapper multiGetResponseWrapperWithSizeProfiling = + new MultiGetResponseWrapper(RECORD_COUNT, new MultiGetResponseStatsWithSizeProfiling(RECORD_COUNT)); + ComputeResponseWrapper computeResponseWrapperWithSizeProfiling = + new ComputeResponseWrapper(RECORD_COUNT, new ComputeResponseStatsWithSizeProfiling(RECORD_COUNT)); + ParallelMultiKeyResponseWrapper parallelMultiGetResponseWrapper = + ParallelMultiKeyResponseWrapper.multiGet(CHUNK_COUNT, CHUNK_SIZE, s -> new MultiGetResponseWrapper(s)); + ParallelMultiKeyResponseWrapper parallelComputeResponseWrapper = + ParallelMultiKeyResponseWrapper.compute(CHUNK_COUNT, CHUNK_SIZE, s -> new ComputeResponseWrapper(s)); + ParallelMultiKeyResponseWrapper parallelMultiGetResponseWrapperWithSizeProfiling = + ParallelMultiKeyResponseWrapper.multiGet( + CHUNK_COUNT, + CHUNK_SIZE, + s -> new MultiGetResponseWrapper(s, new MultiGetResponseStatsWithSizeProfiling(s))); + ParallelMultiKeyResponseWrapper parallelComputeResponseWrapperWithSizeProfiling = + ParallelMultiKeyResponseWrapper.compute( + CHUNK_COUNT, + CHUNK_SIZE, + s -> new ComputeResponseWrapper(s, new ComputeResponseStatsWithSizeProfiling(s))); + int multiGetSerializedSize = 0; + int computeSerializedSize = 0; + for (int i = 0; i < RECORD_COUNT; i++) { + MultiGetResponseRecordV1 multiGetResponseRecord = new MultiGetResponseRecordV1(i, SERIALIZED_VALUE, 1); + ComputeResponseRecordV1 computeResponseRecord = new ComputeResponseRecordV1(i, SERIALIZED_VALUE); + multiGetSerializedSize += MultiGetResponseWrapper.SERIALIZER.serialize(multiGetResponseRecord).length; + computeSerializedSize += ComputeResponseWrapper.SERIALIZER.serialize(computeResponseRecord).length; + multiGetResponseWrapper.addRecord(multiGetResponseRecord); + computeResponseWrapper.addRecord(computeResponseRecord); + multiGetResponseWrapperWithSizeProfiling.addRecord(multiGetResponseRecord); + computeResponseWrapperWithSizeProfiling.addRecord(computeResponseRecord); + int chunkIndex = i % CHUNK_COUNT; + parallelMultiGetResponseWrapper.getChunk(chunkIndex).addRecord(multiGetResponseRecord); + parallelComputeResponseWrapper.getChunk(chunkIndex).addRecord(computeResponseRecord); + parallelMultiGetResponseWrapperWithSizeProfiling.getChunk(chunkIndex).addRecord(multiGetResponseRecord); + parallelComputeResponseWrapperWithSizeProfiling.getChunk(chunkIndex).addRecord(computeResponseRecord); + } + + return new Object[][] { + /** {@link MultiGetResponseWrapper} */ + { multiGetResponseWrapper, multiGetSerializedSize }, + /** {@link ComputeResponseWrapper} */ + { computeResponseWrapper, computeSerializedSize }, + /** {@link MultiGetResponseWrapper} which carry {@link MultiGetResponseStatsWithSizeProfiling} */ + { multiGetResponseWrapperWithSizeProfiling, multiGetSerializedSize }, + /** {@link ComputeResponseWrapper} which carry {@link ComputeResponseStatsWithSizeProfiling} */ + { computeResponseWrapperWithSizeProfiling, computeSerializedSize }, + /** {@link ParallelMultiKeyResponseWrapper} containing {@link MultiGetResponseWrapper} chunks */ + { parallelMultiGetResponseWrapper, multiGetSerializedSize }, + /** {@link ParallelMultiKeyResponseWrapper} containing {@link ComputeResponseWrapper} chunks */ + { parallelComputeResponseWrapper, computeSerializedSize }, + /** {@link ParallelMultiKeyResponseWrapper} containing {@link MultiGetResponseWrapper} chunks which carry {@link MultiGetResponseStatsWithSizeProfiling} */ + { parallelMultiGetResponseWrapperWithSizeProfiling, multiGetSerializedSize }, + /** {@link ParallelMultiKeyResponseWrapper} containing {@link ComputeResponseWrapper} chunks which carry {@link ComputeResponseStatsWithSizeProfiling} */ + { parallelComputeResponseWrapperWithSizeProfiling, computeSerializedSize } }; + } + + /** + * N.B.: Depending on which {@link io.netty.buffer.CompositeByteBuf} factory method is used, the + * {@link ByteBuf#readableBytes()} may be incorrect. This unit test ensures we do it right. + */ + @Test(dataProvider = "responseWrapperProvider") + public void testResponseBodySize(AbstractReadResponse responseWrapper, int expectedSerializedSize) { + assertNotNull(responseWrapper); + ByteBuf responseBody = responseWrapper.getResponseBody(); + assertNotNull(responseBody); + assertEquals(responseBody.readableBytes(), expectedSerializedSize); + } +} diff --git a/services/venice-server/src/test/java/com/linkedin/venice/listener/response/stats/ReadResponseLatencyInjector.java b/services/venice-server/src/test/java/com/linkedin/venice/listener/response/stats/ReadResponseLatencyInjector.java new file mode 100644 index 0000000000..a539a0d9a9 --- /dev/null +++ b/services/venice-server/src/test/java/com/linkedin/venice/listener/response/stats/ReadResponseLatencyInjector.java @@ -0,0 +1,12 @@ +package com.linkedin.venice.listener.response.stats; + +public class ReadResponseLatencyInjector { + public static void injectExtraLatency() { + AbstractReadResponseStats.TEST_ONLY_INJECT_SLEEP_DURING_INCREMENT_TO_SIMULATE_RACE_CONDITION = true; + } + + public static void removeExtraLatency() { + AbstractReadResponseStats.TEST_ONLY_INJECT_SLEEP_DURING_INCREMENT_TO_SIMULATE_RACE_CONDITION = false; + } + +} diff --git a/services/venice-server/src/test/java/com/linkedin/venice/listener/response/stats/ResponseStatsUtilTest.java b/services/venice-server/src/test/java/com/linkedin/venice/listener/response/stats/ResponseStatsUtilTest.java new file mode 100644 index 0000000000..fd4a2a13ec --- /dev/null +++ b/services/venice-server/src/test/java/com/linkedin/venice/listener/response/stats/ResponseStatsUtilTest.java @@ -0,0 +1,40 @@ +package com.linkedin.venice.listener.response.stats; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + +import com.linkedin.venice.utils.DoubleAndBooleanConsumer; +import java.util.function.DoubleConsumer; +import java.util.function.IntConsumer; +import org.testng.annotations.Test; + + +public class ResponseStatsUtilTest { + @Test + public void testConsumeFunctions() { + IntConsumer intConsumer = mock(IntConsumer.class); + ResponseStatsUtil.consumeIntIfAbove(intConsumer, 1, 0); + verify(intConsumer).accept(1); + + intConsumer = mock(IntConsumer.class); + ResponseStatsUtil.consumeIntIfAbove(intConsumer, 0, 0); + verify(intConsumer, never()).accept(0); + + DoubleConsumer doubleConsumer = mock(DoubleConsumer.class); + ResponseStatsUtil.consumeDoubleIfAbove(doubleConsumer, 1.0, 0.0); + verify(doubleConsumer).accept(1.0); + + doubleConsumer = mock(DoubleConsumer.class); + ResponseStatsUtil.consumeDoubleIfAbove(doubleConsumer, 0, 0); + verify(doubleConsumer, never()).accept(0); + + DoubleAndBooleanConsumer doubleAndBooleanConsumer = mock(DoubleAndBooleanConsumer.class); + ResponseStatsUtil.consumeDoubleAndBooleanIfAbove(doubleAndBooleanConsumer, 1.0, true, 0.0); + verify(doubleAndBooleanConsumer).accept(1.0, true); + + doubleAndBooleanConsumer = mock(DoubleAndBooleanConsumer.class); + ResponseStatsUtil.consumeDoubleAndBooleanIfAbove(doubleAndBooleanConsumer, 0.0, true, 0.0); + verify(doubleAndBooleanConsumer, never()).accept(1.0, true); + } +} diff --git a/services/venice-server/src/test/java/com/linkedin/venice/stats/AggServerQuotaUsageStatsTest.java b/services/venice-server/src/test/java/com/linkedin/venice/stats/AggServerQuotaUsageStatsTest.java index b5013fc8e3..37bedf9965 100644 --- a/services/venice-server/src/test/java/com/linkedin/venice/stats/AggServerQuotaUsageStatsTest.java +++ b/services/venice-server/src/test/java/com/linkedin/venice/stats/AggServerQuotaUsageStatsTest.java @@ -62,8 +62,8 @@ public void testAggServerQuotaUsageStats() { + metricsRepository.getMetric(readQuotaRejectedQPSString2).value(); double totalRejectedKPS = metricsRepository.getMetric(readQuotaRejectedKPSString).value() + metricsRepository.getMetric(readQuotaRejectedKPSString2).value(); - Assert.assertEquals(metricsRepository.getMetric(totalReadQuotaRejectedQPSString).value(), totalRejectedQPS, 0.01); - Assert.assertEquals(metricsRepository.getMetric(totalReadQuotaRejectedKPSString).value(), totalRejectedKPS, 0.01); + Assert.assertEquals(metricsRepository.getMetric(totalReadQuotaRejectedQPSString).value(), totalRejectedQPS, 0.05); + Assert.assertEquals(metricsRepository.getMetric(totalReadQuotaRejectedKPSString).value(), totalRejectedKPS, 0.05); String readQuotaUsageRatioString = "." + storeName + "--quota_requested_usage_ratio.Gauge"; TokenBucket mockTokenBucket = mock(TokenBucket.class);