vcrfxia commented on code in PR #13252: URL: https://github.com/apache/kafka/pull/13252#discussion_r1114727896
########## streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java: ########## @@ -0,0 +1,227 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; + +import java.util.Objects; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.internals.SerdeGetter; +import org.apache.kafka.streams.query.Position; +import org.apache.kafka.streams.query.PositionBound; +import org.apache.kafka.streams.query.Query; +import org.apache.kafka.streams.query.QueryConfig; +import org.apache.kafka.streams.query.QueryResult; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.TimestampedKeyValueStore; +import org.apache.kafka.streams.state.ValueAndTimestamp; +import org.apache.kafka.streams.state.VersionedBytesStore; +import org.apache.kafka.streams.state.VersionedKeyValueStore; +import org.apache.kafka.streams.state.VersionedRecord; + +/** + * A metered {@link VersionedKeyValueStore} wrapper that is used for recording operation + * metrics, and hence its inner {@link VersionedBytesStore} implementation does not need to provide + * its own metrics collecting functionality. The inner {@code VersionedBytesStore} of this class + * is a {@link KeyValueStore} of type <Bytes,byte[]>, so we use {@link Serde}s + * to convert from <K,ValueAndTimestamp<V>> to <Bytes,byte[]>. In particular, + * {@link NullableValueAndTimestampSerde} is used since putting a tombstone to a versioned key-value + * store requires putting a null value associated with a timestamp. + * + * @param <K> The key type + * @param <V> The (raw) value type + */ +public class MeteredVersionedKeyValueStore<K, V> + extends WrappedStateStore<VersionedBytesStore, K, V> + implements VersionedKeyValueStore<K, V> { + + private final MeteredVersionedKeyValueStoreInternal internal; + + MeteredVersionedKeyValueStore(final VersionedBytesStore inner, + final String metricScope, + final Time time, + final Serde<K> keySerde, + final Serde<ValueAndTimestamp<V>> valueSerde) { + super(inner); + internal = new MeteredVersionedKeyValueStoreInternal(inner, metricScope, time, keySerde, valueSerde); + } + + /** + * Private helper class which represents the functionality of a {@link VersionedKeyValueStore} + * as a {@link TimestampedKeyValueStore} so that the bulk of the metering logic may be + * inherited from {@link MeteredKeyValueStore}. As a result, the implementation of + * {@link MeteredVersionedKeyValueStore} is a simple wrapper to translate from this + * {@link TimestampedKeyValueStore} representation of a versioned key-value store into the + * {@link VersionedKeyValueStore} interface itself. + */ + private class MeteredVersionedKeyValueStoreInternal + extends MeteredKeyValueStore<K, ValueAndTimestamp<V>> + implements TimestampedKeyValueStore<K, V> { Review Comment: `MeteredVersionedKeyValueStoreInternal` and `MeteredTimestampKeyValueStore` use different serdes -- `MeteredTimestampKeyValueStore` uses `ValueAndTimestampSerde` while `MeteredVersionedKeyValueStoreInternal` uses `NullableValueAndTimestampSerde`. If you look at the code for `MeteredTimestampKeyValueStore`, the only method it overrides from `MeteredKeyValueStore` is `prepareValueSerdeForStore()`. We could have `MeteredVersionedKeyValueStoreInternal` extend `MeteredTimestampKeyValueStore` instead of `MeteredKeyValueStore` if we want but `MeteredVersionedKeyValueStoreInternal` needs to override `prepareValueSerdeForStore()` with its own serde anyway, so `MeteredVersionedKeyValueStoreInternal` would get zero functionality from `MeteredTimestampKeyValueStore`. Also, IMO `MeteredVersionedKeyValueStoreInternal` and `MeteredTimestampKeyValueStore` are different conceptually even though they both have the signature `MeteredKeyValueStore<K, ValueAndTimestamp<V>>`. The reason is because the metered layer (in addition to collecting metrics) is responsible for handling serialization to/from the inner bytes stores (pending your other comment, which we can discuss there), and the serialization behavior of the two stores is different, since they use different serdes. This is a more minor reason, though. If you think it's conceptually easier to think of versioned stores as a special case of timestamped stores, we can have `MeteredVersionedKeyValueStoreInternal extends MeteredTimestampKeyValueStore` instead. -- 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