hanyuzheng7 commented on code in PR #14570: URL: https://github.com/apache/kafka/pull/14570#discussion_r1396800666
########## streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java: ########## @@ -102,4 +139,216 @@ static class RawAndDeserializedValue<ValueType> { this.value = value; } } + + @SuppressWarnings("unchecked") + @Override + public <R> QueryResult<R> query(final Query<R> query, + final PositionBound positionBound, + final QueryConfig config) { + + final long start = time.nanoseconds(); + final QueryResult<R> result; + + final StoreQueryUtils.QueryHandler handler = queryHandlers.get(query.getClass()); + if (handler == null) { + result = wrapped().query(query, positionBound, config); + if (config.isCollectExecutionInfo()) { + result.addExecutionInfo( + "Handled in " + getClass() + " in " + (time.nanoseconds() - start) + "ns"); + } + } else { + result = (QueryResult<R>) handler.apply( + query, + positionBound, + config, + this + ); + if (config.isCollectExecutionInfo()) { + result.addExecutionInfo( + "Handled in " + getClass() + " with serdes " + + serdes + " in " + (time.nanoseconds() - start) + "ns"); + } + } + return result; + } + + + + @SuppressWarnings("unchecked") + private <R> QueryResult<R> runTimestampedKeyQuery(final Query<R> query, + final PositionBound positionBound, + final QueryConfig config) { + final QueryResult<R> result; + final TimestampedKeyQuery<K, V> typedKeyQuery = (TimestampedKeyQuery<K, V>) query; + final KeyQuery<Bytes, byte[]> rawKeyQuery = + KeyQuery.withKey(keyBytes(typedKeyQuery.key())); + final QueryResult<byte[]> rawResult = + wrapped().query(rawKeyQuery, positionBound, config); + if (rawResult.isSuccess()) { + final Function<byte[], ValueAndTimestamp<V>> deserializer = getDeserializeValue(serdes, wrapped()); + final ValueAndTimestamp<V> valueAndTimestamp = deserializer.apply(rawResult.getResult()); + final QueryResult<ValueAndTimestamp<V>> typedQueryResult = + InternalQueryResultUtil.copyAndSubstituteDeserializedResult(rawResult, valueAndTimestamp); + result = (QueryResult<R>) typedQueryResult; + } else { + // the generic type doesn't matter, since failed queries have no result set. + result = (QueryResult<R>) rawResult; + } + return result; + } + + @SuppressWarnings("unchecked") + private <R> QueryResult<R> runTimestampedRangeQuery(final Query<R> query, + final PositionBound positionBound, + final QueryConfig config) { + + final QueryResult<R> result; + final TimestampedRangeQuery<K, V> typedQuery = (TimestampedRangeQuery<K, V>) query; + RangeQuery<Bytes, byte[]> rawRangeQuery; + final boolean isKeyAscending = typedQuery.isKeyAscending(); + rawRangeQuery = RangeQuery.withRange( + keyBytes(typedQuery.lowerBound().orElse(null)), + keyBytes(typedQuery.upperBound().orElse(null)) + ); + if (!isKeyAscending) { + rawRangeQuery = rawRangeQuery.withDescendingKeys(); + } + final QueryResult<KeyValueIterator<Bytes, byte[]>> rawResult = + wrapped().query(rawRangeQuery, positionBound, config); + if (rawResult.isSuccess()) { + final KeyValueIterator<Bytes, byte[]> iterator = rawResult.getResult(); + final KeyValueIterator<K, ValueAndTimestamp<V>> resultIterator = (KeyValueIterator<K, ValueAndTimestamp<V>>) new MeteredTimestampedKeyValueStoreIterator( + iterator, + getSensor, + getDeserializeValue(serdes, wrapped()), + false + ); + final QueryResult<KeyValueIterator<K, ValueAndTimestamp<V>>> typedQueryResult = + InternalQueryResultUtil.copyAndSubstituteDeserializedResult( + rawResult, + resultIterator + ); + result = (QueryResult<R>) typedQueryResult; + } else { + // the generic type doesn't matter, since failed queries have no result set. + result = (QueryResult<R>) rawResult; + } + return result; + } + + @SuppressWarnings("unchecked") + private <R> QueryResult<R> runKeyQuery(final Query<R> query, + final PositionBound positionBound, + final QueryConfig config) { + final QueryResult<R> result; + final KeyQuery<K, V> typedKeyQuery = (KeyQuery<K, V>) query; + final KeyQuery<Bytes, byte[]> rawKeyQuery = + KeyQuery.withKey(keyBytes(typedKeyQuery.getKey())); + final QueryResult<byte[]> rawResult = + wrapped().query(rawKeyQuery, positionBound, config); + if (rawResult.isSuccess()) { + final Function<byte[], ValueAndTimestamp<V>> deserializer = getDeserializeValue(serdes, wrapped()); + final ValueAndTimestamp<V> valueAndTimestamp = deserializer.apply(rawResult.getResult()); + final V plainValue = valueAndTimestamp == null ? null : valueAndTimestamp.value(); + final QueryResult<V> typedQueryResult = + InternalQueryResultUtil.copyAndSubstituteDeserializedResult(rawResult, plainValue); + result = (QueryResult<R>) typedQueryResult; + } else { + // the generic type doesn't matter, since failed queries have no result set. + result = (QueryResult<R>) rawResult; + } + return result; + } + + @SuppressWarnings("unchecked") + private <R> QueryResult<R> runRangeQuery(final Query<R> query, + final PositionBound positionBound, + final QueryConfig config) { + + final QueryResult<R> result; + final RangeQuery<K, V> typedQuery = (RangeQuery<K, V>) query; + RangeQuery<Bytes, byte[]> rawRangeQuery; + final boolean isKeyAscending = typedQuery.isKeyAscending(); + rawRangeQuery = RangeQuery.withRange( + keyBytes(typedQuery.getLowerBound().orElse(null)), + keyBytes(typedQuery.getUpperBound().orElse(null)) + ); + if (!isKeyAscending) { + rawRangeQuery = rawRangeQuery.withDescendingKeys(); + } + final QueryResult<KeyValueIterator<Bytes, byte[]>> rawResult = + wrapped().query(rawRangeQuery, positionBound, config); + if (rawResult.isSuccess()) { + final KeyValueIterator<Bytes, byte[]> iterator = rawResult.getResult(); + final KeyValueIterator<K, V> resultIterator = new MeteredTimestampedKeyValueStoreIterator( + iterator, + getSensor, + getDeserializeValue(serdes, wrapped()), + true + ); + final QueryResult<KeyValueIterator<K, V>> typedQueryResult = + InternalQueryResultUtil.copyAndSubstituteDeserializedResult( + rawResult, + resultIterator + ); + result = (QueryResult<R>) typedQueryResult; + } else { + // the generic type doesn't matter, since failed queries have no result set. + result = (QueryResult<R>) rawResult; + } + return result; + } + + @SuppressWarnings("unchecked") + private class MeteredTimestampedKeyValueStoreIterator implements KeyValueIterator<K, V> { + + private final KeyValueIterator<Bytes, byte[]> iter; + private final Sensor sensor; + private final long startNs; + private final Function<byte[], ValueAndTimestamp<V>> valueDeserializer; + + private final boolean isKeyOrRangeQuery; + + private MeteredTimestampedKeyValueStoreIterator(final KeyValueIterator<Bytes, byte[]> iter, + final Sensor sensor, + final Function<byte[], ValueAndTimestamp<V>> valueDeserializer, + final boolean isKeyOrRangeQuery) { Review Comment: fixed it. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org