aliehsaeedii commented on code in PR #21451: URL: https://github.com/apache/kafka/pull/21451#discussion_r2792861602
########## streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreWithHeaders.java: ########## @@ -0,0 +1,100 @@ +/* + * 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 org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.processor.internals.SerdeGetter; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.TimestampedKeyValueStoreWithHeaders; +import org.apache.kafka.streams.state.ValueTimestampHeaders; + +import java.util.Objects; + +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; + +/** + * A Metered {@link TimestampedKeyValueStoreWithHeaders} wrapper that is used for recording operation metrics, and hence + * its inner KeyValueStore implementation does not need to provide its own metrics collecting functionality. + * + * The inner {@link KeyValueStore} of this class is of type <Bytes, byte[]>, + * hence we use {@link Serde}s to convert from <K, ValueTimestampHeaders<V>> to <Bytes, byte[]>. + * + * @param <K> key type + * @param <V> value type (wrapped in {@link ValueTimestampHeaders}) + */ +public class MeteredTimestampedKeyValueStoreWithHeaders<K, V> + extends MeteredKeyValueStore<K, ValueTimestampHeaders<V>> + implements TimestampedKeyValueStoreWithHeaders<K, V> { + + MeteredTimestampedKeyValueStoreWithHeaders(final KeyValueStore<Bytes, byte[]> inner, + final String metricScope, + final Time time, + final Serde<K> keySerde, + final Serde<ValueTimestampHeaders<V>> valueSerde) { + super(inner, metricScope, time, keySerde, valueSerde); + } + + @SuppressWarnings("unchecked") + @Override + protected Serde<ValueTimestampHeaders<V>> prepareValueSerdeForStore(final Serde<ValueTimestampHeaders<V>> valueSerde, + final SerdeGetter getter) { + if (valueSerde == null) { + return new ValueTimestampHeadersSerde<>((Serde<V>) getter.valueSerde()); + } else { + return super.prepareValueSerdeForStore(valueSerde, getter); + } + } + Review Comment: the `get` method is not exactly the same as the `get` method of `MeteredKeyValueStore`. Please see the implementation of the `outerValue`. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
