vcrfxia commented on code in PR #13252: URL: https://github.com/apache/kafka/pull/13252#discussion_r1114772380
########## 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> { + + private final VersionedBytesStore inner; + + MeteredVersionedKeyValueStoreInternal(final VersionedBytesStore inner, + final String metricScope, + final Time time, + final Serde<K> keySerde, + final Serde<ValueAndTimestamp<V>> valueSerde) { + super(inner, metricScope, time, keySerde, valueSerde); + this.inner = inner; + } + + @Override + public void put(final K key, final ValueAndTimestamp<V> value) { + super.put( Review Comment: Let me try to repeat back your suggestion to make sure I understand correctly: as you've pointed out, the current `MeteredVersionedKeyValueStore` implementation first serializes value and timestamp to bytes and concatenates them into a special format (given by `NullableValueAndTimestampSerde`) to pass to inner stores. Then the inner stores have to separate the value and timestamp bytes again and deserialize the timestamp, for use in writing to the changelog or inserting into the inner store. You're suggesting that we make this more efficient by not having the metered layer serialize the timestamp and concatenate with the value bytes, and instead pass an unserialized timestamp and separate value bytes to the inner stores. Is that correct? If so, this suggestion is not possible for the innermost store (`RocksDBVersionedStore`, above which the `VersionedKeyValueToBytesStoreAdapter` is used) because of our decision to maintain compatibility with existing DSL methods for passing key-value stores, e.g., [StreamsBuilder#table()](https://github.com/apache/kafka/blob/3012332e3d82947e434933efd4ab4e9366ab429d/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java#L260) and [KTable methods](https://github.com/apache/kafka/blob/3012332e3d82947e434933efd4ab4e9366ab429d/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java#L153), which are explicitly typed to accept `Materialized<K, V, KeyValueStore<Bytes, byte[]>`, rather than adding new versions of these methods which accept `Materialized` instances to materialize stores other than `KeyValueStore<Bytes, byte[]>`. During the design phase we had discussed this tradeoff already and decided that the benefit of not needing to introduce new methods here is s ubstantial enough to warrant the performance hit from the extra round of serialization and deserialization. The changelogging layer is a different story. If we wanted to, we could indeed have `ChangeLoggingVersionedKeyValueBytesStore` implement `VersionedKeyValueStore<Bytes, byte[]>` instead of `VersionedBytesStore` to avoid needing to extract value and timestamp bytes from the serialized `rawValueAndTimestamp`. The cost of doing so, however, is extra duplicated code. If we do this, then we cannot have `ChangeLoggingVersionedKeyValueBytesStore extends ChangeLoggingKeyValueBytesStore` anymore because `ChangeLoggingKeyValueBytesStore implements KeyValueStore<Bytes, byte[]>` which clashes with `VersionedKeyValueStore<Bytes, byte[]>`. We also wouldn't be able to have the metered versioned store layer extend `MeteredKeyValueStore` anymore either because `MeteredKeyValueStore` is a `WrappedStateStore` with `KeyValueStore<Bytes, byte[]>` inside, and `ChangeLoggingVersionedKeyValueBytesStore` would no longer count as a `KeyValueStore<Bytes, byte[]>`. So, we'd end up more-or-less duplicating the existing code in both `MeteredKeyValueStore` and `ChangeLoggingKeyValueBytesStore` into their versioned counterparts. As a result, I don't think the performance benefit saved from avoiding the extra deserialization in the changelogging layer is worth this extra complexity. FWIW, `ChangeLoggingTimestampedKeyValueBytesStore` also has this same extra deserialization step (separating the concatenated value and timestamp bytes, and deserializing the timestamp for writing to the changelog), I assume for the same reason. -- 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