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 02:34:05 UTC

[GitHub] [kafka] vcrfxia opened a new pull request, #13250: KAFKA-14491: [9/N] Add versioned bytes store and supplier

vcrfxia opened a new pull request, #13250:
URL: https://github.com/apache/kafka/pull/13250

   (This PR is stacked on https://github.com/apache/kafka/pull/13249. Only the last commit needs to be reviewed separately.)
   
   As part of introducing versioned key-value stores in [KIP-889](https://cwiki.apache.org/confluence/display/KAFKA/KIP-889%3A+Versioned+State+Stores), we'd like a way to represent a versioned key-value store (`VersionedKeyValueStore<Bytes, byte[]>`) as a regular key-value store (`KeyValueStore<Bytes, byte[]>`) in order to be compatible 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[]>`. This way, we do not need to introduce new versions of all relevant StreamsBuilder and KTable methods to relax the Materialized type to accept versioned stores. 
   
   This PR introduces the new `VersionedBytesStore extends KeyValueStore<Bytes, byte[]>` interface for this purpose, along with the corresponding supplier (`VersionedBytesStoreSupplier`) and implementation (`RocksDbVersionedKeyValueBytesStoreSupplier`). The `RocksDbVersionedKeyValueBytesStoreSupplier` implementation leverages an adapter (`VersionedKeyValueToBytesStoreAdapter`) to assist in converting from `VersionedKeyValueStore` to `VersionedBytesStore`.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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


[GitHub] [kafka] mjsax commented on a diff in pull request #13250: KAFKA-14491: [9/N] Add versioned bytes store and supplier

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13250:
URL: https://github.com/apache/kafka/pull/13250#discussion_r1109169480


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/VersionedKeyValueToBytesStoreAdapter.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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 java.util.List;
+import java.util.Objects;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes.ByteArraySerde;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.kstream.KTable;
+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.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.KeyValueIterator;
+import org.apache.kafka.streams.state.KeyValueStore;
+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;
+
+/**
+ * Adapts from {@link VersionedKeyValueStore} (user-friendly versioned store interface) to
+ * {@link KeyValueStore}. By representing a {@code VersionedKeyValueStore} as a
+ * {@code KeyValueStore}, this allows reuse of existing {@link StreamsBuilder} and {@link KTable}
+ * method interfaces which accept {@code Materialized<K, V, KeyValueStore<Bytes, byte[]>)}
+ * for versioned key-value stores.
+ */
+public class VersionedKeyValueToBytesStoreAdapter implements VersionedBytesStore {
+    private static final Serde<ValueAndTimestamp<byte[]>> VALUE_AND_TIMESTAMP_SERDE
+        = new NullableValueAndTimestampSerde<>(new ByteArraySerde());
+    private static final Serializer<ValueAndTimestamp<byte[]>> VALUE_AND_TIMESTAMP_SERIALIZER
+        = VALUE_AND_TIMESTAMP_SERDE.serializer();
+    private static final Deserializer<ValueAndTimestamp<byte[]>> VALUE_AND_TIMESTAMP_DESERIALIZER
+        = VALUE_AND_TIMESTAMP_SERDE.deserializer();
+
+    final VersionedKeyValueStore<Bytes, byte[]> inner;
+
+    public VersionedKeyValueToBytesStoreAdapter(final VersionedKeyValueStore<Bytes, byte[]> inner) {
+        this.inner = Objects.requireNonNull(inner);
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] rawValueAndTimestamp) {
+        if (rawValueAndTimestamp == null) {
+            throw new IllegalArgumentException("Put to versioned store must always include timestamp, including for tombstones.");
+        }
+        final ValueAndTimestamp<byte[]> valueAndTimestamp
+            = VALUE_AND_TIMESTAMP_DESERIALIZER.deserialize(null, rawValueAndTimestamp);
+        inner.put(
+            key,
+            valueAndTimestamp.value(),
+            valueAndTimestamp.timestamp()
+        );
+    }
+
+    @Override
+    public byte[] get(final Bytes key) {
+        final VersionedRecord<byte[]> versionedRecord = inner.get(key);
+        return serializeAsBytes(versionedRecord);
+    }
+
+    @Override
+    public byte[] get(final Bytes key, final long asOfTimestamp) {
+        final VersionedRecord<byte[]> versionedRecord = inner.get(key, asOfTimestamp);
+        return serializeAsBytes(versionedRecord);
+    }
+
+    @Override
+    public byte[] delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> versionedRecord = inner.delete(key, timestamp);
+        return serializeAsBytes(versionedRecord);
+    }
+
+    @Override
+    public String name() {
+        return inner.name();
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        inner.init(context, root);
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        inner.init(context, root);
+    }
+
+    @Override
+    public void flush() {
+        inner.flush();
+    }
+
+    @Override
+    public void close() {
+        inner.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return inner.persistent();
+    }
+
+    @Override
+    public boolean isOpen() {
+        return inner.persistent();
+    }
+
+    @Override
+    public <R> QueryResult<R> query(final Query<R> query, final PositionBound positionBound, final QueryConfig config) {
+        return inner.query(query, positionBound, config);
+    }
+
+    @Override
+    public Position getPosition() {
+        return inner.getPosition();
+    }
+
+    @Override
+    public byte[] putIfAbsent(final Bytes key, final byte[] value) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support putIfAbsent(key, value)");
+    }
+
+    @Override
+    public void putAll(final List<KeyValue<Bytes, byte[]>> entries) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support putAll(entries)");
+    }
+
+    @Override
+    public byte[] delete(final Bytes key) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support delete(key)");
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> range(final Bytes from, final Bytes to) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support range(from, to)");
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> reverseRange(final Bytes from, final Bytes to) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support reverseRange(from, to)");
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> all() {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support all()");
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> reverseAll() {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support reverseAll()");
+    }
+
+    @Override
+    public <PS extends Serializer<P>, P> KeyValueIterator<Bytes, byte[]> prefixScan(final P prefix, final PS prefixKeySerializer) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support prefixScan(prefix, prefixKeySerializer)");
+    }
+
+    @Override
+    public long approximateNumEntries() {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support approximateNumEntries()");

Review Comment:
   Just wondering if we could (should?) return the "latest value store" number here?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/VersionedKeyValueToBytesStoreAdapter.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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 java.util.List;
+import java.util.Objects;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes.ByteArraySerde;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.kstream.KTable;
+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.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.KeyValueIterator;
+import org.apache.kafka.streams.state.KeyValueStore;
+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;
+
+/**
+ * Adapts from {@link VersionedKeyValueStore} (user-friendly versioned store interface) to
+ * {@link KeyValueStore}. By representing a {@code VersionedKeyValueStore} as a
+ * {@code KeyValueStore}, this allows reuse of existing {@link StreamsBuilder} and {@link KTable}
+ * method interfaces which accept {@code Materialized<K, V, KeyValueStore<Bytes, byte[]>)}
+ * for versioned key-value stores.
+ */
+public class VersionedKeyValueToBytesStoreAdapter implements VersionedBytesStore {
+    private static final Serde<ValueAndTimestamp<byte[]>> VALUE_AND_TIMESTAMP_SERDE
+        = new NullableValueAndTimestampSerde<>(new ByteArraySerde());
+    private static final Serializer<ValueAndTimestamp<byte[]>> VALUE_AND_TIMESTAMP_SERIALIZER
+        = VALUE_AND_TIMESTAMP_SERDE.serializer();
+    private static final Deserializer<ValueAndTimestamp<byte[]>> VALUE_AND_TIMESTAMP_DESERIALIZER
+        = VALUE_AND_TIMESTAMP_SERDE.deserializer();
+
+    final VersionedKeyValueStore<Bytes, byte[]> inner;
+
+    public VersionedKeyValueToBytesStoreAdapter(final VersionedKeyValueStore<Bytes, byte[]> inner) {
+        this.inner = Objects.requireNonNull(inner);
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] rawValueAndTimestamp) {
+        if (rawValueAndTimestamp == null) {
+            throw new IllegalArgumentException("Put to versioned store must always include timestamp, including for tombstones.");
+        }
+        final ValueAndTimestamp<byte[]> valueAndTimestamp
+            = VALUE_AND_TIMESTAMP_DESERIALIZER.deserialize(null, rawValueAndTimestamp);
+        inner.put(
+            key,
+            valueAndTimestamp.value(),
+            valueAndTimestamp.timestamp()
+        );
+    }
+
+    @Override
+    public byte[] get(final Bytes key) {
+        final VersionedRecord<byte[]> versionedRecord = inner.get(key);
+        return serializeAsBytes(versionedRecord);
+    }
+
+    @Override
+    public byte[] get(final Bytes key, final long asOfTimestamp) {
+        final VersionedRecord<byte[]> versionedRecord = inner.get(key, asOfTimestamp);
+        return serializeAsBytes(versionedRecord);
+    }
+
+    @Override
+    public byte[] delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> versionedRecord = inner.delete(key, timestamp);
+        return serializeAsBytes(versionedRecord);
+    }
+
+    @Override
+    public String name() {
+        return inner.name();
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        inner.init(context, root);
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        inner.init(context, root);
+    }
+
+    @Override
+    public void flush() {
+        inner.flush();
+    }
+
+    @Override
+    public void close() {
+        inner.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return inner.persistent();
+    }
+
+    @Override
+    public boolean isOpen() {
+        return inner.persistent();
+    }
+
+    @Override
+    public <R> QueryResult<R> query(final Query<R> query, final PositionBound positionBound, final QueryConfig config) {
+        return inner.query(query, positionBound, config);
+    }
+
+    @Override
+    public Position getPosition() {
+        return inner.getPosition();
+    }
+
+    @Override
+    public byte[] putIfAbsent(final Bytes key, final byte[] value) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support putIfAbsent(key, value)");
+    }
+
+    @Override
+    public void putAll(final List<KeyValue<Bytes, byte[]>> entries) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support putAll(entries)");
+    }
+
+    @Override
+    public byte[] delete(final Bytes key) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support delete(key)");

