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/05/31 20:24:13 UTC

[GitHub] [kafka] vcrfxia commented on a diff in pull request #13756: KAFKA-14936: Add On Disk Time Ordered Buffer (1/N)

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


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.internals.Change;
+import org.apache.kafka.streams.kstream.internals.FullChangeSerde;
+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.api.Record;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
+
+public class RocksDBTimeOrderedKeyValueBuffer<K, V> extends WrappedStateStore<RocksDBTimeOrderedKeyValueSegmentedBytesStore, Object, Object> implements TimeOrderedKeyValueBuffer<K, V> {
+
+    private final Duration gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRec;
+    private Serde<K> keySerde;
+    private FullChangeSerde<V> valueSerde;
+    private String topic;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod;
+        minTimestamp = Long.MAX_VALUE;
+        numRec = 0;
+        bufferSize = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? FullChangeSerde.wrap((Serde<V>) getter.valueSerde()) : valueSerde;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void evictWhile(final Supplier<Boolean> predicate, final Consumer<Eviction<K, V>> callback) {
+        KeyValue<Bytes, byte[]> keyValue = null;
+
+        if (predicate.get()) {
+            final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
+                .fetchAll(0, wrapped().observedStreamTime - gracePeriod.toMillis());
+            if (iterator.hasNext()) {
+                keyValue = iterator.next();
+            }
+            if (keyValue == null) {
+                if (numRecords() == 0) {
+                    minTimestamp = Long.MAX_VALUE;
+                }
+                return;
+            }
+            BufferValue bufferValue = BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+            K key = keySerde.deserializer().deserialize(topic,
+                PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+
+            Change<V> value = valueSerde.deserializeParts(
+                topic,
+                new Change<>(bufferValue.newValue(), bufferValue.oldValue())
+            );
+            while (keyValue != null && predicate.get() && wrapped().observedStreamTime - gracePeriod.toMillis() >= minTimestamp()) {
+                if (bufferValue.context().timestamp() != minTimestamp) {
+                    throw new IllegalStateException(
+                        "minTimestamp [" + minTimestamp + "] did not match the actual min timestamp [" +
+                            bufferValue.context().timestamp() + "]"
+                    );
+                }
+                callback.accept(new Eviction<>(key, value, bufferValue.context()));
+                wrapped().remove(keyValue.key);
+                numRec--;
+                bufferSize = bufferSize - computeRecordSize(keyValue.key, bufferValue);
+                if (iterator.hasNext()) {
+                    keyValue = iterator.next();
+                    if (keyValue == null) {
+                        minTimestamp = Long.MAX_VALUE;
+                    } else {
+                        bufferValue = BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+                        key = keySerde.deserializer().deserialize(topic,
+                            PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+                        value = valueSerde.deserializeParts(
+                            topic,
+                            new Change<>(bufferValue.newValue(), bufferValue.oldValue())
+                        );
+                        minTimestamp = bufferValue.context().timestamp();
+                    }
+                } else {
+                    keyValue = null;
+                    minTimestamp = Long.MAX_VALUE;
+                }
+            }
+        }
+    }
+
+
+    @Override
+    public Maybe<ValueAndTimestamp<V>> priorValueForBuffered(final K key) {
+        return null;
+    }
+
+    @Override
+    public void put(final long time, final Record<K, Change<V>> record, final ProcessorRecordContext recordContext) {
+        requireNonNull(record.value(), "value cannot be null");
+        requireNonNull(recordContext, "recordContext cannot be null");
+        final Bytes serializedKey = Bytes.wrap(
+            PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.toStoreKeyBinary(keySerde.serializer().serialize(topic, record.key()),
+                record.timestamp(),
+                0).get());

Review Comment:
   Is it correct to always pass zero here? In the event that the stream-side of the join contains multiple records with the same key and timestamp, I think we'll need to increment this value otherwise stream-side records will be inadvertently dropped from the join.
   
   It'd be good to get unit test coverage for this case as well.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.internals.Change;
+import org.apache.kafka.streams.kstream.internals.FullChangeSerde;
+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.api.Record;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
+
+public class RocksDBTimeOrderedKeyValueBuffer<K, V> extends WrappedStateStore<RocksDBTimeOrderedKeyValueSegmentedBytesStore, Object, Object> implements TimeOrderedKeyValueBuffer<K, V> {
+
+    private final Duration gracePeriod;

Review Comment:
   nit: perhaps we should store this as a long (`final long gracePeriodMs`), since it's accessed as a long everywhere it's used?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.internals.Change;
+import org.apache.kafka.streams.kstream.internals.FullChangeSerde;
+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.api.Record;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
+
+public class RocksDBTimeOrderedKeyValueBuffer<K, V> extends WrappedStateStore<RocksDBTimeOrderedKeyValueSegmentedBytesStore, Object, Object> implements TimeOrderedKeyValueBuffer<K, V> {
+
+    private final Duration gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRec;
+    private Serde<K> keySerde;
+    private FullChangeSerde<V> valueSerde;
+    private String topic;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod;
+        minTimestamp = Long.MAX_VALUE;
+        numRec = 0;
+        bufferSize = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? FullChangeSerde.wrap((Serde<V>) getter.valueSerde()) : valueSerde;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void evictWhile(final Supplier<Boolean> predicate, final Consumer<Eviction<K, V>> callback) {
+        KeyValue<Bytes, byte[]> keyValue = null;
+
+        if (predicate.get()) {
+            final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
+                .fetchAll(0, wrapped().observedStreamTime - gracePeriod.toMillis());
+            if (iterator.hasNext()) {
+                keyValue = iterator.next();
+            }
+            if (keyValue == null) {
+                if (numRecords() == 0) {
+                    minTimestamp = Long.MAX_VALUE;
+                }
+                return;
+            }
+            BufferValue bufferValue = BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+            K key = keySerde.deserializer().deserialize(topic,
+                PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+
+            Change<V> value = valueSerde.deserializeParts(
+                topic,
+                new Change<>(bufferValue.newValue(), bufferValue.oldValue())
+            );
+            while (keyValue != null && predicate.get() && wrapped().observedStreamTime - gracePeriod.toMillis() >= minTimestamp()) {

Review Comment:
   Is this time check (`wrapped().observedStreamTime - gracePeriod.toMillis() >= minTimestamp()`) needed here, given that the call to `fetchAll()` above already specifies the same time bound?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.internals.Change;
+import org.apache.kafka.streams.kstream.internals.FullChangeSerde;
+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.api.Record;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
+
+public class RocksDBTimeOrderedKeyValueBuffer<K, V> extends WrappedStateStore<RocksDBTimeOrderedKeyValueSegmentedBytesStore, Object, Object> implements TimeOrderedKeyValueBuffer<K, V> {
+
+    private final Duration gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRec;

Review Comment:
   nit: rename to `numRecords`?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueSegmentedBytesStore.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.Collection;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.KeyFirstWindowKeySchema;
+import org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.TimeFirstWindowKeySchema;
+import org.rocksdb.WriteBatch;
+
+/**
+ * A RocksDB backed time-ordered segmented bytes store for window key schema.
+ */
+public class RocksDBTimeOrderedKeyValueSegmentedBytesStore extends AbstractRocksDBTimeOrderedSegmentedBytesStore {
+
+    RocksDBTimeOrderedKeyValueSegmentedBytesStore(final String name,
+                                                  final String metricsScope,
+                                                  final long retention,
+                                                  final long segmentInterval,
+                                                  final boolean withIndex) {
+        super(name, metricsScope, retention, segmentInterval, new TimeFirstWindowKeySchema(),
+            Optional.ofNullable(withIndex ? new KeyFirstWindowKeySchema() : null));
+    }
+
+    @Override
+    protected KeyValue<Bytes, byte[]> getIndexKeyValue(final Bytes baseKey, final byte[] baseValue) {
+        throw new UnsupportedOperationException("Do not use for TimeOrderedKeyValueStore");
+    }
+
+    @Override
+    Map<KeyValueSegment, WriteBatch> getWriteBatches(final Collection<ConsumerRecord<byte[], byte[]>> records) {

Review Comment:
   Restoration won't work without implementing this method, right? Is that going to be added in a follow-up PR?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueSegmentedBytesStore.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.Collection;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.KeyFirstWindowKeySchema;
+import org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.TimeFirstWindowKeySchema;
+import org.rocksdb.WriteBatch;
+
+/**
+ * A RocksDB backed time-ordered segmented bytes store for window key schema.
+ */
+public class RocksDBTimeOrderedKeyValueSegmentedBytesStore extends AbstractRocksDBTimeOrderedSegmentedBytesStore {
+
+    RocksDBTimeOrderedKeyValueSegmentedBytesStore(final String name,
+                                                  final String metricsScope,
+                                                  final long retention,
+                                                  final long segmentInterval,
+                                                  final boolean withIndex) {

Review Comment:
   Is `withIndex` ever going to be `true` here? If the only use case we're planning to use this new store for is the stream buffer for stream-table joins, then it seems it'll always be false. If that's the case then I think it'd be good to remove this parameter and just always set it to `false` for this store type, particularly since `getIndexKeyValue()` and `getIndexToBaseStoreIterator()` are unsupported anyway.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.internals.Change;
+import org.apache.kafka.streams.kstream.internals.FullChangeSerde;
+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.api.Record;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
+
+public class RocksDBTimeOrderedKeyValueBuffer<K, V> extends WrappedStateStore<RocksDBTimeOrderedKeyValueSegmentedBytesStore, Object, Object> implements TimeOrderedKeyValueBuffer<K, V> {
+
+    private final Duration gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRec;
+    private Serde<K> keySerde;
+    private FullChangeSerde<V> valueSerde;
+    private String topic;

Review Comment:
   nit: can be final



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBufferTest.java:
##########
@@ -0,0 +1,136 @@
+package org.apache.kafka.streams.state.internals;

Review Comment:
   nit: missing header



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.internals.Change;
+import org.apache.kafka.streams.kstream.internals.FullChangeSerde;
+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.api.Record;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
+
+public class RocksDBTimeOrderedKeyValueBuffer<K, V> extends WrappedStateStore<RocksDBTimeOrderedKeyValueSegmentedBytesStore, Object, Object> implements TimeOrderedKeyValueBuffer<K, V> {
+
+    private final Duration gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRec;
+    private Serde<K> keySerde;
+    private FullChangeSerde<V> valueSerde;
+    private String topic;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod;
+        minTimestamp = Long.MAX_VALUE;
+        numRec = 0;
+        bufferSize = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? FullChangeSerde.wrap((Serde<V>) getter.valueSerde()) : valueSerde;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void evictWhile(final Supplier<Boolean> predicate, final Consumer<Eviction<K, V>> callback) {
+        KeyValue<Bytes, byte[]> keyValue = null;
+
+        if (predicate.get()) {
+            final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
+                .fetchAll(0, wrapped().observedStreamTime - gracePeriod.toMillis());
+            if (iterator.hasNext()) {
+                keyValue = iterator.next();
+            }
+            if (keyValue == null) {
+                if (numRecords() == 0) {
+                    minTimestamp = Long.MAX_VALUE;
+                }
+                return;
+            }
+            BufferValue bufferValue = BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+            K key = keySerde.deserializer().deserialize(topic,
+                PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+
+            Change<V> value = valueSerde.deserializeParts(
+                topic,
+                new Change<>(bufferValue.newValue(), bufferValue.oldValue())
+            );
+            while (keyValue != null && predicate.get() && wrapped().observedStreamTime - gracePeriod.toMillis() >= minTimestamp()) {
+                if (bufferValue.context().timestamp() != minTimestamp) {
+                    throw new IllegalStateException(
+                        "minTimestamp [" + minTimestamp + "] did not match the actual min timestamp [" +
+                            bufferValue.context().timestamp() + "]"
+                    );
+                }
+                callback.accept(new Eviction<>(key, value, bufferValue.context()));
+                wrapped().remove(keyValue.key);
+                numRec--;
+                bufferSize = bufferSize - computeRecordSize(keyValue.key, bufferValue);
+                if (iterator.hasNext()) {
+                    keyValue = iterator.next();
+                    if (keyValue == null) {
+                        minTimestamp = Long.MAX_VALUE;
+                    } else {
+                        bufferValue = BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+                        key = keySerde.deserializer().deserialize(topic,
+                            PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+                        value = valueSerde.deserializeParts(
+                            topic,
+                            new Change<>(bufferValue.newValue(), bufferValue.oldValue())
+                        );
+                        minTimestamp = bufferValue.context().timestamp();
+                    }
+                } else {
+                    keyValue = null;
+                    minTimestamp = Long.MAX_VALUE;
+                }
+            }
+        }
+    }
+
+
+    @Override
+    public Maybe<ValueAndTimestamp<V>> priorValueForBuffered(final K key) {
+        return null;

Review Comment:
   Should this be `Maybe.undefined()` instead, in order to avoid NPEs?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.internals.Change;
+import org.apache.kafka.streams.kstream.internals.FullChangeSerde;
+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.api.Record;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
+
+public class RocksDBTimeOrderedKeyValueBuffer<K, V> extends WrappedStateStore<RocksDBTimeOrderedKeyValueSegmentedBytesStore, Object, Object> implements TimeOrderedKeyValueBuffer<K, V> {
+
+    private final Duration gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRec;
+    private Serde<K> keySerde;
+    private FullChangeSerde<V> valueSerde;
+    private String topic;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod;
+        minTimestamp = Long.MAX_VALUE;
+        numRec = 0;
+        bufferSize = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? FullChangeSerde.wrap((Serde<V>) getter.valueSerde()) : valueSerde;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void evictWhile(final Supplier<Boolean> predicate, final Consumer<Eviction<K, V>> callback) {
+        KeyValue<Bytes, byte[]> keyValue = null;
+
+        if (predicate.get()) {
+            final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
+                .fetchAll(0, wrapped().observedStreamTime - gracePeriod.toMillis());
+            if (iterator.hasNext()) {
+                keyValue = iterator.next();
+            }
+            if (keyValue == null) {
+                if (numRecords() == 0) {
+                    minTimestamp = Long.MAX_VALUE;
+                }
+                return;
+            }
+            BufferValue bufferValue = BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+            K key = keySerde.deserializer().deserialize(topic,
+                PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+
+            Change<V> value = valueSerde.deserializeParts(
+                topic,
+                new Change<>(bufferValue.newValue(), bufferValue.oldValue())
+            );
+            while (keyValue != null && predicate.get() && wrapped().observedStreamTime - gracePeriod.toMillis() >= minTimestamp()) {
+                if (bufferValue.context().timestamp() != minTimestamp) {
+                    throw new IllegalStateException(
+                        "minTimestamp [" + minTimestamp + "] did not match the actual min timestamp [" +
+                            bufferValue.context().timestamp() + "]"
+                    );
+                }
+                callback.accept(new Eviction<>(key, value, bufferValue.context()));
+                wrapped().remove(keyValue.key);
+                numRec--;
+                bufferSize = bufferSize - computeRecordSize(keyValue.key, bufferValue);
+                if (iterator.hasNext()) {
+                    keyValue = iterator.next();
+                    if (keyValue == null) {

Review Comment:
   Just checking: is `null` a valid return value from this iterator? Hopefully not unless there's no actual data to be returned, else we could end up stuck in a loop here.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.internals.Change;
+import org.apache.kafka.streams.kstream.internals.FullChangeSerde;
+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.api.Record;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
+
+public class RocksDBTimeOrderedKeyValueBuffer<K, V> extends WrappedStateStore<RocksDBTimeOrderedKeyValueSegmentedBytesStore, Object, Object> implements TimeOrderedKeyValueBuffer<K, V> {
+
+    private final Duration gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRec;
+    private Serde<K> keySerde;
+    private FullChangeSerde<V> valueSerde;
+    private String topic;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod;
+        minTimestamp = Long.MAX_VALUE;
+        numRec = 0;
+        bufferSize = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? FullChangeSerde.wrap((Serde<V>) getter.valueSerde()) : valueSerde;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void evictWhile(final Supplier<Boolean> predicate, final Consumer<Eviction<K, V>> callback) {
+        KeyValue<Bytes, byte[]> keyValue = null;
+
+        if (predicate.get()) {
+            final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
+                .fetchAll(0, wrapped().observedStreamTime - gracePeriod.toMillis());
+            if (iterator.hasNext()) {
+                keyValue = iterator.next();
+            }
+            if (keyValue == null) {
+                if (numRecords() == 0) {
+                    minTimestamp = Long.MAX_VALUE;
+                }
+                return;
+            }
+            BufferValue bufferValue = BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+            K key = keySerde.deserializer().deserialize(topic,
+                PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+
+            Change<V> value = valueSerde.deserializeParts(
+                topic,
+                new Change<>(bufferValue.newValue(), bufferValue.oldValue())
+            );
+            while (keyValue != null && predicate.get() && wrapped().observedStreamTime - gracePeriod.toMillis() >= minTimestamp()) {
+                if (bufferValue.context().timestamp() != minTimestamp) {
+                    throw new IllegalStateException(
+                        "minTimestamp [" + minTimestamp + "] did not match the actual min timestamp [" +
+                            bufferValue.context().timestamp() + "]"
+                    );
+                }
+                callback.accept(new Eviction<>(key, value, bufferValue.context()));
+                wrapped().remove(keyValue.key);
+                numRec--;
+                bufferSize = bufferSize - computeRecordSize(keyValue.key, bufferValue);
+                if (iterator.hasNext()) {
+                    keyValue = iterator.next();
+                    if (keyValue == null) {
+                        minTimestamp = Long.MAX_VALUE;
+                    } else {
+                        bufferValue = BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+                        key = keySerde.deserializer().deserialize(topic,
+                            PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+                        value = valueSerde.deserializeParts(
+                            topic,
+                            new Change<>(bufferValue.newValue(), bufferValue.oldValue())
+                        );
+                        minTimestamp = bufferValue.context().timestamp();
+                    }
+                } else {
+                    keyValue = null;
+                    minTimestamp = Long.MAX_VALUE;
+                }
+            }
+        }
+    }
+
+
+    @Override
+    public Maybe<ValueAndTimestamp<V>> priorValueForBuffered(final K key) {
+        return null;
+    }
+
+    @Override
+    public void put(final long time, final Record<K, Change<V>> record, final ProcessorRecordContext recordContext) {
+        requireNonNull(record.value(), "value cannot be null");
+        requireNonNull(recordContext, "recordContext cannot be null");
+        final Bytes serializedKey = Bytes.wrap(
+            PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.toStoreKeyBinary(keySerde.serializer().serialize(topic, record.key()),
+                record.timestamp(),
+                0).get());
+        final Change<byte[]> serialChange = valueSerde.serializeParts(topic, record.value());
+        final BufferValue buffered = new BufferValue(serialChange.oldValue, serialChange.oldValue, serialChange.newValue, recordContext);
+        if (wrapped().observedStreamTime - gracePeriod.toMillis() > record.timestamp()) {

Review Comment:
   Move this grace period check before serializing the key and value, in order to save on serialization in situations where it's not needed?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.internals.Change;
+import org.apache.kafka.streams.kstream.internals.FullChangeSerde;
+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.api.Record;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
+
+public class RocksDBTimeOrderedKeyValueBuffer<K, V> extends WrappedStateStore<RocksDBTimeOrderedKeyValueSegmentedBytesStore, Object, Object> implements TimeOrderedKeyValueBuffer<K, V> {
+
+    private final Duration gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRec;
+    private Serde<K> keySerde;
+    private FullChangeSerde<V> valueSerde;
+    private String topic;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod;
+        minTimestamp = Long.MAX_VALUE;
+        numRec = 0;
+        bufferSize = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? FullChangeSerde.wrap((Serde<V>) getter.valueSerde()) : valueSerde;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void evictWhile(final Supplier<Boolean> predicate, final Consumer<Eviction<K, V>> callback) {
+        KeyValue<Bytes, byte[]> keyValue = null;
+
+        if (predicate.get()) {
+            final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
+                .fetchAll(0, wrapped().observedStreamTime - gracePeriod.toMillis());

Review Comment:
   This iterator is not being closed right now, which is a resource leak.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.internals.Change;
+import org.apache.kafka.streams.kstream.internals.FullChangeSerde;
+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.api.Record;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
+
+public class RocksDBTimeOrderedKeyValueBuffer<K, V> extends WrappedStateStore<RocksDBTimeOrderedKeyValueSegmentedBytesStore, Object, Object> implements TimeOrderedKeyValueBuffer<K, V> {
+
+    private final Duration gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRec;
+    private Serde<K> keySerde;
+    private FullChangeSerde<V> valueSerde;

Review Comment:
   Hmm, interesting that the serde type here is for `Change<V>` instead of plain `V`. Looks like the reason that's the case is because `TimeOrderedKeyValueBuffer` stores `Change<V>`. What do you think about updating `TimeOrderedKeyValueBuffer` to take a third generic type to represent the type stored in the actual buffer itself, in order to allow this new buffer implementation to simply store `V` instead? (The buffer type would be `Change<V>` for the existing implementations, and just `V` for this new implementation.)
   
   That would simplify a lot of code in this class which doesn't have use cases for old values anyway (also evidenced by the fact that `priorValueForBuffered()` always returns null), and callers of this class won't need to pass in `null` for the old value in `Change` either.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbTimeOrderedKeyValueBytesStoreSupplier.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+public class RocksDbTimeOrderedKeyValueBytesStoreSupplier {
+    private final String name;
+    private final long retentionPeriod;
+    private final boolean withIndex;
+
+    public RocksDbTimeOrderedKeyValueBytesStoreSupplier(final String name,
+                                                        final long retentionPeriod,
+                                                        final boolean withIndex) {
+        this.name = name + "-buffer";
+        this.retentionPeriod = retentionPeriod;
+        this.withIndex = withIndex;
+    }
+
+    public String name() {
+        return name;
+    }
+
+    public RocksDBTimeOrderedKeyValueSegmentedBytesStore get() {
+        return new RocksDBTimeOrderedKeyValueSegmentedBytesStore(
+            name,
+            metricsScope(),
+            retentionPeriod,
+            Math.max(retentionPeriod / 2, 60_000L),
+            withIndex
+        );
+    }
+
+    public String metricsScope() {
+        return "rocksdb-session";

Review Comment:
   Copy-paste error? The new store is not a session store.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.internals.Change;
+import org.apache.kafka.streams.kstream.internals.FullChangeSerde;
+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.api.Record;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
+
+public class RocksDBTimeOrderedKeyValueBuffer<K, V> extends WrappedStateStore<RocksDBTimeOrderedKeyValueSegmentedBytesStore, Object, Object> implements TimeOrderedKeyValueBuffer<K, V> {
+
+    private final Duration gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRec;
+    private Serde<K> keySerde;
+    private FullChangeSerde<V> valueSerde;
+    private String topic;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod;
+        minTimestamp = Long.MAX_VALUE;
+        numRec = 0;
+        bufferSize = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? FullChangeSerde.wrap((Serde<V>) getter.valueSerde()) : valueSerde;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        wrapped().init(context, wrapped());
+    }
+
+    @Override
+    public void evictWhile(final Supplier<Boolean> predicate, final Consumer<Eviction<K, V>> callback) {
+        KeyValue<Bytes, byte[]> keyValue = null;
+
+        if (predicate.get()) {
+            final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
+                .fetchAll(0, wrapped().observedStreamTime - gracePeriod.toMillis());
+            if (iterator.hasNext()) {
+                keyValue = iterator.next();
+            }
+            if (keyValue == null) {
+                if (numRecords() == 0) {
+                    minTimestamp = Long.MAX_VALUE;
+                }
+                return;
+            }
+            BufferValue bufferValue = BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+            K key = keySerde.deserializer().deserialize(topic,
+                PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+
+            Change<V> value = valueSerde.deserializeParts(
+                topic,
+                new Change<>(bufferValue.newValue(), bufferValue.oldValue())
+            );
+            while (keyValue != null && predicate.get() && wrapped().observedStreamTime - gracePeriod.toMillis() >= minTimestamp()) {
+                if (bufferValue.context().timestamp() != minTimestamp) {
+                    throw new IllegalStateException(
+                        "minTimestamp [" + minTimestamp + "] did not match the actual min timestamp [" +
+                            bufferValue.context().timestamp() + "]"
+                    );
+                }
+                callback.accept(new Eviction<>(key, value, bufferValue.context()));
+                wrapped().remove(keyValue.key);
+                numRec--;
+                bufferSize = bufferSize - computeRecordSize(keyValue.key, bufferValue);
+                if (iterator.hasNext()) {
+                    keyValue = iterator.next();
+                    if (keyValue == null) {
+                        minTimestamp = Long.MAX_VALUE;
+                    } else {
+                        bufferValue = BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+                        key = keySerde.deserializer().deserialize(topic,
+                            PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+                        value = valueSerde.deserializeParts(
+                            topic,
+                            new Change<>(bufferValue.newValue(), bufferValue.oldValue())
+                        );
+                        minTimestamp = bufferValue.context().timestamp();
+                    }
+                } else {
+                    keyValue = null;
+                    minTimestamp = Long.MAX_VALUE;
+                }
+            }
+        }
+    }
+
+
+    @Override
+    public Maybe<ValueAndTimestamp<V>> priorValueForBuffered(final K key) {
+        return null;
+    }
+
+    @Override
+    public void put(final long time, final Record<K, Change<V>> record, final ProcessorRecordContext recordContext) {
+        requireNonNull(record.value(), "value cannot be null");
+        requireNonNull(recordContext, "recordContext cannot be null");
+        final Bytes serializedKey = Bytes.wrap(
+            PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.toStoreKeyBinary(keySerde.serializer().serialize(topic, record.key()),
+                record.timestamp(),
+                0).get());
+        final Change<byte[]> serialChange = valueSerde.serializeParts(topic, record.value());
+        final BufferValue buffered = new BufferValue(serialChange.oldValue, serialChange.oldValue, serialChange.newValue, recordContext);
+        if (wrapped().observedStreamTime - gracePeriod.toMillis() > record.timestamp()) {
+            return;
+        }
+        wrapped().put(serializedKey, buffered.serialize(0).array());
+        bufferSize += computeRecordSize(serializedKey, buffered);
+        numRec++;
+        if (minTimestamp() > record.timestamp()) {
+            minTimestamp = record.timestamp();
+        }
+    }
+
+    @Override
+    public int numRecords() {
+        return numRec;
+    }
+
+    @Override
+    public long bufferSize() {
+        return bufferSize;
+    }
+
+    @Override
+    public long minTimestamp() {
+        return minTimestamp;
+    }
+
+    private static long computeRecordSize(final Bytes key, final BufferValue value) {
+        long size = 0L;
+        size += 8; // buffer time

Review Comment:
   Does this comment refer to the size of the record timestamp that's serialized with the record key? If so, I think that's double-counted now, since the key passed to this method is already the serialized form, which includes the timestamp.



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