You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "vcrfxia (via GitHub)" <gi...@apache.org> on 2023/02/15 20:41:33 UTC

[GitHub] [kafka] vcrfxia commented on a diff in pull request #13252: KAFKA-14491: [11/N] Add metered wrapper for versioned stores

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


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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 &lt;Bytes,byte[]&gt;, so we use {@link Serde}s
+ * to convert from &lt;K,ValueAndTimestamp&lt;V&gt&gt; to &lt;Bytes,byte[]&gt;. 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
+                    ? ValueAndTimestamp.makeAllowNullable(null, context.timestamp())
+                    : value
+            );
+        }
+
+        public ValueAndTimestamp<V> get(final K key, final long asOfTimestamp) {
+            Objects.requireNonNull(key, "key cannot be null");
+            try {
+                return maybeMeasureLatency(() -> outerValue(inner.get(keyBytes(key), asOfTimestamp)), time, getSensor);
+            } catch (final ProcessorStateException e) {
+                final String message = String.format(e.getMessage(), key);
+                throw new ProcessorStateException(message, e);
+            }
+        }
+
+        public ValueAndTimestamp<V> delete(final K key, final long timestamp) {
+            Objects.requireNonNull(key, "key cannot be null");
+            try {
+                return maybeMeasureLatency(() -> outerValue(inner.delete(keyBytes(key), timestamp)), time, deleteSensor);
+            } catch (final ProcessorStateException e) {
+                final String message = String.format(e.getMessage(), key);
+                throw new ProcessorStateException(message, e);
+            }
+        }
+
+        @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 = wrapped().query(query, positionBound, config);
+            if (config.isCollectExecutionInfo()) {
+                result.addExecutionInfo(
+                    "Handled in " + getClass() + " in " + (time.nanoseconds() - start) + "ns");
+            }
+            // do not convert return type from inner bytes store to user-friendly interface at this

Review Comment:
   This design choice has implications -- I'm curious to hear reviewer thoughts.
   
   (The comment should more accurately say "do not convert query or return types to/from inner bytes store ...", which I will update in the commit itself.)
   
   The existing MeteredKeyValueStore and MeteredTimestampedKeyValueStore both contain logic to assist in translating `KeyQuery<K, V>` and `RangeQuery<K, V>` types to queries understandable by the underlying bytes store, and to deserialize the returned bytes result back for the original `K` and `V` result types. The return type of these queries for MeteredKeyValueStore and MeteredTimestampedKeyValueStore is `V` for the key query, and `KeyValueIterator<K, V>` for the range query. We'll want to introduce new return types for versioned stores, perhaps `VersionedRecord<V>` for the key query and `Iterator<VersionedRecord<V>>` for the range query. I'd like to postpone this decision-making for a follow-up KIP, but that means not assisting in translating to/from the inner bytes store for now.
   
   So we have three options in the meantime:
   1. Reject all IQv2 queries, since RocksDBVersionedKeyValueStore doesn't support IQv2 yet anyway. I don't think we should do this, because then users who create their own VersionedKeyValueStore implementations won't be able to issue IQv2 queries either.
   2. Allow IQv2 queries as a pass-through at the metered layer. This is what the PR currently does. It means that users who implement their own VersionedKeyValueStores will be able to issue IQv2 queries, but also has backwards compatibility implications for when we introduce type serialization/deserialization support for key and range queries in the future. The behavior at this metered store layer would suddenly change for those query types.
   3. Reject KeyQuery and RangeQuery types in order to reserve them for future behaviors, and pass-through all other query types. This avoids compatibility concerns for the future, while still allowing users to issue IQv2 queries to their own VersionedKeyValueStore implementations in the meantime.
   
   I'm curious to know whether there's existing precedent / discussion on this compatibility concern for introducing custom logic at the metered layer for handling specific types of IQv2 queries.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org