mjsax commented on code in PR #13252:
URL: https://github.com/apache/kafka/pull/13252#discussion_r1110240934


##########
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&gt> 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(
+                key,
+                // versioned stores require a timestamp associated with all 
puts, including tombstones/deletes
+                value == null

Review Comment:
   This one is tricky -- not sure if we should allow it, but instead change the 
usage of the store in the DSL to avoid calling this method? Just substituting 
`context.timestamp()` does not sound right to me?
   
   I guess the current idea was to bridge from TKV to VKV -- could we instead 
bridge the other way around, ie, we change the DSL to "assume" the VKV 
interface but if the underlying store is a TKV we translate with an adaptor?



-- 
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

Reply via email to