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


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueSegmentedBytesStore.java:
##########
@@ -34,10 +34,9 @@ public class RocksDBTimeOrderedKeyValueSegmentedBytesStore 
extends AbstractRocks
     RocksDBTimeOrderedKeyValueSegmentedBytesStore(final String name,
                                                   final String metricsScope,
                                                   final long retention,
-                                                  final long segmentInterval,
-                                                  final boolean withIndex) {
+                                                  final long segmentInterval) {
         super(name, metricsScope, retention, segmentInterval, new 
TimeFirstWindowKeySchema(),
-            Optional.ofNullable(withIndex ? new KeyFirstWindowKeySchema() : 
null));
+            Optional.ofNullable(null));

Review Comment:
   nit: `Optional.empty()`



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBuffer.java:
##########
@@ -78,18 +78,21 @@ public void init(final StateStoreContext context, final 
StateStore root) {
 
     @Override
     public void evictWhile(final Supplier<Boolean> predicate, final 
Consumer<Eviction<K, V>> callback) {
-        KeyValue<Bytes, byte[]> keyValue = null;
+        KeyValue<Bytes, byte[]> keyValue;
 
         if (predicate.get()) {
             final KeyValueIterator<Bytes, byte[]> iterator = wrapped()
                 .fetchAll(0, wrapped().observedStreamTime - 
gracePeriod.toMillis());
             if (iterator.hasNext()) {
                 keyValue = iterator.next();
-            }
-            if (keyValue == null) {
-                minTimestamp = Long.MAX_VALUE;
+            } else {
+                if (numRecords() == 0) {
+                    minTimestamp = Long.MAX_VALUE;
+                }
+                iterator.close();

Review Comment:
   Use try-with-resource block instead, so we don't need to remember to close 
the iterator in all possible places before returning? Also so that if an 
exception occurs, the iterator is still not leaked.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbTimeOrderedKeyValueBytesStoreSupplier.java:
##########
@@ -38,13 +35,12 @@ public RocksDBTimeOrderedKeyValueSegmentedBytesStore get() {
             name,
             metricsScope(),
             retentionPeriod,
-            Math.max(retentionPeriod / 2, 60_000L),
-            withIndex
+            Math.max(retentionPeriod / 2, 60_000L)
         );
     }
 
     public String metricsScope() {
-        return "rocksdb-session";
+        return "rocksdb-buffer";

Review Comment:
   Do we want this store to specifically be used as a buffer, or should it 
support non-buffer use cases for RocksDB time-ordered stores as well? If we 
want it to be more extensible in the future, we can simply use `rocksdb` here 
(similar to `RocksDbKeyValueBytesStoreSupplier`). We'd also want to remove the 
`-buffer` suffix from the store name too.



##########
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:
   I'm +1 for all user-facing time measurements being exposed as `Duration` but 
I think for all our other stores we convert the user-passed `Duration` into 
`long` (millis) at the outer level and only work with millis for inner stores, 
to avoid needing to call `toMillis()` everywhere?



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