patrickstuedi commented on a change in pull request #11227: URL: https://github.com/apache/kafka/pull/11227#discussion_r695631238
########## File path: streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java ########## @@ -74,7 +74,7 @@ public boolean hasNext() { close(); currentSegment = segments.next(); try { - if (from == null || to == null) { + if (from == null && to == null) { if (forward) { currentIterator = currentSegment.all(); Review comment: Isn't it sufficient to distinguish the forward and reverse cases and just call range(from, to) or reverseRange(from, to)? ########## File path: streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java ########## @@ -124,8 +124,8 @@ final List<S> searchSpace = keySchema.segmentsToSearch(segments, from, to, forward); - final Bytes binaryFrom = keySchema.lowerRange(keyFrom, from); - final Bytes binaryTo = keySchema.upperRange(keyTo, to); + final Bytes binaryFrom = keyFrom == null ? null : keySchema.lowerRange(keyFrom, from); + final Bytes binaryTo = keyTo == null ? null : keySchema.upperRange(keyTo, to); Review comment: Do you have tests for this store? Generally can you make sure that your tests cover all the stores you added infinite range support. -- 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