Review Comment:
   Should we point to `delete(key, ts)` in the error message?



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


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13250: KAFKA-14491: [9/N] Add versioned bytes store and supplier

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13250:
URL: https://github.com/apache/kafka/pull/13250#discussion_r1109282015


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/VersionedKeyValueToBytesStoreAdapter.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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 java.util.List;
+import java.util.Objects;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes.ByteArraySerde;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.kstream.KTable;
+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.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.KeyValueIterator;
+import org.apache.kafka.streams.state.KeyValueStore;
+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;
+
+/**
+ * Adapts from {@link VersionedKeyValueStore} (user-friendly versioned store interface) to
+ * {@link KeyValueStore}. By representing a {@code VersionedKeyValueStore} as a
+ * {@code KeyValueStore}, this allows reuse of existing {@link StreamsBuilder} and {@link KTable}
+ * method interfaces which accept {@code Materialized<K, V, KeyValueStore<Bytes, byte[]>)}
+ * for versioned key-value stores.
+ */
+public class VersionedKeyValueToBytesStoreAdapter implements VersionedBytesStore {
+    private static final Serde<ValueAndTimestamp<byte[]>> VALUE_AND_TIMESTAMP_SERDE
+        = new NullableValueAndTimestampSerde<>(new ByteArraySerde());
+    private static final Serializer<ValueAndTimestamp<byte[]>> VALUE_AND_TIMESTAMP_SERIALIZER
+        = VALUE_AND_TIMESTAMP_SERDE.serializer();
+    private static final Deserializer<ValueAndTimestamp<byte[]>> VALUE_AND_TIMESTAMP_DESERIALIZER
+        = VALUE_AND_TIMESTAMP_SERDE.deserializer();
+
+    final VersionedKeyValueStore<Bytes, byte[]> inner;
+
+    public VersionedKeyValueToBytesStoreAdapter(final VersionedKeyValueStore<Bytes, byte[]> inner) {
+        this.inner = Objects.requireNonNull(inner);
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] rawValueAndTimestamp) {
+        if (rawValueAndTimestamp == null) {
+            throw new IllegalArgumentException("Put to versioned store must always include timestamp, including for tombstones.");
+        }
+        final ValueAndTimestamp<byte[]> valueAndTimestamp
+            = VALUE_AND_TIMESTAMP_DESERIALIZER.deserialize(null, rawValueAndTimestamp);
+        inner.put(
+            key,
+            valueAndTimestamp.value(),
+            valueAndTimestamp.timestamp()
+        );
+    }
+
+    @Override
+    public byte[] get(final Bytes key) {
+        final VersionedRecord<byte[]> versionedRecord = inner.get(key);
+        return serializeAsBytes(versionedRecord);
+    }
+
+    @Override
+    public byte[] get(final Bytes key, final long asOfTimestamp) {
+        final VersionedRecord<byte[]> versionedRecord = inner.get(key, asOfTimestamp);
+        return serializeAsBytes(versionedRecord);
+    }
+
+    @Override
+    public byte[] delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> versionedRecord = inner.delete(key, timestamp);
+        return serializeAsBytes(versionedRecord);
+    }
+
+    @Override
+    public String name() {
+        return inner.name();
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        inner.init(context, root);
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        inner.init(context, root);
+    }
+
+    @Override
+    public void flush() {
+        inner.flush();
+    }
+
+    @Override
+    public void close() {
+        inner.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return inner.persistent();
+    }
+
+    @Override
+    public boolean isOpen() {
+        return inner.persistent();
+    }
+
+    @Override
+    public <R> QueryResult<R> query(final Query<R> query, final PositionBound positionBound, final QueryConfig config) {
+        return inner.query(query, positionBound, config);
+    }
+
+    @Override
+    public Position getPosition() {
+        return inner.getPosition();
+    }
+
+    @Override
+    public byte[] putIfAbsent(final Bytes key, final byte[] value) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support putIfAbsent(key, value)");
+    }
+
+    @Override
+    public void putAll(final List<KeyValue<Bytes, byte[]>> entries) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support putAll(entries)");
+    }
+
+    @Override
+    public byte[] delete(final Bytes key) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support delete(key)");
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> range(final Bytes from, final Bytes to) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support range(from, to)");
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> reverseRange(final Bytes from, final Bytes to) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support reverseRange(from, to)");
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> all() {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support all()");
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> reverseAll() {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support reverseAll()");
+    }
+
+    @Override
+    public <PS extends Serializer<P>, P> KeyValueIterator<Bytes, byte[]> prefixScan(final P prefix, final PS prefixKeySerializer) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support prefixScan(prefix, prefixKeySerializer)");
+    }
+
+    @Override
+    public long approximateNumEntries() {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support approximateNumEntries()");

Review Comment:
   We can definitely discuss whether we want to expose this for the RocksDB-based implementation but I think it'll require adding the method (perhaps with a default implementation which throws UnsupportOperationException) into the VersionedKeyValueStore interface, since this adapter here is general-purpose and won't be able to adapt from a generic VersionedKeyValueStore implementation without the method being part of the interface. In light of that, let's defer to future discussions for a follow-up KIP since there are other methods we might like to add to the interface in the future as well.



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


[GitHub] [kafka] mjsax merged pull request #13250: KAFKA-14491: [9/N] Add versioned bytes store and supplier

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax merged PR #13250:
URL: https://github.com/apache/kafka/pull/13250


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


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13250: KAFKA-14491: [9/N] Add versioned bytes store and supplier

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13250:
URL: https://github.com/apache/kafka/pull/13250#discussion_r1109276286


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/VersionedKeyValueToBytesStoreAdapter.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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 java.util.List;
+import java.util.Objects;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes.ByteArraySerde;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.kstream.KTable;
+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.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.KeyValueIterator;
+import org.apache.kafka.streams.state.KeyValueStore;
+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;
+
+/**
+ * Adapts from {@link VersionedKeyValueStore} (user-friendly versioned store interface) to
+ * {@link KeyValueStore}. By representing a {@code VersionedKeyValueStore} as a
+ * {@code KeyValueStore}, this allows reuse of existing {@link StreamsBuilder} and {@link KTable}
+ * method interfaces which accept {@code Materialized<K, V, KeyValueStore<Bytes, byte[]>)}
+ * for versioned key-value stores.
+ */
+public class VersionedKeyValueToBytesStoreAdapter implements VersionedBytesStore {
+    private static final Serde<ValueAndTimestamp<byte[]>> VALUE_AND_TIMESTAMP_SERDE
+        = new NullableValueAndTimestampSerde<>(new ByteArraySerde());
+    private static final Serializer<ValueAndTimestamp<byte[]>> VALUE_AND_TIMESTAMP_SERIALIZER
+        = VALUE_AND_TIMESTAMP_SERDE.serializer();
+    private static final Deserializer<ValueAndTimestamp<byte[]>> VALUE_AND_TIMESTAMP_DESERIALIZER
+        = VALUE_AND_TIMESTAMP_SERDE.deserializer();
+
+    final VersionedKeyValueStore<Bytes, byte[]> inner;
+
+    public VersionedKeyValueToBytesStoreAdapter(final VersionedKeyValueStore<Bytes, byte[]> inner) {
+        this.inner = Objects.requireNonNull(inner);
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] rawValueAndTimestamp) {
+        if (rawValueAndTimestamp == null) {
+            throw new IllegalArgumentException("Put to versioned store must always include timestamp, including for tombstones.");
+        }
+        final ValueAndTimestamp<byte[]> valueAndTimestamp
+            = VALUE_AND_TIMESTAMP_DESERIALIZER.deserialize(null, rawValueAndTimestamp);
+        inner.put(
+            key,
+            valueAndTimestamp.value(),
+            valueAndTimestamp.timestamp()
+        );
+    }
+
+    @Override
+    public byte[] get(final Bytes key) {
+        final VersionedRecord<byte[]> versionedRecord = inner.get(key);
+        return serializeAsBytes(versionedRecord);
+    }
+
+    @Override
+    public byte[] get(final Bytes key, final long asOfTimestamp) {
+        final VersionedRecord<byte[]> versionedRecord = inner.get(key, asOfTimestamp);
+        return serializeAsBytes(versionedRecord);
+    }
+
+    @Override
+    public byte[] delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> versionedRecord = inner.delete(key, timestamp);
+        return serializeAsBytes(versionedRecord);
+    }
+
+    @Override
+    public String name() {
+        return inner.name();
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        inner.init(context, root);
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        inner.init(context, root);
+    }
+
+    @Override
+    public void flush() {
+        inner.flush();
+    }
+
+    @Override
+    public void close() {
+        inner.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return inner.persistent();
+    }
+
+    @Override
+    public boolean isOpen() {
+        return inner.persistent();
+    }
+
+    @Override
+    public <R> QueryResult<R> query(final Query<R> query, final PositionBound positionBound, final QueryConfig config) {
+        return inner.query(query, positionBound, config);
+    }
+
+    @Override
+    public Position getPosition() {
+        return inner.getPosition();
+    }
+
+    @Override
+    public byte[] putIfAbsent(final Bytes key, final byte[] value) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support putIfAbsent(key, value)");
+    }
+
+    @Override
+    public void putAll(final List<KeyValue<Bytes, byte[]>> entries) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support putAll(entries)");
+    }
+
+    @Override
+    public byte[] delete(final Bytes key) {
+        throw new UnsupportedOperationException("Versioned key-value stores do not support delete(key)");

Review Comment:
   Sure, updated.



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