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


##########
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:
   Good catch, completely skipped my mind



##########
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:
   Yeah it doesn't really make sense here.



##########
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:
   Just a miss read. I thought session referred to the metrics access.... I'll 
change it to buffer.



##########
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:
   I'm not sure, I'll add some tests and try that out. Thanks for pointing it 
out



##########
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:
   No, I don't think it is necessary. Probably leftover, but I removed it now



##########
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:
   It shouldn't return null, there is no such element exception. I think we can 
just remove the if body as it will not execute.



##########
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:
   correct. I haven't finished the restoration logic yet



##########
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:
   Yeah, I missed that



##########
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:
   We can. but I like Duration because it forces a standard measurement of time 
(ie user can set it to minutes without having to convert to millis) also 
doesn't let non zero. 



##########
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:
   Why do I always forget that with test files?



##########
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:
   I messed around with this idea, unfortunately with how the evictions work 
that just pushes the complexity into the in memory implementation. It is a lot 
easier to wrap a value in a change record than the easier way around. At least 
not without changing how the fullchange serde works and a whole host of other 
things



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