mjsax commented on code in PR #14626:
URL: https://github.com/apache/kafka/pull/14626#discussion_r1414568376


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/LogicalSegmentIterator.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.utils.Bytes;
+import org.apache.kafka.streams.query.ResultOrder;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.VersionedRecordIterator;
+import org.rocksdb.Snapshot;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+
+public class LogicalSegmentIterator implements VersionedRecordIterator {
+    protected final ListIterator<LogicalKeyValueSegment> segmentIterator;
+    private final Bytes key;
+    private final Long fromTime;
+    private final Long toTime;
+    private final ResultOrder order;
+    private ListIterator<VersionedRecord<byte[]>> iterator;
+    private volatile boolean open = true;
+
+    // defined for creating/releasing the snapshot. 
+    private LogicalKeyValueSegment snapshotOwner = null;
+    private Snapshot snapshot = null;
+
+
+
+    public LogicalSegmentIterator(final ListIterator<LogicalKeyValueSegment> 
segmentIterator,
+                                  final Bytes key,
+                                  final Long fromTime,
+                                  final Long toTime,
+                                  final ResultOrder order) {
+
+        this.segmentIterator = segmentIterator;
+        this.key = key;
+        this.fromTime = fromTime;
+        this.toTime = toTime;
+        this.iterator = Collections.emptyListIterator();
+        this.order = order;
+    }
+
+    @Override
+    public void close() {
+        open = false;
+        // user may refuse consuming all returned records, so release the 
snapshot when closing the iterator if it is not released yet!
+        releaseSnapshot();
+    }
+
+    @Override
+    public boolean hasNext() {
+        if (!open) {
+            throw new IllegalStateException("The iterator is out of scope.");
+        }
+        final boolean hasStillLoad = order.equals(ResultOrder.ASCENDING) ? 
iterator.hasPrevious() : iterator.hasNext();

Review Comment:
   We might want to add a comment about why `ASCENDING` is using 
`hasPrevious()` -- it's not intuitive and only makes sense if one knows that 
data is storing descending inside a segement.



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