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


##########
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:
   We are using it to store the record context. As for not supporting raw 
values, I don't think it would be much better unless we just made our own 
class. We can do that, but maybe we come back with an optimization later when 
there is time.



##########
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:
   I'll add int the protection for overflow, but I will figure out the seqnum 
being reset when I add the recovery logic in a later PR. That will have tests 
for things like this anyways and I think this PR is already getting too big.
   



##########
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 think we can just removed this check as it was actually always false.



##########
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:
   no, the root is null. The store than needs to be initialized is the wrapped 
one



##########
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:
   We can, shouldn't need to but might as well.



##########
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:
   Sure we can



##########
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:
   I think if we want to process something right away the correct thing todo 
would be to handle it outside of the buffer instead of insert then immeadilty 
evict. That would cost more serialization



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