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


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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, V> {
+
+    private final long gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRecords;
+    private Serde<K> keySerde;
+    private Serde<V> valueSerde;
+    private final String topic;
+    private int seqnum;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final 
RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod.toMillis();
+        minTimestamp = Long.MAX_VALUE;
+        numRecords = 0;
+        bufferSize = 0;
+        seqnum = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? getter.valueSerde() : valueSerde;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        wrapped().init(context, wrapped());

Review Comment:
   Should this be `init(context, root)` instead?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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, V> {
+
+    private final long gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRecords;
+    private Serde<K> keySerde;
+    private Serde<V> valueSerde;
+    private final String topic;
+    private int seqnum;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final 
RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod.toMillis();
+        minTimestamp = Long.MAX_VALUE;
+        numRecords = 0;
+        bufferSize = 0;
+        seqnum = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? 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;
+
+        if (predicate.get()) {
+            try (final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
+                .fetchAll(0, wrapped().observedStreamTime - gracePeriod)) {
+                while (iterator.hasNext() && predicate.get()) {
+                    keyValue = iterator.next();
+
+                    final BufferValue bufferValue = 
BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+                    final K key = keySerde.deserializer().deserialize(topic,
+                        
PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+                    minTimestamp = bufferValue.context().timestamp();
+
+                    if (wrapped().observedStreamTime - gracePeriod > 
minTimestamp) {
+                        return;
+                    }
+
+                    final V value = 
valueSerde.deserializer().deserialize(topic, bufferValue.newValue());
+
+                    if (bufferValue.context().timestamp() != minTimestamp) {
+                        throw new IllegalStateException(
+                            "minTimestamp [" + minTimestamp + "] did not match 
the actual min timestamp [" +
+                                bufferValue.context().timestamp() + "]"
+                        );
+                    }
+
+                    callback.accept(new Eviction<K, V>(key, value, 
bufferValue.context()));
+
+                    wrapped().remove(keyValue.key);
+                    numRecords--;
+                    bufferSize = bufferSize - computeRecordSize(keyValue.key, 
bufferValue);
+                }
+                if (numRecords == 0) {
+                    minTimestamp = Long.MAX_VALUE;
+                }
+            }
+        }
+    }
+
+
+    @Override
+    public Maybe<ValueAndTimestamp<V>> priorValueForBuffered(final K key) {
+        return Maybe.undefined();
+    }
+
+    @Override
+    public void put(final long time, final Record<K, V> record, final 
ProcessorRecordContext recordContext) {
+        requireNonNull(record.value(), "value cannot be null");
+        requireNonNull(recordContext, "recordContext cannot be null");
+        if (wrapped().observedStreamTime - gracePeriod > record.timestamp()) {
+            return;
+        }
+        final Bytes serializedKey = Bytes.wrap(
+            
PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.toStoreKeyBinary(keySerde.serializer().serialize(topic,
 record.key()),
+                record.timestamp(),
+                seqnum++).get());
+        final byte[] serialChange = valueSerde.serializer().serialize(topic, 
record.value());
+        final BufferValue buffered = new BufferValue(serialChange, 
serialChange, serialChange, recordContext);

Review Comment:
   How come we're still using `BufferValue`, now that we're writing just the 
value itself (and not a `Change` object) into the buffer? Is it because we 
still need to serialize the record context with the value? That makes sense to 
me if so, but two comments if so:
   1. `priorValue` and `oldValue` should be passed as null, rather than as 
`serialChange` here, in order to avoid storing the same value an extra time in 
the serialization.
   2. Even with the suggested change above, we'd still be serializing an extra 
8 bytes with each buffer record (4 bytes each for the length of the 
`priorValue` and `oldValue`, both of which are zero) which are unnecessary. How 
do you feel about modifying the `BufferValue` class to support storing raw 
values in addition to `Change` objects? I.e., with the "raw value" setting we 
wouldn't serialize the length of `priorValue` or `oldValue` at all, since 
they're understood to always be null.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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, V> {
+
+    private final long gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRecords;
+    private Serde<K> keySerde;
+    private Serde<V> valueSerde;
+    private final String topic;
+    private int seqnum;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final 
RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod.toMillis();
+        minTimestamp = Long.MAX_VALUE;
+        numRecords = 0;
+        bufferSize = 0;
+        seqnum = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? 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;
+
+        if (predicate.get()) {
+            try (final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
+                .fetchAll(0, wrapped().observedStreamTime - gracePeriod)) {
+                while (iterator.hasNext() && predicate.get()) {
+                    keyValue = iterator.next();
+
+                    final BufferValue bufferValue = 
BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+                    final K key = keySerde.deserializer().deserialize(topic,
+                        
PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+                    minTimestamp = bufferValue.context().timestamp();
+
+                    if (wrapped().observedStreamTime - gracePeriod > 
minTimestamp) {
+                        return;
+                    }
+
+                    final V value = 
valueSerde.deserializer().deserialize(topic, bufferValue.newValue());
+
+                    if (bufferValue.context().timestamp() != minTimestamp) {
+                        throw new IllegalStateException(
+                            "minTimestamp [" + minTimestamp + "] did not match 
the actual min timestamp [" +
+                                bufferValue.context().timestamp() + "]"
+                        );
+                    }
+
+                    callback.accept(new Eviction<K, V>(key, value, 
bufferValue.context()));
+
+                    wrapped().remove(keyValue.key);
+                    numRecords--;
+                    bufferSize = bufferSize - computeRecordSize(keyValue.key, 
bufferValue);
+                }
+                if (numRecords == 0) {
+                    minTimestamp = Long.MAX_VALUE;
+                }
+            }
+        }
+    }
+
+
+    @Override
+    public Maybe<ValueAndTimestamp<V>> priorValueForBuffered(final K key) {
+        return Maybe.undefined();
+    }
+
+    @Override
+    public void put(final long time, final Record<K, V> record, final 
ProcessorRecordContext recordContext) {
+        requireNonNull(record.value(), "value cannot be null");
+        requireNonNull(recordContext, "recordContext cannot be null");
+        if (wrapped().observedStreamTime - gracePeriod > record.timestamp()) {
+            return;

Review Comment:
   Wondering about the edge case for this contract: is it better for the buffer 
to reject the put entirely, or to still put the record in and let it be evicted 
as soon as the next eviction occurs? Maybe it doesn't matter if we never expect 
this case to actually occur, just curious to hear your thoughts.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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, V> {
+
+    private final long gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRecords;
+    private Serde<K> keySerde;
+    private Serde<V> valueSerde;
+    private final String topic;
+    private int seqnum;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final 
RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod.toMillis();
+        minTimestamp = Long.MAX_VALUE;
+        numRecords = 0;
+        bufferSize = 0;
+        seqnum = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? 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;
+
+        if (predicate.get()) {
+            try (final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
+                .fetchAll(0, wrapped().observedStreamTime - gracePeriod)) {
+                while (iterator.hasNext() && predicate.get()) {
+                    keyValue = iterator.next();
+
+                    final BufferValue bufferValue = 
BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+                    final K key = keySerde.deserializer().deserialize(topic,
+                        
PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+                    minTimestamp = bufferValue.context().timestamp();
+
+                    if (wrapped().observedStreamTime - gracePeriod > 
minTimestamp) {
+                        return;
+                    }
+
+                    final V value = 
valueSerde.deserializer().deserialize(topic, bufferValue.newValue());
+
+                    if (bufferValue.context().timestamp() != minTimestamp) {
+                        throw new IllegalStateException(
+                            "minTimestamp [" + minTimestamp + "] did not match 
the actual min timestamp [" +
+                                bufferValue.context().timestamp() + "]"
+                        );
+                    }
+
+                    callback.accept(new Eviction<K, V>(key, value, 
bufferValue.context()));
+
+                    wrapped().remove(keyValue.key);
+                    numRecords--;
+                    bufferSize = bufferSize - computeRecordSize(keyValue.key, 
bufferValue);
+                }
+                if (numRecords == 0) {
+                    minTimestamp = Long.MAX_VALUE;
+                }
+            }
+        }
+    }
+
+
+    @Override
+    public Maybe<ValueAndTimestamp<V>> priorValueForBuffered(final K key) {
+        return Maybe.undefined();
+    }
+
+    @Override
+    public void put(final long time, final Record<K, V> record, final 
ProcessorRecordContext recordContext) {
+        requireNonNull(record.value(), "value cannot be null");

Review Comment:
   Should we validate that the key is not null either?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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, V> {
+
+    private final long gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRecords;
+    private Serde<K> keySerde;
+    private Serde<V> valueSerde;
+    private final String topic;
+    private int seqnum;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final 
RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod.toMillis();
+        minTimestamp = Long.MAX_VALUE;
+        numRecords = 0;
+        bufferSize = 0;
+        seqnum = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? 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;
+
+        if (predicate.get()) {
+            try (final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
+                .fetchAll(0, wrapped().observedStreamTime - gracePeriod)) {
+                while (iterator.hasNext() && predicate.get()) {
+                    keyValue = iterator.next();
+
+                    final BufferValue bufferValue = 
BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+                    final K key = keySerde.deserializer().deserialize(topic,
+                        
PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+                    minTimestamp = bufferValue.context().timestamp();
+
+                    if (wrapped().observedStreamTime - gracePeriod > 
minTimestamp) {

Review Comment:
   I don't understand this if-condition. On the line above we set 
`minTimestamp` to be the timestamp of this buffer record, which is between `0` 
and `wrapped().observedStreamTime - gracePeriod` based on the bounds of the 
iterator. So, won't this if-condition always evaluate to true, in which case 
this method never evicts anything? 



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbTimeOrderedKeyValueBytesStoreSupplier.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+    public RocksDbTimeOrderedKeyValueBytesStoreSupplier(final String name,
+                                                        final long 
retentionPeriod) {
+        this.name = name + "-buffer";

Review Comment:
   Should we remove the `-buffer` suffix from the name here, if the intention 
is that `RocksDbTimeOrderedKeyValueBytesStoreSupplier` can be used for 
non-buffer use cases as well?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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, V> {
+
+    private final long gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRecords;
+    private Serde<K> keySerde;
+    private Serde<V> valueSerde;
+    private final String topic;
+    private int seqnum;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final 
RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod.toMillis();
+        minTimestamp = Long.MAX_VALUE;
+        numRecords = 0;
+        bufferSize = 0;
+        seqnum = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? 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;
+
+        if (predicate.get()) {
+            try (final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
+                .fetchAll(0, wrapped().observedStreamTime - gracePeriod)) {
+                while (iterator.hasNext() && predicate.get()) {
+                    keyValue = iterator.next();
+
+                    final BufferValue bufferValue = 
BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+                    final K key = keySerde.deserializer().deserialize(topic,
+                        
PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+                    minTimestamp = bufferValue.context().timestamp();
+
+                    if (wrapped().observedStreamTime - gracePeriod > 
minTimestamp) {
+                        return;
+                    }
+
+                    final V value = 
valueSerde.deserializer().deserialize(topic, bufferValue.newValue());
+
+                    if (bufferValue.context().timestamp() != minTimestamp) {
+                        throw new IllegalStateException(
+                            "minTimestamp [" + minTimestamp + "] did not match 
the actual min timestamp [" +
+                                bufferValue.context().timestamp() + "]"
+                        );
+                    }
+
+                    callback.accept(new Eviction<K, V>(key, value, 
bufferValue.context()));
+
+                    wrapped().remove(keyValue.key);
+                    numRecords--;
+                    bufferSize = bufferSize - computeRecordSize(keyValue.key, 
bufferValue);
+                }
+                if (numRecords == 0) {
+                    minTimestamp = Long.MAX_VALUE;
+                }
+            }
+        }
+    }
+
+
+    @Override
+    public Maybe<ValueAndTimestamp<V>> priorValueForBuffered(final K key) {
+        return Maybe.undefined();
+    }
+
+    @Override
+    public void put(final long time, final Record<K, V> record, final 
ProcessorRecordContext recordContext) {
+        requireNonNull(record.value(), "value cannot be null");
+        requireNonNull(recordContext, "recordContext cannot be null");
+        if (wrapped().observedStreamTime - gracePeriod > record.timestamp()) {
+            return;
+        }
+        final Bytes serializedKey = Bytes.wrap(
+            
PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.toStoreKeyBinary(keySerde.serializer().serialize(topic,
 record.key()),
+                record.timestamp(),
+                seqnum++).get());
+        final byte[] serialChange = valueSerde.serializer().serialize(topic, 
record.value());

Review Comment:
   nit: rename variable now that this is no longer a `Change` object.



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBufferTest.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.api.Record;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.test.MockInternalNewProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.time.Duration;
+import java.util.concurrent.atomic.AtomicInteger;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.equalTo;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.StrictStubs.class)
+public class RocksDBTimeOrderedKeyValueBufferTest {
+    public RocksDBTimeOrderedKeyValueBuffer<String, String> buffer;
+    @Mock
+    public SerdeGetter serdeGetter;
+    public InternalProcessorContext<String, String> context;
+    public StreamsMetricsImpl streamsMetrics;
+    @Mock
+    public Sensor sensor;
+    public long offset;
+
+    @Before
+    public void setUp() {
+        when(serdeGetter.keySerde()).thenReturn(new Serdes.StringSerde());
+        when(serdeGetter.valueSerde()).thenReturn(new Serdes.StringSerde());
+        final Metrics metrics = new Metrics();
+        offset = 0;
+        streamsMetrics = new StreamsMetricsImpl(metrics, "test-client", 
StreamsConfig.METRICS_LATEST, new MockTime());
+        context = new 
MockInternalNewProcessorContext<>(StreamsTestUtils.getStreamsConfig(), new 
TaskId(0, 0), TestUtils.tempDirectory());
+    }
+
+    public void createJoin(final Duration grace) {
+        final RocksDBTimeOrderedKeyValueSegmentedBytesStore store = new 
RocksDbTimeOrderedKeyValueBytesStoreSupplier("testing",  100).get();
+        buffer = new RocksDBTimeOrderedKeyValueBuffer<>(store, grace, 
"testing");
+        buffer.setSerdesIfNull(serdeGetter);
+        store.init((StateStoreContext) context, store);
+        buffer.init((StateStoreContext) context, store);
+    }
+
+    private void pipeRecord(final String key, final String value, final long 
time) {
+        final Record<String, String> record = new Record<>(key, value, time);
+        context.setRecordContext(new ProcessorRecordContext(time, offset++, 0, 
"testing", new RecordHeaders()));
+        buffer.put(time, record, context.recordContext());
+    }
+
+    @Test
+    public void shouldAddAndEvictRecord() {
+        createJoin(Duration.ZERO);
+        final AtomicInteger count = new AtomicInteger(0);
+        pipeRecord("1", "0", 0L);
+        buffer.evictWhile(() -> buffer.numRecords() > 0, r -> 
count.getAndIncrement());
+        assertThat(count.get(), equalTo(1));
+    }
+
+    @Test
+    public void shouldAddAndEvictRecordTwice() {
+        createJoin(Duration.ZERO);
+        final AtomicInteger count = new AtomicInteger(0);
+        pipeRecord("1", "0", 0L);
+        buffer.evictWhile(() -> buffer.numRecords() > 0, r -> 
count.getAndIncrement());
+        assertThat(count.get(), equalTo(1));
+        pipeRecord("2", "0", 1L);
+        buffer.evictWhile(() -> buffer.numRecords() > 0, r -> 
count.getAndIncrement());
+        assertThat(count.get(), equalTo(2));
+    }
+
+    @Test
+    public void shouldAddAndEvictRecordTwiceWithNonZeroGrace() {
+        createJoin(Duration.ofMillis(1));
+        final AtomicInteger count = new AtomicInteger(0);
+        pipeRecord("1", "0", 0L);
+        buffer.evictWhile(() -> buffer.numRecords() > 0, r -> 
count.getAndIncrement());
+        assertThat(count.get(), equalTo(0));
+        pipeRecord("2", "0", 1L);
+        buffer.evictWhile(() -> buffer.numRecords() > 0, r -> 
count.getAndIncrement());
+        assertThat(count.get(), equalTo(1));
+    }
+
+    @Test
+    public void shouldAddRecrodsTwiceAndEvictRecordsOnce() {

Review Comment:
   nit: typo in `Recrods`



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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, T> extends 
WrappedStateStore<RocksDBTimeOrderedKeyValueSegmentedBytesStore, Object, 
Object> implements TimeOrderedKeyValueBuffer<K, V, V> {
+
+    private final long gracePeriod;
+    private long bufferSize;
+    private long minTimestamp;
+    private int numRecords;
+    private Serde<K> keySerde;
+    private Serde<V> valueSerde;
+    private final String topic;
+    private int seqnum;
+
+    public RocksDBTimeOrderedKeyValueBuffer(final 
RocksDBTimeOrderedKeyValueSegmentedBytesStore store,
+                                            final Duration gracePeriod,
+                                            final String topic) {
+        super(store);
+        this.gracePeriod = gracePeriod.toMillis();
+        minTimestamp = Long.MAX_VALUE;
+        numRecords = 0;
+        bufferSize = 0;
+        seqnum = 0;
+        this.topic = topic;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void setSerdesIfNull(final SerdeGetter getter) {
+        keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
+        valueSerde = valueSerde == null ? 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;
+
+        if (predicate.get()) {
+            try (final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
+                .fetchAll(0, wrapped().observedStreamTime - gracePeriod)) {
+                while (iterator.hasNext() && predicate.get()) {
+                    keyValue = iterator.next();
+
+                    final BufferValue bufferValue = 
BufferValue.deserialize(ByteBuffer.wrap(keyValue.value));
+                    final K key = keySerde.deserializer().deserialize(topic,
+                        
PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.extractStoreKeyBytes(keyValue.key.get()));
+                    minTimestamp = bufferValue.context().timestamp();
+
+                    if (wrapped().observedStreamTime - gracePeriod > 
minTimestamp) {
+                        return;
+                    }
+
+                    V value = valueSerde.deserializer().deserialize(topic, 
bufferValue.newValue());
+
+                    if (bufferValue.context().timestamp() != minTimestamp) {
+                        throw new IllegalStateException(
+                            "minTimestamp [" + minTimestamp + "] did not match 
the actual min timestamp [" +
+                                bufferValue.context().timestamp() + "]"
+                        );
+                    }
+
+                    callback.accept(new Eviction<K, V>(key, value, 
bufferValue.context()));
+
+                    wrapped().remove(keyValue.key);
+                    numRecords--;
+                    bufferSize = bufferSize - computeRecordSize(keyValue.key, 
bufferValue);
+                }
+                if (numRecords == 0) {
+                    minTimestamp = Long.MAX_VALUE;
+                }
+            }
+        }
+    }
+
+
+    @Override
+    public Maybe<ValueAndTimestamp<V>> priorValueForBuffered(final K key) {
+        return Maybe.undefined();
+    }
+
+    @Override
+    public void put(final long time, final Record<K, V> record, final 
ProcessorRecordContext recordContext) {
+        requireNonNull(record.value(), "value cannot be null");
+        requireNonNull(recordContext, "recordContext cannot be null");
+        if (wrapped().observedStreamTime - gracePeriod > record.timestamp()) {
+            return;
+        }
+        final Bytes serializedKey = Bytes.wrap(
+            
PrefixedWindowKeySchemas.TimeFirstWindowKeySchema.toStoreKeyBinary(keySerde.serializer().serialize(topic,
 record.key()),
+                record.timestamp(),
+                seqnum++).get());

Review Comment:
   +1 to Bruno's confusion here. I'm not sure how this works for 
`RocksDBTimeOrderedWindowStore` either.
   
   Also, if we go with the approach of incrementing seqnum, we need to guard 
against overflow. (`RocksDBTimeOrderedWindowStore` does this in 
`maybeUpdateSeqnumForDups()`.)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

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


Reply via email to