From 900639bf632cb5dd9ca3568f312a3e6159014db1 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 5 Aug 2020 18:47:37 +0100 Subject: [PATCH 01/29] key/value reverse operations --- .../kafka/streams/state/internals/RocksDBTimestampedStore.java | 1 - 1 file changed, 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java index e16577513395e..aa65e7f971317 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java @@ -431,5 +431,4 @@ public KeyValue makeNext() { } } } - } From 37c340e81494e396fbf7d052654c3bd3f9aff580 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Sat, 8 Aug 2020 10:19:29 +0100 Subject: [PATCH 02/29] improve ordering docs --- .../internals/AbstractMergedSortedCacheStoreIterator.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java index a5e56aa0d9ac0..40ab65dafccc3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java @@ -191,5 +191,4 @@ public void close() { cacheIterator.close(); storeIterator.close(); } -} - +} \ No newline at end of file From 92dcec1b20248ef0647323471892f711ad5c7b71 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 11 Aug 2020 13:47:57 +0100 Subject: [PATCH 03/29] fix range validator not needed --- .../kafka/streams/state/internals/InMemoryKeyValueStore.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java index b02459dc57a67..adac4a4729557 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java @@ -196,4 +196,4 @@ public Bytes peekNextKey() { throw new UnsupportedOperationException("peekNextKey() not supported in " + getClass().getName()); } } -} +} \ No newline at end of file From 26048cf65d3f8f5552d774e3c735233c76462a90 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 12 Aug 2020 15:46:03 +0100 Subject: [PATCH 04/29] improve tests --- .../internals/AbstractKeyValueStoreTest.java | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java index 61f317d495416..b68622f2dc654 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java @@ -556,4 +556,36 @@ public void shouldNotThrowInvalidReverseRangeExceptionWithFromLargerThanTo() { ); } } + + @Test + public void shouldNotThrowInvalidRangeExceptionWithFromLargerThanTo() { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister()) { + final KeyValueIterator iterator = store.range(2, 1); + assertFalse(iterator.hasNext()); + + final List messages = appender.getMessages(); + assertThat( + messages, + hasItem("Returning empty iterator for fetch with invalid key range: from > to." + + " This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes." + + " Note that the built-in numerical serdes do not follow this for negative numbers") + ); + } + } + + @Test + public void shouldNotThrowInvalidReverseRangeExceptionWithFromLargerThanTo() { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister()) { + final KeyValueIterator iterator = store.reverseRange(2, 1); + assertFalse(iterator.hasNext()); + + final List messages = appender.getMessages(); + assertThat( + messages, + hasItem("Returning empty iterator for fetch with invalid key range: from > to." + + " This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes." + + " Note that the built-in numerical serdes do not follow this for negative numbers") + ); + } + } } From 07530e91e9913e6ecc0fe9995a25594a4fbb3a26 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 20 Aug 2020 10:28:13 +0100 Subject: [PATCH 05/29] improve range wrong order warning --- .../internals/AbstractKeyValueStoreTest.java | 32 ------------------- 1 file changed, 32 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java index b68622f2dc654..61f317d495416 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java @@ -556,36 +556,4 @@ public void shouldNotThrowInvalidReverseRangeExceptionWithFromLargerThanTo() { ); } } - - @Test - public void shouldNotThrowInvalidRangeExceptionWithFromLargerThanTo() { - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister()) { - final KeyValueIterator iterator = store.range(2, 1); - assertFalse(iterator.hasNext()); - - final List messages = appender.getMessages(); - assertThat( - messages, - hasItem("Returning empty iterator for fetch with invalid key range: from > to." + - " This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes." + - " Note that the built-in numerical serdes do not follow this for negative numbers") - ); - } - } - - @Test - public void shouldNotThrowInvalidReverseRangeExceptionWithFromLargerThanTo() { - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister()) { - final KeyValueIterator iterator = store.reverseRange(2, 1); - assertFalse(iterator.hasNext()); - - final List messages = appender.getMessages(); - assertThat( - messages, - hasItem("Returning empty iterator for fetch with invalid key range: from > to." + - " This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes." + - " Note that the built-in numerical serdes do not follow this for negative numbers") - ); - } - } } From a733d3fe6056c44e11c6e7a5a0b40873f570aca3 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 5 Aug 2020 18:25:59 +0100 Subject: [PATCH 06/29] key/value reverse operation --- .../internals/AbstractReadOnlyDecorator.java | 53 +++++++++++++++++++ .../state/internals/KeyValueSegmentsTest.java | 41 ++++++++++++-- 2 files changed, 91 insertions(+), 3 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java index 7c622fc0146e1..466877b9adebc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.time.Instant; import java.util.List; import org.apache.kafka.streams.KeyValue; @@ -180,6 +181,13 @@ public WindowStoreIterator fetch(final K key, return wrapped().fetch(key, timeFrom, timeTo); } + @Override + public WindowStoreIterator backwardFetch(final K key, + final Instant timeFrom, + final Instant timeTo) { + return wrapped().backwardFetch(key, timeFrom, timeTo); + } + @Override @Deprecated public KeyValueIterator, V> fetch(final K from, @@ -189,17 +197,36 @@ public KeyValueIterator, V> fetch(final K from, return wrapped().fetch(from, to, timeFrom, timeTo); } + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final Instant timeFrom, + final Instant timeTo) { + return wrapped().backwardFetch(from, to, timeFrom, timeTo); + } + @Override public KeyValueIterator, V> all() { return wrapped().all(); } + @Override + public KeyValueIterator, V> backwardAll() { + return wrapped().backwardAll(); + } + @Override @Deprecated public KeyValueIterator, V> fetchAll(final long timeFrom, final long timeTo) { return wrapped().fetchAll(timeFrom, timeTo); } + + @Override + public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) { + return wrapped().backwardFetchAll(timeFrom, timeTo); + } } static class TimestampedWindowStoreReadOnlyDecorator @@ -226,6 +253,13 @@ public KeyValueIterator, AGG> findSessions(final K key, return wrapped().findSessions(key, earliestSessionEndTime, latestSessionStartTime); } + @Override + public KeyValueIterator, AGG> backwardFindSessions(final K key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + return wrapped().backwardFindSessions(key, earliestSessionEndTime, latestSessionStartTime); + } + @Override public KeyValueIterator, AGG> findSessions(final K keyFrom, final K keyTo, @@ -234,6 +268,14 @@ public KeyValueIterator, AGG> findSessions(final K keyFrom, return wrapped().findSessions(keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime); } + @Override + public KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, + final K keyTo, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + return wrapped().backwardFindSessions(keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime); + } + @Override public void remove(final Windowed sessionKey) { throw new UnsupportedOperationException(ERROR_MESSAGE); @@ -255,10 +297,21 @@ public KeyValueIterator, AGG> fetch(final K key) { return wrapped().fetch(key); } + @Override + public KeyValueIterator, AGG> backwardFetch(final K key) { + return wrapped().backwardFetch(key); + } + @Override public KeyValueIterator, AGG> fetch(final K from, final K to) { return wrapped().fetch(from, to); } + + @Override + public KeyValueIterator, AGG> backwardFetch(final K from, + final K to) { + return wrapped().backwardFetch(from, to); + } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java index ff67b78478c9d..442723d0c7459 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java @@ -187,13 +187,33 @@ public void shouldGetSegmentsWithinTimeRange() { segments.getOrCreateSegmentIfLive(3, context, streamTime); segments.getOrCreateSegmentIfLive(4, context, streamTime); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); assertEquals(2, segments.get(2).id); } + @Test + public void shouldGetSegmentsWithinBackwardTimeRange() { + updateStreamTimeAndCreateSegment(0); + updateStreamTimeAndCreateSegment(1); + updateStreamTimeAndCreateSegment(2); + updateStreamTimeAndCreateSegment(3); + final long streamTime = updateStreamTimeAndCreateSegment(4); + segments.getOrCreateSegmentIfLive(0, context, streamTime); + segments.getOrCreateSegmentIfLive(1, context, streamTime); + segments.getOrCreateSegmentIfLive(2, context, streamTime); + segments.getOrCreateSegmentIfLive(3, context, streamTime); + segments.getOrCreateSegmentIfLive(4, context, streamTime); + + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); + assertEquals(3, segments.size()); + assertEquals(0, segments.get(2).id); + assertEquals(1, segments.get(1).id); + assertEquals(2, segments.get(0).id); + } + @Test public void shouldGetSegmentsWithinTimeRangeOutOfOrder() { updateStreamTimeAndCreateSegment(4); @@ -202,13 +222,28 @@ public void shouldGetSegmentsWithinTimeRangeOutOfOrder() { updateStreamTimeAndCreateSegment(1); updateStreamTimeAndCreateSegment(3); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); assertEquals(2, segments.get(2).id); } + @Test + public void shouldGetSegmentsWithinTimeBackwardRangeOutOfOrder() { + updateStreamTimeAndCreateSegment(4); + updateStreamTimeAndCreateSegment(2); + updateStreamTimeAndCreateSegment(0); + updateStreamTimeAndCreateSegment(1); + updateStreamTimeAndCreateSegment(3); + + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); + assertEquals(3, segments.size()); + assertEquals(2, segments.get(0).id); + assertEquals(1, segments.get(1).id); + assertEquals(0, segments.get(2).id); + } + @Test public void shouldRollSegments() { updateStreamTimeAndCreateSegment(0); @@ -309,7 +344,7 @@ public void shouldClearSegmentsOnClose() { } private void verifyCorrectSegments(final long first, final int numSegments) { - final List result = this.segments.segments(0, Long.MAX_VALUE); + final List result = this.segments.segments(0, Long.MAX_VALUE, false); assertEquals(numSegments, result.size()); for (int i = 0; i < numSegments; i++) { assertEquals(i + first, result.get(i).id); From 843980ba4eb35e67409f759f8af3be3991e8e4c1 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 5 Aug 2020 18:33:56 +0100 Subject: [PATCH 07/29] window backward operations --- .../streams/state/ReadOnlyWindowStore.java | 150 ++++++++--- .../kafka/streams/state/WindowStore.java | 35 +-- .../AbstractRocksDBSegmentedBytesStore.java | 77 +++++- .../state/internals/AbstractSegments.java | 17 +- .../state/internals/CachingWindowStore.java | 175 +++++++++++-- .../ChangeLoggingWindowBytesStore.java | 29 ++ .../CompositeReadOnlyWindowStore.java | 70 ++++- .../state/internals/InMemoryWindowStore.java | 90 +++++-- .../MergedSortedCacheWindowStoreIterator.java | 5 +- ...ortedCacheWindowStoreKeyValueIterator.java | 5 +- .../state/internals/MeteredWindowStore.java | 44 ++++ .../internals/ReadOnlyWindowStoreFacade.java | 28 ++ .../state/internals/RocksDBWindowStore.java | 38 +++ .../state/internals/SegmentIterator.java | 13 +- .../state/internals/SegmentedBytesStore.java | 37 ++- .../streams/state/internals/Segments.java | 4 +- .../TimestampedWindowStoreBuilder.java | 37 ++- .../state/internals/WindowKeySchema.java | 9 +- ...owToTimestampedWindowByteStoreAdapter.java | 26 ++ .../AbstractWindowBytesStoreTest.java | 196 +++++++++++--- .../internals/CachingWindowStoreTest.java | 203 +++++++++++++- .../ChangeLoggingWindowBytesStoreTest.java | 26 ++ .../CompositeReadOnlyWindowStoreTest.java | 247 +++++++++++++++--- .../internals/InMemoryWindowStoreTest.java | 25 +- ...edCacheWrappedWindowStoreIteratorTest.java | 86 +++++- ...rappedWindowStoreKeyValueIteratorTest.java | 70 ++++- .../internals/MeteredWindowStoreTest.java | 11 +- .../internals/ReadOnlyWindowStoreStub.java | 181 ++++++++++++- .../state/internals/SegmentIteratorTest.java | 78 +++++- .../internals/TimestampedSegmentsTest.java | 41 ++- .../state/internals/WindowKeySchemaTest.java | 64 ++--- .../kafka/test/SegmentedBytesStoreStub.java | 112 -------- .../apache/kafka/test/StreamsTestUtils.java | 3 +- 33 files changed, 1829 insertions(+), 403 deletions(-) delete mode 100644 streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java index 0c43d39e3b7da..9c44123866dd9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java @@ -33,11 +33,11 @@ public interface ReadOnlyWindowStore { /** * Get the value of key from a window. * - * @param key the key to fetch - * @param time start timestamp (inclusive) of the window + * @param key the key to fetch + * @param time start timestamp (inclusive) of the window * @return The value or {@code null} if no value is found in the window * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for any key. + * @throws NullPointerException If {@code null} is used for any key. */ V fetch(K key, long time); @@ -67,12 +67,12 @@ public interface ReadOnlyWindowStore { * For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest * available window to the newest/latest window. * - * @param key the key to fetch - * @param timeFrom time range start (inclusive) - * @param timeTo time range end (inclusive) + * @param key the key to fetch + * @param timeFrom time range start (inclusive) + * @param timeTo time range end (inclusive) * @return an iterator over key-value pairs {@code } * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for key. + * @throws NullPointerException If {@code null} is used for key. * @deprecated Use {@link #fetch(Object, Instant, Instant)} instead */ @Deprecated @@ -104,28 +104,68 @@ public interface ReadOnlyWindowStore { * For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest * available window to the newest/latest window. * - * @param key the key to fetch - * @param from time range start (inclusive) - * @param to time range end (inclusive) + * @param key the key to fetch + * @param from time range start (inclusive) + * @param to time range end (inclusive) * @return an iterator over key-value pairs {@code } * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for key. - * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + * @throws NullPointerException If {@code null} is used for key. + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ WindowStoreIterator fetch(K key, Instant from, Instant to) throws IllegalArgumentException; + /** + * Get all the key-value pairs with the given key and the time range from all the existing windows + * in backward order with respect to time (from end to beginning of time). + *

+ * This iterator must be closed after use. + *

+ * The time range is inclusive and applies to the starting timestamp of the window. + * For example, if we have the following windows: + *

+     * +-------------------------------+
+     * |  key  | start time | end time |
+     * +-------+------------+----------+
+     * |   A   |     10     |    20    |
+     * +-------+------------+----------+
+     * |   A   |     15     |    25    |
+     * +-------+------------+----------+
+     * |   A   |     20     |    30    |
+     * +-------+------------+----------+
+     * |   A   |     25     |    35    |
+     * +--------------------------------
+     * 
+ * And we call {@code store.backwardFetch("A", Instant.ofEpochMilli(10), Instant.ofEpochMilli(20))} then the + * results will contain the first three windows from the table above in backward order, + * i.e., all those where 10 <= start time <= 20. + *

+ * For each key, the iterator guarantees ordering of windows, starting from the newest/latest + * available window to the oldest/earliest window. + * + * @param key the key to fetch + * @param from time range start (inclusive) + * @param to time range end (inclusive) + * @return an iterator over key-value pairs {@code } + * @throws InvalidStateStoreException if the store is not initialized + * @throws NullPointerException If {@code null} is used for key. + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + */ + default WindowStoreIterator backwardFetch(K key, Instant from, Instant to) throws IllegalArgumentException { + throw new UnsupportedOperationException(); + } + /** * Get all the key-value pairs in the given key range and time range from all the existing windows. *

* This iterator must be closed after use. * - * @param from the first key in the range - * @param to the last key in the range - * @param timeFrom time range start (inclusive) - * @param timeTo time range end (inclusive) + * @param from the first key in the range + * @param to the last key in the range + * @param timeFrom time range start (inclusive) + * @param timeTo time range end (inclusive) * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for any key. + * @throws NullPointerException If {@code null} is used for any key. * @deprecated Use {@link #fetch(Object, Object, Instant, Instant)} instead */ @Deprecated @@ -136,26 +176,57 @@ public interface ReadOnlyWindowStore { *

* This iterator must be closed after use. * - * @param from the first key in the range - * @param to the last key in the range - * @param fromTime time range start (inclusive) - * @param toTime time range end (inclusive) + * @param from the first key in the range + * @param to the last key in the range + * @param fromTime time range start (inclusive) + * @param toTime time range end (inclusive) * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for any key. - * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + * @throws NullPointerException If {@code null} is used for any key. + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ KeyValueIterator, V> fetch(K from, K to, Instant fromTime, Instant toTime) throws IllegalArgumentException; /** - * Gets all the key-value pairs in the existing windows. - * - * @return an iterator over windowed key-value pairs {@code , value>} - * @throws InvalidStateStoreException if the store is not initialized - */ + * Get all the key-value pairs in the given key range and time range from all the existing windows + * in backward order with respect to time (from end to beginning of time). + *

+ * This iterator must be closed after use. + * + * @param from the first key in the range + * @param to the last key in the range + * @param fromTime time range start (inclusive) + * @param toTime time range end (inclusive) + * @return an iterator over windowed key-value pairs {@code , value>} + * @throws InvalidStateStoreException if the store is not initialized + * @throws NullPointerException If {@code null} is used for any key. + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + */ + default KeyValueIterator, V> backwardFetch(K from, K to, Instant fromTime, Instant toTime) + throws IllegalArgumentException { + throw new UnsupportedOperationException(); + } + + /** + * Gets all the key-value pairs in the existing windows. + * + * @return an iterator over windowed key-value pairs {@code , value>} + * @throws InvalidStateStoreException if the store is not initialized + */ KeyValueIterator, V> all(); - + + /** + * Gets all the key-value pairs in the existing windows in backward order + * with respect to time (from end to beginning of time). + * + * @return an iterator over windowed key-value pairs {@code , value>} + * @throws InvalidStateStoreException if the store is not initialized + */ + default KeyValueIterator, V> backwardAll() { + throw new UnsupportedOperationException(); + } + /** * Gets all the key-value pairs that belong to the windows within in the given time range. * @@ -163,7 +234,7 @@ KeyValueIterator, V> fetch(K from, K to, Instant fromTime, Instant t * @param timeTo the end of the time slot from which to search (inclusive) * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException if {@code null} is used for any key + * @throws NullPointerException if {@code null} is used for any key * @deprecated Use {@link #fetchAll(Instant, Instant)} instead */ @Deprecated @@ -176,8 +247,23 @@ KeyValueIterator, V> fetch(K from, K to, Instant fromTime, Instant t * @param to the end of the time slot from which to search (inclusive) * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException if {@code null} is used for any key - * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + * @throws NullPointerException if {@code null} is used for any key + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ KeyValueIterator, V> fetchAll(Instant from, Instant to) throws IllegalArgumentException; + + /** + * Gets all the key-value pairs that belong to the windows within in the given time range in backward order + * with respect to time (from end to beginning of time). + * + * @param from the beginning of the time slot from which to search (inclusive) + * @param to the end of the time slot from which to search (inclusive) + * @return an iterator over windowed key-value pairs {@code , value>} + * @throws InvalidStateStoreException if the store is not initialized + * @throws NullPointerException if {@code null} is used for any key + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + */ + default KeyValueIterator, V> backwardFetchAll(Instant from, Instant to) throws IllegalArgumentException { + throw new UnsupportedOperationException(); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java index f8c93e2c31dca..e9f1db0507c2b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java @@ -42,15 +42,13 @@ public interface WindowStore extends StateStore, ReadOnlyWindowStore * It's highly recommended to use {@link WindowStore#put(Object, Object, long)} instead, as the record timestamp * is unlikely to be the correct windowStartTimestamp in general. * - * @param key The key to associate the value to + * @param key The key to associate the value to * @param value The value to update, it can be null; * if the serialized bytes are also null it is interpreted as delete * @throws NullPointerException if the given key is {@code null} - * * @deprecated as timestamp is not provided for the key-value pair, this causes inconsistency * to identify the window frame to which the key belongs. * Use {@link #put(Object, Object, long)} instead. - * */ @Deprecated void put(K key, V value); @@ -61,8 +59,8 @@ public interface WindowStore extends StateStore, ReadOnlyWindowStore * If serialized value bytes are null it is interpreted as delete. Note that deletes will be * ignored in the case of an underlying store that retains duplicates. * - * @param key The key to associate the value to - * @param value The value; can be null + * @param key The key to associate the value to + * @param value The value; can be null * @param windowStartTimestamp The timestamp of the beginning of the window to put the key/value into * @throws NullPointerException if the given key is {@code null} */ @@ -94,14 +92,15 @@ public interface WindowStore extends StateStore, ReadOnlyWindowStore * For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest * available window to the newest/latest window. * - * @param key the key to fetch - * @param timeFrom time range start (inclusive) - * @param timeTo time range end (inclusive) + * @param key the key to fetch + * @param timeFrom time range start (inclusive) + * @param timeTo time range end (inclusive) * @return an iterator over key-value pairs {@code } * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException if the given key is {@code null} + * @throws NullPointerException if the given key is {@code null} */ - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed + // note, this method must be kept if super#fetch(...) is removed + @SuppressWarnings("deprecation") WindowStoreIterator fetch(K key, long timeFrom, long timeTo); @Override @@ -119,15 +118,16 @@ default WindowStoreIterator fetch(final K key, *

* This iterator must be closed after use. * - * @param from the first key in the range - * @param to the last key in the range - * @param timeFrom time range start (inclusive) - * @param timeTo time range end (inclusive) + * @param from the first key in the range + * @param to the last key in the range + * @param timeFrom time range start (inclusive) + * @param timeTo time range end (inclusive) * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException if one of the given keys is {@code null} + * @throws NullPointerException if one of the given keys is {@code null} */ - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed + // note, this method must be kept if super#fetch(...) is removed + @SuppressWarnings("deprecation") KeyValueIterator, V> fetch(K from, K to, long timeFrom, long timeTo); @Override @@ -150,7 +150,8 @@ default KeyValueIterator, V> fetch(final K from, * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized */ - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed + // note, this method must be kept if super#fetchAll(...) is removed + @SuppressWarnings("deprecation") KeyValueIterator, V> fetchAll(long timeFrom, long timeTo); @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java index 39335967aaef2..28107dc0e97e6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java @@ -63,7 +63,21 @@ public class AbstractRocksDBSegmentedBytesStore implements Se public KeyValueIterator fetch(final Bytes key, final long from, final long to) { - final List searchSpace = keySchema.segmentsToSearch(segments, from, to); + return fetch(key, from, to, false); + } + + @Override + public KeyValueIterator backwardFetch(final Bytes key, + final long from, + final long to) { + return fetch(key, from, to, true); + } + + KeyValueIterator fetch(final Bytes key, + final long from, + final long to, + final boolean backward) { + final List searchSpace = keySchema.segmentsToSearch(segments, from, to, backward); final Bytes binaryFrom = keySchema.lowerRangeFixedSize(key, from); final Bytes binaryTo = keySchema.upperRangeFixedSize(key, to); @@ -72,7 +86,8 @@ public KeyValueIterator fetch(final Bytes key, searchSpace.iterator(), keySchema.hasNextCondition(key, key, from, to), binaryFrom, - binaryTo); + binaryTo, + backward); } @Override @@ -80,6 +95,22 @@ public KeyValueIterator fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) { + return fetch(keyFrom, keyTo, from, to, false); + } + + @Override + public KeyValueIterator backwardFetch(final Bytes keyFrom, + final Bytes keyTo, + final long from, + final long to) { + return fetch(keyFrom, keyTo, from, to, true); + } + + KeyValueIterator fetch(final Bytes keyFrom, + final Bytes keyTo, + final long from, + final long to, + final boolean backward) { if (keyFrom.compareTo(keyTo) > 0) { LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + "This may be due to range arguments set in the wrong order, " + @@ -88,7 +119,7 @@ public KeyValueIterator fetch(final Bytes keyFrom, return KeyValueIterators.emptyIterator(); } - final List searchSpace = keySchema.segmentsToSearch(segments, from, to); + final List searchSpace = keySchema.segmentsToSearch(segments, from, to, backward); final Bytes binaryFrom = keySchema.lowerRange(keyFrom, from); final Bytes binaryTo = keySchema.upperRange(keyTo, to); @@ -97,30 +128,58 @@ public KeyValueIterator fetch(final Bytes keyFrom, searchSpace.iterator(), keySchema.hasNextCondition(keyFrom, keyTo, from, to), binaryFrom, - binaryTo); + binaryTo, + backward); } @Override public KeyValueIterator all() { - final List searchSpace = segments.allSegments(); + final List searchSpace = segments.allSegments(false); + + return new SegmentIterator<>( + searchSpace.iterator(), + keySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE), + null, + null, + false); + } + + @Override + public KeyValueIterator backwardAll() { + final List searchSpace = segments.allSegments(true); return new SegmentIterator<>( searchSpace.iterator(), keySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE), null, - null); + null, + true); } @Override public KeyValueIterator fetchAll(final long timeFrom, final long timeTo) { - final List searchSpace = segments.segments(timeFrom, timeTo); + final List searchSpace = segments.segments(timeFrom, timeTo, false); return new SegmentIterator<>( searchSpace.iterator(), keySchema.hasNextCondition(null, null, timeFrom, timeTo), null, - null); + null, + false); + } + + @Override + public KeyValueIterator backwardFetchAll(final long timeFrom, + final long timeTo) { + final List searchSpace = segments.segments(timeFrom, timeTo, true); + + return new SegmentIterator<>( + searchSpace.iterator(), + keySchema.hasNextCondition(null, null, timeFrom, timeTo), + null, + null, + true); } @Override @@ -211,7 +270,7 @@ public boolean isOpen() { // Visible for testing List getSegments() { - return segments.allSegments(); + return segments.allSegments(false); } // Visible for testing diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java index 044a57448502b..666a773374f0d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java @@ -27,6 +27,7 @@ import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -117,9 +118,14 @@ public void openExisting(final ProcessorContext context, final long streamTime) } @Override - public List segments(final long timeFrom, final long timeTo) { + public List segments(final long timeFrom, final long timeTo, final boolean backward) { final List result = new ArrayList<>(); - final NavigableMap segmentsInRange = segments.subMap( + final NavigableMap segmentsInRange; + if (backward) segmentsInRange = segments.subMap( + segmentId(timeFrom), true, + segmentId(timeTo), true + ).descendingMap(); + else segmentsInRange = segments.subMap( segmentId(timeFrom), true, segmentId(timeTo), true ); @@ -132,9 +138,12 @@ public List segments(final long timeFrom, final long timeTo) { } @Override - public List allSegments() { + public List allSegments(final boolean backward) { final List result = new ArrayList<>(); - for (final S segment : segments.values()) { + final Collection values; + if (backward) values = segments.descendingMap().values(); + else values = segments.values(); + for (final S segment : values) { if (segment.isOpen()) { result.add(segment); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index 8867602c45771..cde0a2e55411f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; @@ -33,10 +34,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Instant; import java.util.LinkedList; import java.util.NoSuchElementException; import java.util.concurrent.atomic.AtomicLong; +import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; import static org.apache.kafka.streams.state.internals.ExceptionUtils.executeAll; import static org.apache.kafka.streams.state.internals.ExceptionUtils.throwSuppressed; @@ -153,7 +156,7 @@ public synchronized void put(final Bytes key, // since this function may not access the underlying inner store, we need to validate // if store is open outside as well. validateStoreOpen(); - + final Bytes keyBytes = WindowKeySchema.toStoreKeyBinary(key, windowStartTimestamp, 0); final LRUCacheEntry entry = new LRUCacheEntry( @@ -201,10 +204,11 @@ public synchronized WindowStoreIterator fetch(final Bytes key, } final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? - new CacheIteratorWrapper(key, timeFrom, timeTo) : - context.cache().range(cacheName, - cacheFunction.cacheKey(keySchema.lowerRangeFixedSize(key, timeFrom)), - cacheFunction.cacheKey(keySchema.upperRangeFixedSize(key, timeTo)) + new CacheIteratorWrapper(key, timeFrom, timeTo, false) : + context.cache().range( + cacheName, + cacheFunction.cacheKey(keySchema.lowerRangeFixedSize(key, timeFrom)), + cacheFunction.cacheKey(keySchema.upperRangeFixedSize(key, timeTo)) ); final HasNextCondition hasNextCondition = keySchema.hasNextCondition(key, key, timeFrom, timeTo); @@ -212,7 +216,37 @@ public synchronized WindowStoreIterator fetch(final Bytes key, cacheIterator, hasNextCondition, cacheFunction ); - return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator); + return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator, false); + } + + @Override + public synchronized WindowStoreIterator backwardFetch(final Bytes key, + final Instant from, + final Instant to) { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + // since this function may not access the underlying inner store, we need to validate + // if store is open outside as well. + validateStoreOpen(); + + final WindowStoreIterator underlyingIterator = wrapped().backwardFetch(key, from, to); + if (context.cache() == null) { + return underlyingIterator; + } + + final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? + new CacheIteratorWrapper(key, timeFrom, timeTo, true) : + context.cache().reverseRange( + cacheName, + cacheFunction.cacheKey(keySchema.lowerRangeFixedSize(key, timeFrom)), + cacheFunction.cacheKey(keySchema.upperRangeFixedSize(key, timeTo)) + ); + + final HasNextCondition hasNextCondition = keySchema.hasNextCondition(key, key, timeFrom, timeTo); + final PeekingKeyValueIterator filteredCacheIterator = + new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); + + return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator, true); } @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @@ -240,10 +274,57 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, } final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? - new CacheIteratorWrapper(from, to, timeFrom, timeTo) : - context.cache().range(cacheName, - cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), - cacheFunction.cacheKey(keySchema.upperRange(to, timeTo)) + new CacheIteratorWrapper(from, to, timeFrom, timeTo, false) : + context.cache().range( + cacheName, + cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), + cacheFunction.cacheKey(keySchema.upperRange(to, timeTo)) + ); + + final HasNextCondition hasNextCondition = keySchema.hasNextCondition(from, to, timeFrom, timeTo); + final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); + + return new MergedSortedCacheWindowStoreKeyValueIterator( + filteredCacheIterator, + underlyingIterator, + bytesSerdes, + windowSize, + cacheFunction, + false + ); + } + + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, + final Bytes to, + final Instant fromTime, + final Instant toTime) { + if (from.compareTo(to) > 0) { + LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + + "Note that the built-in numerical serdes do not follow this for negative numbers"); + return KeyValueIterators.emptyIterator(); + } + + final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")); + final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime")); + + // since this function may not access the underlying inner store, we need to validate + // if store is open outside as well. + validateStoreOpen(); + + final KeyValueIterator, byte[]> underlyingIterator = + wrapped().backwardFetch(from, to, fromTime, toTime); + if (context.cache() == null) { + return underlyingIterator; + } + + final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? + new CacheIteratorWrapper(from, to, timeFrom, timeTo, true) : + context.cache().reverseRange( + cacheName, + cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), + cacheFunction.cacheKey(keySchema.upperRange(to, timeTo)) ); final HasNextCondition hasNextCondition = keySchema.hasNextCondition(from, to, timeFrom, timeTo); @@ -254,7 +335,8 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, underlyingIterator, bytesSerdes, windowSize, - cacheFunction + cacheFunction, + true ); } @@ -271,11 +353,36 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); return new MergedSortedCacheWindowStoreKeyValueIterator( - filteredCacheIterator, - underlyingIterator, - bytesSerdes, - windowSize, - cacheFunction + filteredCacheIterator, + underlyingIterator, + bytesSerdes, + windowSize, + cacheFunction, + false + ); + } + + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, + final Instant to) { + validateStoreOpen(); + + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + + final KeyValueIterator, byte[]> underlyingIterator = wrapped().backwardFetchAll(from, to); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = context.cache().reverseAll(cacheName); + + final HasNextCondition hasNextCondition = keySchema.hasNextCondition(null, null, timeFrom, timeTo); + final PeekingKeyValueIterator filteredCacheIterator = + new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); + return new MergedSortedCacheWindowStoreKeyValueIterator( + filteredCacheIterator, + underlyingIterator, + bytesSerdes, + windowSize, + cacheFunction, + true ); } @@ -283,7 +390,7 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, public KeyValueIterator, byte[]> all() { validateStoreOpen(); - final KeyValueIterator, byte[]> underlyingIterator = wrapped().all(); + final KeyValueIterator, byte[]> underlyingIterator = wrapped().all(); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = context.cache().all(cacheName); return new MergedSortedCacheWindowStoreKeyValueIterator( @@ -291,7 +398,25 @@ public KeyValueIterator, byte[]> all() { underlyingIterator, bytesSerdes, windowSize, - cacheFunction + cacheFunction, + false + ); + } + + @Override + public KeyValueIterator, byte[]> backwardAll() { + validateStoreOpen(); + + final KeyValueIterator, byte[]> underlyingIterator = wrapped().backwardAll(); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = context.cache().reverseAll(cacheName); + + return new MergedSortedCacheWindowStoreKeyValueIterator( + cacheIterator, + underlyingIterator, + bytesSerdes, + windowSize, + cacheFunction, + true ); } @@ -315,12 +440,11 @@ public synchronized void close() { ); if (!suppressed.isEmpty()) { throwSuppressed("Caught an exception while closing caching window store for store " + name(), - suppressed); + suppressed); } } - private class CacheIteratorWrapper implements PeekingKeyValueIterator { private final long segmentInterval; @@ -338,14 +462,16 @@ private class CacheIteratorWrapper implements PeekingKeyValueIterator fetch(final Bytes key, return wrapped().fetch(key, from, to); } + @Override + public WindowStoreIterator backwardFetch(final Bytes key, + final Instant from, + final Instant to) { + return wrapped().backwardFetch(key, from, to); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, @@ -78,11 +87,25 @@ public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, return wrapped().fetch(keyFrom, keyTo, from, to); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, + final Bytes keyTo, + final Instant from, + final Instant to) { + return wrapped().backwardFetch(keyFrom, keyTo, from, to); + } + @Override public KeyValueIterator, byte[]> all() { return wrapped().all(); } + + @Override + public KeyValueIterator, byte[]> backwardAll() { + return wrapped().backwardAll(); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, @@ -90,6 +113,12 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, return wrapped().fetchAll(timeFrom, timeTo); } + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) { + return wrapped().backwardFetchAll(timeFrom, timeTo); + } + @Deprecated @Override public void put(final Bytes key, final byte[] value) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java index fbfc7a02e9c3f..81368fe9bb8d2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java @@ -60,8 +60,8 @@ public V fetch(final K key, final long time) { } } catch (final InvalidStateStoreException e) { throw new InvalidStateStoreException( - "State store is not available anymore and may have been migrated to another instance; " + - "please re-discover its location from the state metadata."); + "State store is not available anymore and may have been migrated to another instance; " + + "please re-discover its location from the state metadata."); } } return null; @@ -84,8 +84,31 @@ public WindowStoreIterator fetch(final K key, } } catch (final InvalidStateStoreException e) { throw new InvalidStateStoreException( - "State store is not available anymore and may have been migrated to another instance; " + - "please re-discover its location from the state metadata."); + "State store is not available anymore and may have been migrated to another instance; " + + "please re-discover its location from the state metadata."); + } + } + return KeyValueIterators.emptyWindowStoreIterator(); + } + + @Override + public WindowStoreIterator backwardFetch(final K key, + final Instant timeFrom, + final Instant timeTo) { + Objects.requireNonNull(key, "key can't be null"); + final List> stores = provider.stores(storeName, windowStoreType); + for (final ReadOnlyWindowStore windowStore : stores) { + try { + final WindowStoreIterator result = windowStore.backwardFetch(key, timeFrom, timeTo); + if (!result.hasNext()) { + result.close(); + } else { + return result; + } + } catch (final InvalidStateStoreException e) { + throw new InvalidStateStoreException( + "State store is not available anymore and may have been migrated to another instance; " + + "please re-discover its location from the state metadata."); } } return KeyValueIterators.emptyWindowStoreIterator(); @@ -119,6 +142,22 @@ public KeyValueIterator, V> fetch(final K from, nextIteratorFunction)); } + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final Instant timeFrom, + final Instant timeTo) { + Objects.requireNonNull(from, "from can't be null"); + Objects.requireNonNull(to, "to can't be null"); + final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = + store -> store.backwardFetch(from, to, timeFrom, timeTo); + return new DelegatingPeekingKeyValueIterator<>( + storeName, + new CompositeKeyValueIterator<>( + provider.stores(storeName, windowStoreType).iterator(), + nextIteratorFunction)); + } + @Override public KeyValueIterator, V> fetch(final K from, final K to, @@ -142,6 +181,17 @@ public KeyValueIterator, V> all() { nextIteratorFunction)); } + @Override + public KeyValueIterator, V> backwardAll() { + final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = + ReadOnlyWindowStore::backwardAll; + return new DelegatingPeekingKeyValueIterator<>( + storeName, + new CompositeKeyValueIterator<>( + provider.stores(storeName, windowStoreType).iterator(), + nextIteratorFunction)); + } + @Override @Deprecated public KeyValueIterator, V> fetchAll(final long timeFrom, @@ -155,6 +205,18 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, nextIteratorFunction)); } + @Override + public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) { + final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = + store -> store.backwardFetchAll(timeFrom, timeTo); + return new DelegatingPeekingKeyValueIterator<>( + storeName, + new CompositeKeyValueIterator<>( + provider.stores(storeName, windowStoreType).iterator(), + nextIteratorFunction)); + } + @SuppressWarnings("deprecation") // removing fetchAll(long from, long to) will fix this @Override public KeyValueIterator, V> fetchAll(final Instant from, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java index 14d33cb4ea47b..aa6d4007e32fd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.TimeWindow; import org.apache.kafka.streams.processor.ProcessorContext; @@ -34,6 +35,7 @@ import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; +import java.time.Instant; import java.util.Iterator; import java.util.Map; import java.util.NoSuchElementException; @@ -43,6 +45,7 @@ import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; +import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; import static org.apache.kafka.streams.state.internals.WindowKeySchema.extractStoreKeyBytes; import static org.apache.kafka.streams.state.internals.WindowKeySchema.extractStoreTimestamp; @@ -101,9 +104,8 @@ public void init(final ProcessorContext context, final StateStore root) { ); if (root != null) { - context.register(root, (key, value) -> { - put(Bytes.wrap(extractStoreKeyBytes(key)), value, extractStoreTimestamp(key)); - }); + context.register(root, (key, value) -> + put(Bytes.wrap(extractStoreKeyBytes(key)), value, extractStoreTimestamp(key))); } open = true; } @@ -163,7 +165,17 @@ public byte[] fetch(final Bytes key, final long windowStartTimestamp) { @Deprecated @Override public WindowStoreIterator fetch(final Bytes key, final long timeFrom, final long timeTo) { + return fetch(key, timeFrom, timeTo, false); + } + + @Override + public WindowStoreIterator backwardFetch(final Bytes key, final Instant from, final Instant to) { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + return fetch(key, timeFrom, timeTo, true); + } + WindowStoreIterator fetch(final Bytes key, final long timeFrom, final long timeTo, final boolean backward) { Objects.requireNonNull(key, "key cannot be null"); removeExpiredSegments(); @@ -175,7 +187,9 @@ public WindowStoreIterator fetch(final Bytes key, final long timeFrom, f return WrappedInMemoryWindowStoreIterator.emptyIterator(); } - return registerNewWindowStoreIterator( + if (backward) return registerNewWindowStoreIterator( + key, segmentMap.subMap(minTime, true, timeTo, true).descendingMap().entrySet().iterator()); + else return registerNewWindowStoreIterator( key, segmentMap.subMap(minTime, true, timeTo, true).entrySet().iterator()); } @@ -185,6 +199,24 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to, final long timeFrom, final long timeTo) { + return fetch(from, to, timeFrom, timeTo, false); + } + + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, + final Bytes to, + final Instant fromTime, + final Instant toTime) { + final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")); + final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime")); + return fetch(from, to, timeFrom, timeTo, true); + } + + KeyValueIterator, byte[]> fetch(final Bytes from, + final Bytes to, + final long timeFrom, + final long timeTo, + final boolean backward) { Objects.requireNonNull(from, "from key cannot be null"); Objects.requireNonNull(to, "to key cannot be null"); @@ -205,13 +237,26 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, return KeyValueIterators.emptyIterator(); } - return registerNewWindowedKeyValueIterator( + if (backward) return registerNewWindowedKeyValueIterator( + from, to, segmentMap.subMap(minTime, true, timeTo, true).descendingMap().entrySet().iterator()); + else return registerNewWindowedKeyValueIterator( from, to, segmentMap.subMap(minTime, true, timeTo, true).entrySet().iterator()); } @Deprecated @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { + return fetchAll(timeFrom, timeTo, false); + } + + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, final Instant to) { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + return fetchAll(timeFrom, timeTo, true); + } + + KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo, final boolean backward) { removeExpiredSegments(); // add one b/c records expire exactly retentionPeriod ms after created @@ -221,7 +266,9 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, f return KeyValueIterators.emptyIterator(); } - return registerNewWindowedKeyValueIterator( + if (backward) return registerNewWindowedKeyValueIterator( + null, null, segmentMap.subMap(minTime, true, timeTo, true).descendingMap().entrySet().iterator()); + else return registerNewWindowedKeyValueIterator( null, null, segmentMap.subMap(minTime, true, timeTo, true).entrySet().iterator()); } @@ -235,6 +282,16 @@ public KeyValueIterator, byte[]> all() { null, null, segmentMap.tailMap(minTime, false).entrySet().iterator()); } + @Override + public KeyValueIterator, byte[]> backwardAll() { + removeExpiredSegments(); + + final long minTime = observedStreamTime - retentionPeriod; + + return registerNewWindowedKeyValueIterator( + null, null, segmentMap.tailMap(minTime, false).descendingMap().entrySet().iterator()); + } + @Override public boolean persistent() { return false; @@ -286,7 +343,7 @@ private static Bytes wrapForDups(final Bytes key, final int seqnum) { } private static Bytes getKey(final Bytes keyBytes) { - final byte[] bytes = new byte[keyBytes.get().length - SEQNUM_SIZE]; + final byte[] bytes = new byte[keyBytes.get().length - SEQNUM_SIZE]; System.arraycopy(keyBytes.get(), 0, bytes, 0, bytes.length); return Bytes.wrap(bytes); } @@ -311,11 +368,11 @@ private WrappedWindowedKeyValueIterator registerNewWindowedKeyValueIterator(fina final WrappedWindowedKeyValueIterator iterator = new WrappedWindowedKeyValueIterator(from, - to, - segmentIterator, - openIterators::remove, - retainDuplicates, - windowSize); + to, + segmentIterator, + openIterators::remove, + retainDuplicates, + windowSize); openIterators.add(iterator); return iterator; } @@ -327,7 +384,6 @@ interface ClosingCallback { private static abstract class InMemoryWindowStoreIteratorWrapper { - private Iterator>> segmentIterator; private Iterator> recordIterator; private KeyValue next; private long currentTime; @@ -335,6 +391,7 @@ private static abstract class InMemoryWindowStoreIteratorWrapper { private final boolean allKeys; private final Bytes keyFrom; private final Bytes keyTo; + private final Iterator>> segmentIterator; private final boolean retainDuplicates; private final ClosingCallback callback; @@ -420,13 +477,13 @@ Long minTime() { } } - private static class WrappedInMemoryWindowStoreIterator extends InMemoryWindowStoreIteratorWrapper implements WindowStoreIterator { + private static class WrappedInMemoryWindowStoreIterator extends InMemoryWindowStoreIteratorWrapper implements WindowStoreIterator { WrappedInMemoryWindowStoreIterator(final Bytes keyFrom, final Bytes keyTo, final Iterator>> segmentIterator, final ClosingCallback callback, - final boolean retainDuplicates) { + final boolean retainDuplicates) { super(keyFrom, keyTo, segmentIterator, callback, retainDuplicates); } @@ -450,7 +507,8 @@ public KeyValue next() { } public static WrappedInMemoryWindowStoreIterator emptyIterator() { - return new WrappedInMemoryWindowStoreIterator(null, null, null, it -> { }, false); + return new WrappedInMemoryWindowStoreIterator(null, null, null, it -> { + }, false); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java index 7d40dda2bc5ce..46004f57b89d0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java @@ -31,8 +31,9 @@ class MergedSortedCacheWindowStoreIterator extends AbstractMergedSortedCacheStor MergedSortedCacheWindowStoreIterator(final PeekingKeyValueIterator cacheIterator, - final KeyValueIterator storeIterator) { - super(cacheIterator, storeIterator, true); + final KeyValueIterator storeIterator, + final boolean forward) { + super(cacheIterator, storeIterator, forward); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreKeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreKeyValueIterator.java index 36e922fee4bb5..afc6a042fed84 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreKeyValueIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreKeyValueIterator.java @@ -35,9 +35,10 @@ class MergedSortedCacheWindowStoreKeyValueIterator final KeyValueIterator, byte[]> underlyingIterator, final StateSerdes serdes, final long windowSize, - final SegmentedCacheFunction cacheFunction + final SegmentedCacheFunction cacheFunction, + final boolean forward ) { - super(filteredCacheIterator, underlyingIterator, true); + super(filteredCacheIterator, underlyingIterator, forward); this.serdes = serdes; this.windowSize = windowSize; this.cacheFunction = cacheFunction; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index b253f39a4f632..fcd761016d2ee 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -33,6 +33,8 @@ import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics; +import java.time.Instant; + import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; public class MeteredWindowStore @@ -169,6 +171,19 @@ public WindowStoreIterator fetch(final K key, ); } + @Override + public WindowStoreIterator backwardFetch(final K key, + final Instant timeFrom, + final Instant timeTo) { + return new MeteredWindowStoreIterator<>( + wrapped().backwardFetch(keyBytes(key), timeFrom, timeTo), + fetchSensor, + streamsMetrics, + serdes, + time + ); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed @Override public KeyValueIterator, V> fetch(final K from, @@ -183,6 +198,19 @@ public KeyValueIterator, V> fetch(final K from, time); } + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final Instant timeFrom, + final Instant timeTo) { + return new MeteredWindowedKeyValueIterator<>( + wrapped().backwardFetch(keyBytes(from), keyBytes(to), timeFrom, timeTo), + fetchSensor, + streamsMetrics, + serdes, + time); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, V> fetchAll(final long timeFrom, @@ -195,11 +223,27 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, time); } + @Override + public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) { + return new MeteredWindowedKeyValueIterator<>( + wrapped().backwardFetchAll(timeFrom, timeTo), + fetchSensor, + streamsMetrics, + serdes, + time); + } + @Override public KeyValueIterator, V> all() { return new MeteredWindowedKeyValueIterator<>(wrapped().all(), fetchSensor, streamsMetrics, serdes, time); } + @Override + public KeyValueIterator, V> backwardAll() { + return new MeteredWindowedKeyValueIterator<>(wrapped().backwardAll(), fetchSensor, streamsMetrics, serdes, time); + } + @Override public void flush() { maybeMeasureLatency(super::flush, time, flushSensor); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java index 713959db46d37..ff84d4e2da041 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java @@ -56,6 +56,13 @@ public WindowStoreIterator fetch(final K key, return new WindowStoreIteratorFacade<>(inner.fetch(key, from, to)); } + @Override + public WindowStoreIterator backwardFetch(final K key, + final Instant from, + final Instant to) throws IllegalArgumentException { + return new WindowStoreIteratorFacade<>(inner.backwardFetch(key, from, to)); + } + @Override @SuppressWarnings("deprecation") public KeyValueIterator, V> fetch(final K from, @@ -73,6 +80,14 @@ public KeyValueIterator, V> fetch(final K from, return new KeyValueIteratorFacade<>(inner.fetch(from, to, fromTime, toTime)); } + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final Instant fromTime, + final Instant toTime) throws IllegalArgumentException { + return new KeyValueIteratorFacade<>(inner.backwardFetch(from, to, fromTime, toTime)); + } + @Override @SuppressWarnings("deprecation") public KeyValueIterator, V> fetchAll(final long timeFrom, @@ -87,12 +102,25 @@ public KeyValueIterator, V> fetchAll(final Instant from, return new KeyValueIteratorFacade<>(innerIterator); } + @Override + public KeyValueIterator, V> backwardFetchAll(final Instant from, + final Instant to) throws IllegalArgumentException { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.backwardFetchAll(from, to); + return new KeyValueIteratorFacade<>(innerIterator); + } + @Override public KeyValueIterator, V> all() { final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.all(); return new KeyValueIteratorFacade<>(innerIterator); } + @Override + public KeyValueIterator, V> backwardAll() { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.backwardAll(); + return new KeyValueIteratorFacade<>(innerIterator); + } + private static class WindowStoreIteratorFacade implements WindowStoreIterator { final KeyValueIterator> innerIterator; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index 010c09215843a..fbe03fef29fa0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; @@ -24,6 +25,10 @@ import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; +import java.time.Instant; + +import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; + public class RocksDBWindowStore extends WrappedStateStore implements WindowStore { @@ -76,6 +81,14 @@ public WindowStoreIterator fetch(final Bytes key, final long timeFrom, f return new WindowStoreIteratorWrapper(bytesIterator, windowSize).valuesIterator(); } + @Override + public WindowStoreIterator backwardFetch(final Bytes key, final Instant from, final Instant to) { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + final KeyValueIterator bytesIterator = wrapped().backwardFetch(key, timeFrom, timeTo); + return new WindowStoreIteratorWrapper(bytesIterator, windowSize).valuesIterator(); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, byte[]> fetch(final Bytes from, @@ -86,12 +99,29 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, + final Bytes to, + final Instant fromTime, + final Instant toTime) { + final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "to")); + final KeyValueIterator bytesIterator = wrapped().backwardFetch(from, to, timeFrom, timeTo); + return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + } + @Override public KeyValueIterator, byte[]> all() { final KeyValueIterator bytesIterator = wrapped().all(); return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); } + @Override + public KeyValueIterator, byte[]> backwardAll() { + final KeyValueIterator bytesIterator = wrapped().backwardAll(); + return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { @@ -99,6 +129,14 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, f return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); } + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, final Instant to) { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + final KeyValueIterator bytesIterator = wrapped().backwardFetchAll(timeFrom, timeTo); + return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + } + private void maybeUpdateSeqnumForDups() { if (retainDuplicates) { seqnum = (seqnum + 1) & 0x7FFFFFFF; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java index e4f828bca512f..c0fbefbc4a7a7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java @@ -31,6 +31,7 @@ class SegmentIterator implements KeyValueIterator segments; protected final HasNextCondition hasNextCondition; @@ -40,11 +41,13 @@ class SegmentIterator implements KeyValueIterator segments, final HasNextCondition hasNextCondition, final Bytes from, - final Bytes to) { + final Bytes to, + final boolean reverse) { this.segments = segments; this.hasNextCondition = hasNextCondition; this.from = from; this.to = to; + this.reverse = reverse; } @Override @@ -67,14 +70,16 @@ public Bytes peekNextKey() { public boolean hasNext() { boolean hasNext = false; while ((currentIterator == null || !(hasNext = hasNextConditionHasNext()) || !currentSegment.isOpen()) - && segments.hasNext()) { + && segments.hasNext()) { close(); currentSegment = segments.next(); try { if (from == null || to == null) { - currentIterator = currentSegment.all(); + if (reverse) currentIterator = currentSegment.reverseAll(); + else currentIterator = currentSegment.all(); } else { - currentIterator = currentSegment.range(from, to); + if (reverse) currentIterator = currentSegment.reverseRange(from, to); + else currentIterator = currentSegment.range(from, to); } } catch (final InvalidStateStoreException e) { // segment may have been closed so we ignore it. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java index 068dc5f0dc1bb..07082e8ea3631 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java @@ -40,6 +40,16 @@ public interface SegmentedBytesStore extends StateStore { */ KeyValueIterator fetch(final Bytes key, final long from, final long to); + /** + * Fetch all records from the segmented store with the provided key and time range + * from all existing segments in backward order (from latest to earliest) + * @param key the key to match + * @param from earliest time to match + * @param to latest time to match + * @return an iterator over key-value pairs + */ + KeyValueIterator backwardFetch(final Bytes key, final long from, final long to); + /** * Fetch all records from the segmented store in the provided key range and time range * from all existing segments @@ -50,7 +60,18 @@ public interface SegmentedBytesStore extends StateStore { * @return an iterator over key-value pairs */ KeyValueIterator fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to); - + + /** + * Fetch all records from the segmented store in the provided key range and time range + * from all existing segments in backward order (from latest to earliest) + * @param keyFrom The first key that could be in the range + * @param keyTo The last key that could be in the range + * @param from earliest time to match + * @param to latest time to match + * @return an iterator over key-value pairs + */ + KeyValueIterator backwardFetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to); + /** * Gets all the key-value pairs in the existing windows. * @@ -58,7 +79,15 @@ public interface SegmentedBytesStore extends StateStore { * @throws InvalidStateStoreException if the store is not initialized */ KeyValueIterator all(); - + + /** + * Gets all the key-value pairs in the existing windows in backward order (from latest to earliest). + * + * @return an iterator over windowed key-value pairs {@code , value>} + * @throws InvalidStateStoreException if the store is not initialized + */ + KeyValueIterator backwardAll(); + /** * Gets all the key-value pairs that belong to the windows within in the given time range. * @@ -70,6 +99,8 @@ public interface SegmentedBytesStore extends StateStore { */ KeyValueIterator fetchAll(final long from, final long to); + KeyValueIterator backwardFetchAll(final long from, final long to); + /** * Remove the record with the provided key. The key * should be a composite of the record key, and the timestamp information etc @@ -171,6 +202,6 @@ interface KeySchema { * @param to * @return List of segments to search */ - List segmentsToSearch(Segments segments, long from, long to); + List segmentsToSearch(Segments segments, long from, long to, boolean backward); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java index bcfb02ebaf37f..38a5d318bcf66 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java @@ -34,9 +34,9 @@ interface Segments { void openExisting(final ProcessorContext context, final long streamTime); - List segments(final long timeFrom, final long timeTo); + List segments(final long timeFrom, final long timeTo, final boolean backward); - List allSegments(); + List allSegments(final boolean backward); void flush(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java index 43189395bcc62..ebb1be5aa77b0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java @@ -29,14 +29,16 @@ import org.apache.kafka.streams.state.WindowBytesStoreSupplier; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; - -import java.util.Objects; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Instant; +import java.util.Objects; + public class TimestampedWindowStoreBuilder extends AbstractStoreBuilder, TimestampedWindowStore> { - private final Logger log = LoggerFactory.getLogger(TimestampedWindowStoreBuilder.class); + + private static final Logger LOG = LoggerFactory.getLogger(TimestampedWindowStoreBuilder.class); private final WindowBytesStoreSupplier storeSupplier; @@ -60,7 +62,7 @@ public TimestampedWindowStore build() { } } if (storeSupplier.retainDuplicates() && enableCaching) { - log.warn("Disabling caching for {} since store was configured to retain duplicates", storeSupplier.name()); + LOG.warn("Disabling caching for {} since store was configured to retain duplicates", storeSupplier.name()); enableCaching = false; } @@ -141,6 +143,13 @@ public WindowStoreIterator fetch(final Bytes key, return wrapped.fetch(key, timeFrom, timeTo); } + @Override + public WindowStoreIterator backwardFetch(final Bytes key, + final Instant timeFrom, + final Instant timeTo) { + return wrapped.backwardFetch(key, timeFrom, timeTo); + } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, byte[]> fetch(final Bytes from, @@ -150,6 +159,14 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, return wrapped.fetch(from, to, timeFrom, timeTo); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, + final Bytes to, + final Instant timeFrom, + final Instant timeTo) { + return wrapped.backwardFetch(from, to, timeFrom, timeTo); + } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, @@ -157,11 +174,22 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, return wrapped.fetchAll(timeFrom, timeTo); } + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) { + return wrapped.backwardFetchAll(timeFrom, timeTo); + } + @Override public KeyValueIterator, byte[]> all() { return wrapped.all(); } + @Override + public KeyValueIterator, byte[]> backwardAll() { + return wrapped.backwardAll(); + } + @Override public void flush() { wrapped.flush(); @@ -171,6 +199,7 @@ public void flush() { public void close() { wrapped.close(); } + @Override public boolean isOpen() { return wrapped.isOpen(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java index 9218ccf0a7752..a675ae490f8c9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java @@ -23,11 +23,11 @@ import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.TimeWindow; import org.apache.kafka.streams.state.StateSerdes; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; import java.util.List; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class WindowKeySchema implements RocksDBSegmentedBytesStore.KeySchema { @@ -93,8 +93,9 @@ public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom, @Override public List segmentsToSearch(final Segments segments, final long from, - final long to) { - return segments.segments(from, to); + final long to, + final boolean backward) { + return segments.segments(from, to, backward); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java index 7bf8a0c5872d3..c36c4146f8dc2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java @@ -74,6 +74,13 @@ public WindowStoreIterator fetch(final Bytes key, return new WindowToTimestampedWindowIteratorAdapter(store.fetch(key, from, to)); } + @Override + public WindowStoreIterator backwardFetch(final Bytes key, + final Instant from, + final Instant to) { + return new WindowToTimestampedWindowIteratorAdapter(store.backwardFetch(key, from, to)); + } + @Override @SuppressWarnings("deprecation") public KeyValueIterator, byte[]> fetch(final Bytes from, @@ -83,6 +90,14 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetch(from, to, timeFrom, timeTo)); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, + final Bytes to, + final Instant fromTime, + final Instant toTime) { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardFetch(from, to, fromTime, toTime)); + } + @Override public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to, @@ -96,6 +111,11 @@ public KeyValueIterator, byte[]> all() { return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.all()); } + @Override + public KeyValueIterator, byte[]> backwardAll() { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardAll()); + } + @Override @SuppressWarnings("deprecation") public KeyValueIterator, byte[]> fetchAll(final long timeFrom, @@ -103,6 +123,12 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetchAll(timeFrom, timeTo)); } + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, + final Instant to) { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardFetchAll(from, to)); + } + @Override public KeyValueIterator, byte[]> fetchAll(final Instant from, final Instant to) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java index a40621697c33a..c57576c42414d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java @@ -50,6 +50,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Properties; @@ -64,6 +65,7 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; @@ -269,9 +271,27 @@ public void shouldGetAll() { final KeyValue, String> four = windowedPair(4, "four", startTime + 4); final KeyValue, String> five = windowedPair(5, "five", startTime + 5); - assertEquals( - new HashSet<>(asList(zero, one, two, four, five)), - toSet(windowStore.all()) + assertArrayEquals( + new LinkedHashSet<>(asList(zero, one, two, four, five)).toArray(), + toSet(windowStore.all()).toArray() + ); + } + + @Test + public void shouldGetBackwardAll() { + final long startTime = SEGMENT_INTERVAL - 4L; + + putFirstBatch(windowStore, startTime, context); + + final KeyValue, String> zero = windowedPair(0, "zero", startTime + 0); + final KeyValue, String> one = windowedPair(1, "one", startTime + 1); + final KeyValue, String> two = windowedPair(2, "two", startTime + 2); + final KeyValue, String> four = windowedPair(4, "four", startTime + 4); + final KeyValue, String> five = windowedPair(5, "five", startTime + 5); + + assertArrayEquals( + new LinkedHashSet<>(asList(five, four, two, one, zero)).toArray(), + toSet(windowStore.backwardAll()).toArray() ); } @@ -287,17 +307,43 @@ public void shouldFetchAllInTimeRange() { final KeyValue, String> four = windowedPair(4, "four", startTime + 4); final KeyValue, String> five = windowedPair(5, "five", startTime + 5); - assertEquals( - new HashSet<>(asList(one, two, four)), - toSet(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))) + assertArrayEquals( + new LinkedHashSet<>(asList(one, two, four)).toArray(), + toSet(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))).toArray() ); - assertEquals( - new HashSet<>(asList(zero, one, two)), - toSet(windowStore.fetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))) + assertArrayEquals( + new LinkedHashSet<>(asList(zero, one, two)).toArray(), + toSet(windowStore.fetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))).toArray() ); - assertEquals( - new HashSet<>(asList(one, two, four, five)), - toSet(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))) + assertArrayEquals( + new LinkedHashSet<>(asList(one, two, four, five)).toArray(), + toSet(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))).toArray() + ); + } + + @Test + public void shouldBackwardFetchAllInTimeRange() { + final long startTime = SEGMENT_INTERVAL - 4L; + + putFirstBatch(windowStore, startTime, context); + + final KeyValue, String> zero = windowedPair(0, "zero", startTime + 0); + final KeyValue, String> one = windowedPair(1, "one", startTime + 1); + final KeyValue, String> two = windowedPair(2, "two", startTime + 2); + final KeyValue, String> four = windowedPair(4, "four", startTime + 4); + final KeyValue, String> five = windowedPair(5, "five", startTime + 5); + + assertArrayEquals( + new LinkedHashSet<>(asList(four, two, one)).toArray(), + toSet(windowStore.backwardFetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(asList(two, one, zero)).toArray(), + toSet(windowStore.backwardFetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(asList(five, four, two, one)).toArray(), + toSet(windowStore.backwardFetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))).toArray() ); } @@ -313,69 +359,147 @@ public void testFetchRange() { final KeyValue, String> four = windowedPair(4, "four", startTime + 4); final KeyValue, String> five = windowedPair(5, "five", startTime + 5); - assertEquals( - new HashSet<>(asList(zero, one)), + assertArrayEquals( + new LinkedHashSet<>(asList(zero, one)).toArray(), toSet(windowStore.fetch( 0, 1, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))) + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() ); - assertEquals( - new HashSet<>(Collections.singletonList(one)), + assertArrayEquals( + new LinkedHashSet<>(Collections.singletonList(one)).toArray(), toSet(windowStore.fetch( 1, 1, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))) + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() ); - assertEquals( - new HashSet<>(asList(one, two)), + assertArrayEquals( + new LinkedHashSet<>(asList(one, two)).toArray(), toSet(windowStore.fetch( 1, 3, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))) + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() ); - assertEquals( - new HashSet<>(asList(zero, one, two)), + assertArrayEquals( + new LinkedHashSet<>(asList(zero, one, two)).toArray(), toSet(windowStore.fetch( 0, 5, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))) + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() ); - assertEquals( - new HashSet<>(asList(zero, one, two, four, five)), + assertArrayEquals( + new LinkedHashSet<>(asList(zero, one, two, four, five)).toArray(), toSet(windowStore.fetch( 0, 5, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))) + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))).toArray() ); - assertEquals( - new HashSet<>(asList(two, four, five)), + assertArrayEquals( + new LinkedHashSet<>(asList(two, four, five)).toArray(), toSet(windowStore.fetch( 0, 5, ofEpochMilli(startTime + 2L), - ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))) + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))).toArray() ); - assertEquals( - new HashSet<>(Collections.emptyList()), + assertArrayEquals( + new LinkedHashSet<>(Collections.emptyList()).toArray(), toSet(windowStore.fetch( 4, 5, ofEpochMilli(startTime + 2L), - ofEpochMilli(startTime + WINDOW_SIZE))) + ofEpochMilli(startTime + WINDOW_SIZE))).toArray() ); - assertEquals( - new HashSet<>(Collections.emptyList()), + assertArrayEquals( + new LinkedHashSet<>(Collections.emptyList()).toArray(), toSet(windowStore.fetch( 0, 3, ofEpochMilli(startTime + 3L), - ofEpochMilli(startTime + WINDOW_SIZE + 5))) + ofEpochMilli(startTime + WINDOW_SIZE + 5))).toArray() + ); + } + + @Test + public void testBackwardFetchRange() { + final long startTime = SEGMENT_INTERVAL - 4L; + + putFirstBatch(windowStore, startTime, context); + + final KeyValue, String> zero = windowedPair(0, "zero", startTime + 0); + final KeyValue, String> one = windowedPair(1, "one", startTime + 1); + final KeyValue, String> two = windowedPair(2, "two", startTime + 2); + final KeyValue, String> four = windowedPair(4, "four", startTime + 4); + final KeyValue, String> five = windowedPair(5, "five", startTime + 5); + + assertArrayEquals( + new LinkedHashSet<>(asList(one, zero)).toArray(), + toSet(windowStore.backwardFetch( + 0, + 1, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(Collections.singletonList(one)).toArray(), + toSet(windowStore.backwardFetch( + 1, + 1, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(asList(two, one)).toArray(), + toSet(windowStore.backwardFetch( + 1, + 3, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(asList(two, one, zero)).toArray(), + toSet(windowStore.backwardFetch( + 0, + 5, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(asList(five, four, two, one, zero)).toArray(), + toSet(windowStore.backwardFetch( + 0, + 5, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(asList(five, four, two)).toArray(), + toSet(windowStore.backwardFetch( + 0, + 5, + ofEpochMilli(startTime + 2L), + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(Collections.emptyList()).toArray(), + toSet(windowStore.backwardFetch( + 4, + 5, + ofEpochMilli(startTime + 2L), + ofEpochMilli(startTime + WINDOW_SIZE))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(Collections.emptyList()).toArray(), + toSet(windowStore.backwardFetch( + 0, + 3, + ofEpochMilli(startTime + 3L), + ofEpochMilli(startTime + WINDOW_SIZE + 5))).toArray() ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java index 49a3a95a910b9..c083bd4bbbfb9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java @@ -294,6 +294,29 @@ public void shouldGetAllFromCache() { assertFalse(iterator.hasNext()); } + @Test + @SuppressWarnings("deprecation") + public void shouldGetAllBackwardFromCache() { + cachingStore.put(bytesKey("a"), bytesValue("a")); + cachingStore.put(bytesKey("b"), bytesValue("b")); + cachingStore.put(bytesKey("c"), bytesValue("c")); + cachingStore.put(bytesKey("d"), bytesValue("d")); + cachingStore.put(bytesKey("e"), bytesValue("e")); + cachingStore.put(bytesKey("f"), bytesValue("f")); + cachingStore.put(bytesKey("g"), bytesValue("g")); + cachingStore.put(bytesKey("h"), bytesValue("h")); + + final KeyValueIterator, byte[]> iterator = cachingStore.backwardAll(); + final String[] array = {"h", "g", "f", "e", "d", "c", "b", "a"}; + for (final String s : array) { + verifyWindowedKeyValue( + iterator.next(), + new Windowed<>(bytesKey(s), new TimeWindow(DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE)), + s); + } + assertFalse(iterator.hasNext()); + } + @Test @SuppressWarnings("deprecation") public void shouldFetchAllWithinTimestampRange() { @@ -337,6 +360,49 @@ public void shouldFetchAllWithinTimestampRange() { assertFalse(iterator2.hasNext()); } + @Test + @SuppressWarnings("deprecation") + public void shouldFetchAllBackwardWithinTimestampRange() { + final String[] array = {"a", "b", "c", "d", "e", "f", "g", "h"}; + for (int i = 0; i < array.length; i++) { + context.setTime(i); + cachingStore.put(bytesKey(array[i]), bytesValue(array[i])); + } + + final KeyValueIterator, byte[]> iterator = + cachingStore.backwardFetchAll(ofEpochMilli(0), ofEpochMilli(7)); + for (int i = array.length - 1; i >= 0; i--) { + final String str = array[i]; + verifyWindowedKeyValue( + iterator.next(), + new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), + str); + } + assertFalse(iterator.hasNext()); + + final KeyValueIterator, byte[]> iterator1 = + cachingStore.backwardFetchAll(ofEpochMilli(2), ofEpochMilli(4)); + for (int i = 4; i >= 2; i--) { + final String str = array[i]; + verifyWindowedKeyValue( + iterator1.next(), + new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), + str); + } + assertFalse(iterator1.hasNext()); + + final KeyValueIterator, byte[]> iterator2 = + cachingStore.backwardFetchAll(ofEpochMilli(5), ofEpochMilli(7)); + for (int i = 7; i >= 5; i--) { + final String str = array[i]; + verifyWindowedKeyValue( + iterator2.next(), + new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), + str); + } + assertFalse(iterator2.hasNext()); + } + @Test public void shouldFlushEvictedItemsIntoUnderlyingStore() { final int added = addItemsToCache(); @@ -455,6 +521,18 @@ public void shouldIterateAcrossWindows() { assertFalse(fetch.hasNext()); } + @Test + public void shouldIterateBackwardAcrossWindows() { + cachingStore.put(bytesKey("1"), bytesValue("a"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE); + + final WindowStoreIterator fetch = + cachingStore.backwardFetch(bytesKey("1"), ofEpochMilli(DEFAULT_TIMESTAMP), ofEpochMilli(DEFAULT_TIMESTAMP + WINDOW_SIZE)); + verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP + WINDOW_SIZE, "b"); + verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP, "a"); + assertFalse(fetch.hasNext()); + } + @Test public void shouldIterateCacheAndStore() { final Bytes key = Bytes.wrap("1".getBytes()); @@ -467,6 +545,18 @@ public void shouldIterateCacheAndStore() { assertFalse(fetch.hasNext()); } + @Test + public void shouldIterateBackwardCacheAndStore() { + final Bytes key = Bytes.wrap("1".getBytes()); + bytesStore.put(WindowKeySchema.toStoreKeyBinary(key, DEFAULT_TIMESTAMP, 0), "a".getBytes()); + cachingStore.put(key, bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE); + final WindowStoreIterator fetch = + cachingStore.backwardFetch(bytesKey("1"), ofEpochMilli(DEFAULT_TIMESTAMP), ofEpochMilli(DEFAULT_TIMESTAMP + WINDOW_SIZE)); + verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP + WINDOW_SIZE, "b"); + verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP, "a"); + assertFalse(fetch.hasNext()); + } + @Test public void shouldIterateCacheAndStoreKeyRange() { final Bytes key = Bytes.wrap("1".getBytes()); @@ -486,6 +576,25 @@ public void shouldIterateCacheAndStoreKeyRange() { assertFalse(fetchRange.hasNext()); } + @Test + public void shouldIterateBackwardCacheAndStoreKeyRange() { + final Bytes key = Bytes.wrap("1".getBytes()); + bytesStore.put(WindowKeySchema.toStoreKeyBinary(key, DEFAULT_TIMESTAMP, 0), "a".getBytes()); + cachingStore.put(key, bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE); + + final KeyValueIterator, byte[]> fetchRange = + cachingStore.backwardFetch(key, bytesKey("2"), ofEpochMilli(DEFAULT_TIMESTAMP), ofEpochMilli(DEFAULT_TIMESTAMP + WINDOW_SIZE)); + verifyWindowedKeyValue( + fetchRange.next(), + new Windowed<>(key, new TimeWindow(DEFAULT_TIMESTAMP + WINDOW_SIZE, DEFAULT_TIMESTAMP + WINDOW_SIZE + WINDOW_SIZE)), + "b"); + verifyWindowedKeyValue( + fetchRange.next(), + new Windowed<>(key, new TimeWindow(DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE)), + "a"); + assertFalse(fetchRange.hasNext()); + } + @Test @SuppressWarnings("deprecation") public void shouldClearNamespaceCacheOnClose() { @@ -532,6 +641,24 @@ public void shouldFetchAndIterateOverExactKeys() { verifyKeyValueList(expected, actual); } + @Test + public void shouldBackwardFetchAndIterateOverExactKeys() { + cachingStore.put(bytesKey("a"), bytesValue("0001"), 0); + cachingStore.put(bytesKey("aa"), bytesValue("0002"), 0); + cachingStore.put(bytesKey("a"), bytesValue("0003"), 1); + cachingStore.put(bytesKey("aa"), bytesValue("0004"), 1); + cachingStore.put(bytesKey("a"), bytesValue("0005"), SEGMENT_INTERVAL); + + final List> expected = asList( + KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")), + KeyValue.pair(1L, bytesValue("0003")), + KeyValue.pair(0L, bytesValue("0001")) + ); + final List> actual = + toList(cachingStore.backwardFetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); + verifyKeyValueList(expected, actual); + } + @Test public void shouldFetchAndIterateOverKeyRange() { cachingStore.put(bytesKey("a"), bytesValue("0001"), 0); @@ -568,6 +695,42 @@ public void shouldFetchAndIterateOverKeyRange() { ); } + @Test + public void shouldFetchAndIterateOverKeyBackwardRange() { + cachingStore.put(bytesKey("a"), bytesValue("0001"), 0); + cachingStore.put(bytesKey("aa"), bytesValue("0002"), 0); + cachingStore.put(bytesKey("a"), bytesValue("0003"), 1); + cachingStore.put(bytesKey("aa"), bytesValue("0004"), 1); + cachingStore.put(bytesKey("a"), bytesValue("0005"), SEGMENT_INTERVAL); + + verifyKeyValueList( + asList( + windowedPair("a", "0005", SEGMENT_INTERVAL), + windowedPair("a", "0003", 1), + windowedPair("a", "0001", 0) + ), + toList(cachingStore.backwardFetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + ); + + verifyKeyValueList( + asList( + windowedPair("aa", "0004", 1), + windowedPair("aa", "0002", 0)), + toList(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + ); + + verifyKeyValueList( + asList( + windowedPair("a", "0005", SEGMENT_INTERVAL), + windowedPair("aa", "0004", 1), + windowedPair("aa", "0002", 0), + windowedPair("a", "0003", 1), + windowedPair("a", "0001", 0) + ), + toList(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + ); + } + @Test public void shouldReturnSameResultsForSingleKeyFetchAndEqualKeyRangeFetch() { cachingStore.put(bytesKey("a"), bytesValue("0001"), 0); @@ -584,6 +747,24 @@ public void shouldReturnSameResultsForSingleKeyFetchAndEqualKeyRangeFetch() { assertFalse(keyRangeIterator.hasNext()); } + @Test + public void shouldReturnSameResultsForSingleKeyFetchAndEqualKeyRangeBackwardFetch() { + cachingStore.put(bytesKey("a"), bytesValue("0001"), 0); + cachingStore.put(bytesKey("aa"), bytesValue("0002"), 1); + cachingStore.put(bytesKey("aa"), bytesValue("0003"), 2); + cachingStore.put(bytesKey("aaa"), bytesValue("0004"), 3); + + final WindowStoreIterator singleKeyIterator = + cachingStore.backwardFetch(bytesKey("aa"), Instant.ofEpochMilli(0L), Instant.ofEpochMilli(5L)); + final KeyValueIterator, byte[]> keyRangeIterator = + cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), Instant.ofEpochMilli(0L), Instant.ofEpochMilli(5L)); + + assertEquals(stringFrom(singleKeyIterator.next().value), stringFrom(keyRangeIterator.next().value)); + assertEquals(stringFrom(singleKeyIterator.next().value), stringFrom(keyRangeIterator.next().value)); + assertFalse(singleKeyIterator.hasNext()); + assertFalse(keyRangeIterator.hasNext()); + } + @Test(expected = NullPointerException.class) @SuppressWarnings("deprecation") public void shouldThrowNullPointerExceptionOnPutNullKey() { @@ -616,7 +797,7 @@ public void shouldNotThrowInvalidRangeExceptionWithNegativeFromKey() { final Bytes keyFrom = Bytes.wrap(Serdes.Integer().serializer().serialize("", -1)); final Bytes keyTo = Bytes.wrap(Serdes.Integer().serializer().serialize("", 1)); - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(CachingWindowStore.class)) { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(BytesRangeValidator.class)) { final KeyValueIterator, byte[]> iterator = cachingStore.fetch(keyFrom, keyTo, 0L, 10L); assertFalse(iterator.hasNext()); @@ -631,6 +812,26 @@ public void shouldNotThrowInvalidRangeExceptionWithNegativeFromKey() { } } + @Test + public void shouldNotThrowInvalidBackwardRangeExceptionWithNegativeFromKey() { + final Bytes keyFrom = Bytes.wrap(Serdes.Integer().serializer().serialize("", -1)); + final Bytes keyTo = Bytes.wrap(Serdes.Integer().serializer().serialize("", 1)); + + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(BytesRangeValidator.class)) { + final KeyValueIterator, byte[]> iterator = + cachingStore.backwardFetch(keyFrom, keyTo, Instant.ofEpochMilli(0L), Instant.ofEpochMilli(10L)); + assertFalse(iterator.hasNext()); + + final List messages = appender.getMessages(); + assertThat( + messages, + hasItem("Returning empty iterator for fetch with invalid key range: from > to." + + " This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes." + + " Note that the built-in numerical serdes do not follow this for negative numbers") + ); + } + } + @Test public void shouldCloseCacheAndWrappedStoreAfterErrorDuringCacheFlush() { setUpCloseTests(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java index f4cb5232746eb..7bc1e92cd3472 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java @@ -30,6 +30,8 @@ import org.junit.Test; import org.junit.runner.RunWith; +import java.time.Instant; + import static java.time.Instant.ofEpochMilli; @RunWith(EasyMockRunner.class) @@ -95,6 +97,18 @@ public void shouldDelegateToUnderlyingStoreWhenFetching() { EasyMock.verify(inner); } + @Test + public void shouldDelegateToUnderlyingStoreWhenBackwardFetching() { + EasyMock + .expect(inner.backwardFetch(bytesKey, Instant.ofEpochMilli(0), Instant.ofEpochMilli(10))) + .andReturn(KeyValueIterators.emptyWindowStoreIterator()); + + init(); + + store.backwardFetch(bytesKey, ofEpochMilli(0), ofEpochMilli(10)); + EasyMock.verify(inner); + } + @Test public void shouldDelegateToUnderlyingStoreWhenFetchingRange() { EasyMock @@ -107,6 +121,18 @@ public void shouldDelegateToUnderlyingStoreWhenFetchingRange() { EasyMock.verify(inner); } + @Test + public void shouldDelegateToUnderlyingStoreWhenBackwardFetchingRange() { + EasyMock + .expect(inner.backwardFetch(bytesKey, bytesKey, Instant.ofEpochMilli(0), Instant.ofEpochMilli(1))) + .andReturn(KeyValueIterators.emptyIterator()); + + init(); + + store.backwardFetch(bytesKey, bytesKey, ofEpochMilli(0), ofEpochMilli(1)); + EasyMock.verify(inner); + } + @Test @SuppressWarnings("deprecation") public void shouldRetainDuplicatesWhenSet() { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java index 514d0940a0c29..39313e17e3c0c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java @@ -44,6 +44,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsEqual.equalTo; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThrows; public class CompositeReadOnlyWindowStoreTest { @@ -69,8 +70,8 @@ public void before() { windowStore = new CompositeReadOnlyWindowStore<>( new WrappingStoreProvider(asList(stubProviderOne, stubProviderTwo), StoreQueryParameters.fromNameAndType(storeName, QueryableStoreTypes.windowStore())), - QueryableStoreTypes.windowStore(), - storeName + QueryableStoreTypes.windowStore(), + storeName ); } @@ -79,18 +80,41 @@ public void shouldFetchValuesFromWindowStore() { underlyingWindowStore.put("my-key", "my-value", 0L); underlyingWindowStore.put("my-key", "my-later-value", 10L); - final WindowStoreIterator iterator = windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); + final WindowStoreIterator iterator = + windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); final List> results = StreamsTestUtils.toList(iterator); - assertEquals(asList(new KeyValue<>(0L, "my-value"), - new KeyValue<>(10L, "my-later-value")), - results); + assertEquals( + asList(new KeyValue<>(0L, "my-value"), new KeyValue<>(10L, "my-later-value")), + results); + } + + @Test + public void shouldBackwardFetchValuesFromWindowStore() { + underlyingWindowStore.put("my-key", "my-value", 0L); + underlyingWindowStore.put("my-key", "my-later-value", 10L); + + final WindowStoreIterator iterator = + windowStore.backwardFetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); + final List> results = StreamsTestUtils.toList(iterator); + + assertEquals( + asList(new KeyValue<>(10L, "my-later-value"), new KeyValue<>(0L, "my-value")), + results); } @Test public void shouldReturnEmptyIteratorIfNoData() { - final WindowStoreIterator iterator = windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); - assertEquals(false, iterator.hasNext()); + final WindowStoreIterator iterator = + windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); + assertFalse(iterator.hasNext()); + } + + @Test + public void shouldReturnBackwardEmptyIteratorIfNoData() { + final WindowStoreIterator iterator = + windowStore.backwardFetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); + assertFalse(iterator.hasNext()); } @Test @@ -102,10 +126,28 @@ public void shouldFindValueForKeyWhenMultiStores() { underlyingWindowStore.put("key-one", "value-one", 0L); secondUnderlying.put("key-two", "value-two", 10L); - final List> keyOneResults = StreamsTestUtils.toList(windowStore.fetch("key-one", ofEpochMilli(0L), - ofEpochMilli(1L))); - final List> keyTwoResults = StreamsTestUtils.toList(windowStore.fetch("key-two", ofEpochMilli(10L), - ofEpochMilli(11L))); + final List> keyOneResults = + StreamsTestUtils.toList(windowStore.fetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L))); + final List> keyTwoResults = + StreamsTestUtils.toList(windowStore.fetch("key-two", ofEpochMilli(10L), ofEpochMilli(11L))); + + assertEquals(Collections.singletonList(KeyValue.pair(0L, "value-one")), keyOneResults); + assertEquals(Collections.singletonList(KeyValue.pair(10L, "value-two")), keyTwoResults); + } + + @Test + public void shouldFindValueForKeyWhenMultiStoresBackwards() { + final ReadOnlyWindowStoreStub secondUnderlying = new + ReadOnlyWindowStoreStub<>(WINDOW_SIZE); + stubProviderTwo.addStore(storeName, secondUnderlying); + + underlyingWindowStore.put("key-one", "value-one", 0L); + secondUnderlying.put("key-two", "value-two", 10L); + + final List> keyOneResults = + StreamsTestUtils.toList(windowStore.backwardFetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L))); + final List> keyTwoResults = + StreamsTestUtils.toList(windowStore.backwardFetch("key-two", ofEpochMilli(10L), ofEpochMilli(11L))); assertEquals(Collections.singletonList(KeyValue.pair(0L, "value-one")), keyOneResults); assertEquals(Collections.singletonList(KeyValue.pair(10L, "value-two")), keyTwoResults); @@ -116,7 +158,18 @@ public void shouldNotGetValuesFromOtherStores() { otherUnderlyingStore.put("some-key", "some-value", 0L); underlyingWindowStore.put("some-key", "my-value", 1L); - final List> results = StreamsTestUtils.toList(windowStore.fetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L))); + final List> results = + StreamsTestUtils.toList(windowStore.fetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L))); + assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results); + } + + @Test + public void shouldNotGetValuesBackwardFromOtherStores() { + otherUnderlyingStore.put("some-key", "some-value", 0L); + underlyingWindowStore.put("some-key", "my-value", 1L); + + final List> results = + StreamsTestUtils.toList(windowStore.backwardFetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L))); assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results); } @@ -124,7 +177,7 @@ public void shouldNotGetValuesFromOtherStores() { public void shouldThrowInvalidStateStoreExceptionOnRebalance() { final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); EasyMock.expect(storeProvider.stores(anyString(), anyObject())) - .andThrow(new InvalidStateStoreException("store is unavailable")); + .andThrow(new InvalidStateStoreException("store is unavailable")); EasyMock.replay(storeProvider); final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>( @@ -135,24 +188,74 @@ public void shouldThrowInvalidStateStoreExceptionOnRebalance() { store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); } + @Test(expected = InvalidStateStoreException.class) + public void shouldThrowInvalidStateStoreExceptionOnRebalanceWhenBackwards() { + final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); + EasyMock.expect(storeProvider.stores(anyString(), anyObject())) + .andThrow(new InvalidStateStoreException("store is unavailable")); + EasyMock.replay(storeProvider); + + final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>( + storeProvider, + QueryableStoreTypes.windowStore(), + "foo" + ); + store.backwardFetch("key", ofEpochMilli(1), ofEpochMilli(10)); + } + @Test public void shouldThrowInvalidStateStoreExceptionIfFetchThrows() { underlyingWindowStore.setOpen(false); final CompositeReadOnlyWindowStore store = - new CompositeReadOnlyWindowStore<>( - new WrappingStoreProvider(singletonList(stubProviderOne), StoreQueryParameters.fromNameAndType("window-store", QueryableStoreTypes.windowStore())), - QueryableStoreTypes.windowStore(), - "window-store" - ); + new CompositeReadOnlyWindowStore<>( + new WrappingStoreProvider(singletonList(stubProviderOne), StoreQueryParameters.fromNameAndType("window-store", QueryableStoreTypes.windowStore())), + QueryableStoreTypes.windowStore(), + "window-store" + ); try { store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); Assert.fail("InvalidStateStoreException was expected"); } catch (final InvalidStateStoreException e) { Assert.assertEquals("State store is not available anymore and may have been migrated to another instance; " + - "please re-discover its location from the state metadata.", e.getMessage()); + "please re-discover its location from the state metadata.", e.getMessage()); + } + } + + @Test + public void shouldThrowInvalidStateStoreExceptionIfBackwardFetchThrows() { + underlyingWindowStore.setOpen(false); + final CompositeReadOnlyWindowStore store = + new CompositeReadOnlyWindowStore<>( + new WrappingStoreProvider(singletonList(stubProviderOne), StoreQueryParameters.fromNameAndType("window-store", QueryableStoreTypes.windowStore())), + QueryableStoreTypes.windowStore(), + "window-store" + ); + try { + store.backwardFetch("key", ofEpochMilli(1), ofEpochMilli(10)); + Assert.fail("InvalidStateStoreException was expected"); + } catch (final InvalidStateStoreException e) { + Assert.assertEquals("State store is not available anymore and may have been migrated to another instance; " + + "please re-discover its location from the state metadata.", e.getMessage()); } } + @Test + public void emptyBackwardIteratorAlwaysReturnsFalse() { + final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); + EasyMock.expect(storeProvider.stores(anyString(), anyObject())).andReturn(emptyList()); + EasyMock.replay(storeProvider); + + final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>( + storeProvider, + QueryableStoreTypes.windowStore(), + "foo" + ); + final WindowStoreIterator windowStoreIterator = + store.backwardFetch("key", ofEpochMilli(1), ofEpochMilli(10)); + + Assert.assertFalse(windowStoreIterator.hasNext()); + } + @Test public void emptyIteratorAlwaysReturnsFalse() { final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); @@ -164,11 +267,28 @@ public void emptyIteratorAlwaysReturnsFalse() { QueryableStoreTypes.windowStore(), "foo" ); - final WindowStoreIterator windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); + final WindowStoreIterator windowStoreIterator = + store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); Assert.assertFalse(windowStoreIterator.hasNext()); } + @Test + public void emptyBackwardIteratorPeekNextKeyShouldThrowNoSuchElementException() { + final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); + EasyMock.expect(storeProvider.stores(anyString(), anyObject())).andReturn(emptyList()); + EasyMock.replay(storeProvider); + + final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>( + storeProvider, + QueryableStoreTypes.windowStore(), + "foo" + ); + final WindowStoreIterator windowStoreIterator = store.backwardFetch("key", ofEpochMilli(1), ofEpochMilli(10)); + assertThrows(NoSuchElementException.class, windowStoreIterator::peekNextKey); + } + + @Test public void emptyIteratorPeekNextKeyShouldThrowNoSuchElementException() { final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); @@ -180,7 +300,8 @@ public void emptyIteratorPeekNextKeyShouldThrowNoSuchElementException() { QueryableStoreTypes.windowStore(), "foo" ); - final WindowStoreIterator windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); + final WindowStoreIterator windowStoreIterator = + store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); assertThrows(NoSuchElementException.class, windowStoreIterator::peekNextKey); } @@ -195,7 +316,24 @@ public void emptyIteratorNextShouldThrowNoSuchElementException() { QueryableStoreTypes.windowStore(), "foo" ); - final WindowStoreIterator windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); + final WindowStoreIterator windowStoreIterator = + store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); + assertThrows(NoSuchElementException.class, windowStoreIterator::next); + } + + @Test + public void emptyBackwardIteratorNextShouldThrowNoSuchElementException() { + final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); + EasyMock.expect(storeProvider.stores(anyString(), anyObject())).andReturn(emptyList()); + EasyMock.replay(storeProvider); + + final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>( + storeProvider, + QueryableStoreTypes.windowStore(), + "foo" + ); + final WindowStoreIterator windowStoreIterator = + store.backwardFetch("key", ofEpochMilli(1), ofEpochMilli(10)); assertThrows(NoSuchElementException.class, windowStoreIterator::next); } @@ -205,10 +343,24 @@ public void shouldFetchKeyRangeAcrossStores() { stubProviderTwo.addStore(storeName, secondUnderlying); underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); - final List, String>> results = StreamsTestUtils.toList(windowStore.fetch("a", "b", ofEpochMilli(0), ofEpochMilli(10))); + final List, String>> results = + StreamsTestUtils.toList(windowStore.fetch("a", "b", ofEpochMilli(0), ofEpochMilli(10))); + assertThat(results, equalTo(Arrays.asList( + KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), + KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); + } + + @Test + public void shouldBackwardFetchKeyRangeAcrossStores() { + final ReadOnlyWindowStoreStub secondUnderlying = new ReadOnlyWindowStoreStub<>(WINDOW_SIZE); + stubProviderTwo.addStore(storeName, secondUnderlying); + underlyingWindowStore.put("a", "a", 0L); + secondUnderlying.put("b", "b", 10L); + final List, String>> results = + StreamsTestUtils.toList(windowStore.backwardFetch("a", "b", ofEpochMilli(0), ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( - KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), - KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); + KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), + KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); } @Test @@ -223,31 +375,58 @@ public void shouldFetchKeyValueAcrossStores() { assertThat(windowStore.fetch("a", 10L), equalTo(null)); } - @Test public void shouldGetAllAcrossStores() { final ReadOnlyWindowStoreStub secondUnderlying = new - ReadOnlyWindowStoreStub<>(WINDOW_SIZE); + ReadOnlyWindowStoreStub<>(WINDOW_SIZE); stubProviderTwo.addStore(storeName, secondUnderlying); underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); final List, String>> results = StreamsTestUtils.toList(windowStore.all()); assertThat(results, equalTo(Arrays.asList( - KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), - KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); + KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), + KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); + } + + @Test + public void shouldGetBackwardAllAcrossStores() { + final ReadOnlyWindowStoreStub secondUnderlying = new + ReadOnlyWindowStoreStub<>(WINDOW_SIZE); + stubProviderTwo.addStore(storeName, secondUnderlying); + underlyingWindowStore.put("a", "a", 0L); + secondUnderlying.put("b", "b", 10L); + final List, String>> results = StreamsTestUtils.toList(windowStore.backwardAll()); + assertThat(results, equalTo(Arrays.asList( + KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), + KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); } @Test public void shouldFetchAllAcrossStores() { final ReadOnlyWindowStoreStub secondUnderlying = new - ReadOnlyWindowStoreStub<>(WINDOW_SIZE); + ReadOnlyWindowStoreStub<>(WINDOW_SIZE); + stubProviderTwo.addStore(storeName, secondUnderlying); + underlyingWindowStore.put("a", "a", 0L); + secondUnderlying.put("b", "b", 10L); + final List, String>> results = + StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(0), ofEpochMilli(10))); + assertThat(results, equalTo(Arrays.asList( + KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), + KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); + } + + @Test + public void shouldBackwardFetchAllAcrossStores() { + final ReadOnlyWindowStoreStub secondUnderlying = new + ReadOnlyWindowStoreStub<>(WINDOW_SIZE); stubProviderTwo.addStore(storeName, secondUnderlying); underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); - final List, String>> results = StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(0), ofEpochMilli(10))); + final List, String>> results = + StreamsTestUtils.toList(windowStore.backwardFetchAll(ofEpochMilli(0), ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( - KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), - KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); + KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), + KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); } @Test(expected = NullPointerException.class) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryWindowStoreTest.java index 15b31f0dd355a..e5f214292de68 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryWindowStoreTest.java @@ -16,13 +16,6 @@ */ package org.apache.kafka.streams.state.internals; -import static java.time.Duration.ofMillis; -import static org.apache.kafka.streams.state.internals.WindowKeySchema.toStoreKeyBinary; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; - -import java.util.LinkedList; -import java.util.List; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValue; @@ -35,16 +28,24 @@ import org.apache.kafka.streams.state.WindowStoreIterator; import org.junit.Test; +import java.util.LinkedList; +import java.util.List; + +import static java.time.Duration.ofMillis; +import static org.apache.kafka.streams.state.internals.WindowKeySchema.toStoreKeyBinary; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + public class InMemoryWindowStoreTest extends AbstractWindowBytesStoreTest { private final static String STORE_NAME = "InMemoryWindowStore"; @Override WindowStore buildWindowStore(final long retentionPeriod, - final long windowSize, - final boolean retainDuplicates, - final Serde keySerde, - final Serde valueSerde) { + final long windowSize, + final boolean retainDuplicates, + final Serde keySerde, + final Serde valueSerde) { return Stores.windowStoreBuilder( Stores.inMemoryWindowStore( STORE_NAME, @@ -174,5 +175,5 @@ public void testExpiration() { assertEquals(windowedPair(1, "six", 5 * (RETENTION_PERIOD / 4)), iterator.next()); assertFalse(iterator.hasNext()); } - + } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java index 77d97badc780f..7559b846badbf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java @@ -28,6 +28,7 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import static org.hamcrest.CoreMatchers.equalTo; @@ -72,7 +73,7 @@ public void shouldIterateOverValueFromBothIterators() { ); final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( - cacheIterator, storeIterator + cacheIterator, storeIterator, false ); int index = 0; while (iterator.hasNext()) { @@ -84,6 +85,45 @@ public void shouldIterateOverValueFromBothIterators() { iterator.close(); } + + @Test + public void shouldReverseIterateOverValueFromBothIterators() { + final List> expectedKvPairs = new ArrayList<>(); + for (long t = 0; t < 100; t += 20) { + final byte[] v1Bytes = String.valueOf(t).getBytes(); + final KeyValue v1 = KeyValue.pair(t, v1Bytes); + windowStoreKvPairs.add(v1); + expectedKvPairs.add(KeyValue.pair(t, v1Bytes)); + final Bytes keyBytes = WindowKeySchema.toStoreKeyBinary("a", t + 10, 0, stateSerdes); + final byte[] valBytes = String.valueOf(t + 10).getBytes(); + expectedKvPairs.add(KeyValue.pair(t + 10, valBytes)); + cache.put(namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(keyBytes), new LRUCacheEntry(valBytes)); + } + + final Bytes fromBytes = WindowKeySchema.toStoreKeyBinary("a", 0, 0, stateSerdes); + final Bytes toBytes = WindowKeySchema.toStoreKeyBinary("a", 100, 0, stateSerdes); + Collections.reverse(windowStoreKvPairs); + final KeyValueIterator storeIterator = + new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); + + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.reverseRange( + namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes) + ); + + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( + cacheIterator, storeIterator, true + ); + int index = 0; + Collections.reverse(expectedKvPairs); + while (iterator.hasNext()) { + final KeyValue next = iterator.next(); + final KeyValue expected = expectedKvPairs.get(index++); + assertArrayEquals(expected.value, next.value); + assertEquals(expected.key, next.key); + } + iterator.close(); + } + @Test public void shouldPeekNextStoreKey() { windowStoreKvPairs.add(KeyValue.pair(10L, "a".getBytes())); @@ -95,7 +135,7 @@ public void shouldPeekNextStoreKey() { namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes) ); final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( - cacheIterator, storeIterator + cacheIterator, storeIterator, false ); assertThat(iterator.peekNextKey(), equalTo(0L)); iterator.next(); @@ -103,6 +143,27 @@ public void shouldPeekNextStoreKey() { iterator.close(); } + @Test + public void shouldPeekNextStoreKeyReverse() { + windowStoreKvPairs.add(KeyValue.pair(10L, "a".getBytes())); + cache.put(namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(WindowKeySchema.toStoreKeyBinary("a", 0, 0, stateSerdes)), new LRUCacheEntry("b".getBytes())); + final Bytes fromBytes = WindowKeySchema.toStoreKeyBinary("a", 0, 0, stateSerdes); + final Bytes toBytes = WindowKeySchema.toStoreKeyBinary("a", 100, 0, stateSerdes); + final KeyValueIterator storeIterator = + new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.reverseRange( + namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), + SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes) + ); + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( + cacheIterator, storeIterator, true + ); + assertThat(iterator.peekNextKey(), equalTo(10L)); + iterator.next(); + assertThat(iterator.peekNextKey(), equalTo(0L)); + iterator.close(); + } + @Test public void shouldPeekNextCacheKey() { windowStoreKvPairs.add(KeyValue.pair(0L, "a".getBytes())); @@ -111,10 +172,27 @@ public void shouldPeekNextCacheKey() { final Bytes toBytes = WindowKeySchema.toStoreKeyBinary("a", 100, 0, stateSerdes); final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes)); - final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator(cacheIterator, storeIterator); + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator(cacheIterator, storeIterator, false); assertThat(iterator.peekNextKey(), equalTo(0L)); iterator.next(); assertThat(iterator.peekNextKey(), equalTo(10L)); iterator.close(); } -} + + @Test + public void shouldPeekNextCacheKeyReverse() { + windowStoreKvPairs.add(KeyValue.pair(0L, "a".getBytes())); + cache.put(namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(WindowKeySchema.toStoreKeyBinary("a", 10L, 0, stateSerdes)), new LRUCacheEntry("b".getBytes())); + final Bytes fromBytes = WindowKeySchema.toStoreKeyBinary("a", 0, 0, stateSerdes); + final Bytes toBytes = WindowKeySchema.toStoreKeyBinary("a", 100, 0, stateSerdes); + final KeyValueIterator storeIterator = + new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = + cache.reverseRange(namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes)); + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator(cacheIterator, storeIterator, true); + assertThat(iterator.peekNextKey(), equalTo(10L)); + iterator.next(); + assertThat(iterator.peekNextKey(), equalTo(0L)); + iterator.close(); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java index 583e635a3f71d..a160df8e2e6e3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java @@ -63,53 +63,103 @@ public long segmentId(final Bytes key) { @Test public void shouldHaveNextFromStore() { final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = - createIterator(storeKvs, Collections.emptyIterator()); + createIterator(storeKvs, Collections.emptyIterator(), false); + assertTrue(mergeIterator.hasNext()); + } + + @Test + public void shouldHaveNextFromReverseStore() { + final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = + createIterator(storeKvs, Collections.emptyIterator(), true); assertTrue(mergeIterator.hasNext()); } @Test public void shouldGetNextFromStore() { final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = - createIterator(storeKvs, Collections.emptyIterator()); + createIterator(storeKvs, Collections.emptyIterator(), false); + assertThat(convertKeyValuePair(mergeIterator.next()), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); + } + + @Test + public void shouldGetNextFromReverseStore() { + final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = + createIterator(storeKvs, Collections.emptyIterator(), true); assertThat(convertKeyValuePair(mergeIterator.next()), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); } @Test public void shouldPeekNextKeyFromStore() { final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = - createIterator(storeKvs, Collections.emptyIterator()); + createIterator(storeKvs, Collections.emptyIterator(), false); + assertThat(convertWindowedKey(mergeIterator.peekNextKey()), equalTo(new Windowed<>(storeKey, storeWindow))); + } + + @Test + public void shouldPeekNextKeyFromReverseStore() { + final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = + createIterator(storeKvs, Collections.emptyIterator(), true); assertThat(convertWindowedKey(mergeIterator.peekNextKey()), equalTo(new Windowed<>(storeKey, storeWindow))); } @Test public void shouldHaveNextFromCache() { final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = - createIterator(Collections.emptyIterator(), cacheKvs); + createIterator(Collections.emptyIterator(), cacheKvs, false); + assertTrue(mergeIterator.hasNext()); + } + + @Test + public void shouldHaveNextFromReverseCache() { + final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = + createIterator(Collections.emptyIterator(), cacheKvs, true); assertTrue(mergeIterator.hasNext()); } @Test public void shouldGetNextFromCache() { final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = - createIterator(Collections.emptyIterator(), cacheKvs); + createIterator(Collections.emptyIterator(), cacheKvs, false); + assertThat(convertKeyValuePair(mergeIterator.next()), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); + } + + @Test + public void shouldGetNextFromReverseCache() { + final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = + createIterator(Collections.emptyIterator(), cacheKvs, true); assertThat(convertKeyValuePair(mergeIterator.next()), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); } @Test public void shouldPeekNextKeyFromCache() { final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = - createIterator(Collections.emptyIterator(), cacheKvs); + createIterator(Collections.emptyIterator(), cacheKvs, false); + assertThat(convertWindowedKey(mergeIterator.peekNextKey()), equalTo(new Windowed<>(cacheKey, cacheWindow))); + } + + @Test + public void shouldPeekNextKeyFromReverseCache() { + final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = + createIterator(Collections.emptyIterator(), cacheKvs, true); assertThat(convertWindowedKey(mergeIterator.peekNextKey()), equalTo(new Windowed<>(cacheKey, cacheWindow))); } @Test public void shouldIterateBothStoreAndCache() { - final MergedSortedCacheWindowStoreKeyValueIterator iterator = createIterator(storeKvs, cacheKvs); + final MergedSortedCacheWindowStoreKeyValueIterator iterator = createIterator(storeKvs, cacheKvs, false); assertThat(convertKeyValuePair(iterator.next()), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); assertThat(convertKeyValuePair(iterator.next()), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); assertFalse(iterator.hasNext()); } + @Test + public void shouldReverseIterateBothStoreAndCache() { + final MergedSortedCacheWindowStoreKeyValueIterator iterator = createIterator(storeKvs, cacheKvs, true); + assertThat(convertKeyValuePair(iterator.next()), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); + assertThat(convertKeyValuePair(iterator.next()), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); + assertFalse(iterator.hasNext()); + } + private KeyValue, String> convertKeyValuePair(final KeyValue, byte[]> next) { final String value = deserializer.deserialize("", next.value); return KeyValue.pair(convertWindowedKey(next.key), value); @@ -123,7 +173,8 @@ private Windowed convertWindowedKey(final Windowed bytesWindowed) private MergedSortedCacheWindowStoreKeyValueIterator createIterator( final Iterator, byte[]>> storeKvs, - final Iterator> cacheKvs + final Iterator> cacheKvs, + final boolean reverse ) { final DelegatingPeekingKeyValueIterator, byte[]> storeIterator = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(storeKvs)); @@ -135,7 +186,8 @@ private MergedSortedCacheWindowStoreKeyValueIterator createIterator( storeIterator, new StateSerdes<>("name", Serdes.Bytes(), Serdes.ByteArray()), WINDOW_SIZE, - SINGLE_SEGMENT_CACHE_FUNCTION + SINGLE_SEGMENT_CACHE_FUNCTION, + reverse ); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java index 92bc03b7ba366..8671521563786 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java @@ -359,7 +359,8 @@ public void shouldNotThrowNullPointerExceptionIfFetchReturnsNull() { assertNull(store.fetch("a", 0)); } - private interface CachedWindowStore extends WindowStore, CachedStateStore { } + private interface CachedWindowStore extends WindowStore, CachedStateStore { + } @SuppressWarnings("unchecked") @Test @@ -427,9 +428,9 @@ public void shouldRemoveMetricsEvenIfWrappedStoreThrowsOnClose() { private List storeMetrics() { return metrics.metrics() - .keySet() - .stream() - .filter(name -> name.group().equals(storeLevelGroup) && name.tags().equals(tags)) - .collect(Collectors.toList()); + .keySet() + .stream() + .filter(name -> name.group().equals(storeLevelGroup) && name.tags().equals(tags)) + .collect(Collectors.toList()); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java index 468d551d793c0..cc26a454ba7a7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java @@ -46,7 +46,7 @@ public class ReadOnlyWindowStoreStub implements ReadOnlyWindowStore, private final long windowSize; private final Map> data = new HashMap<>(); - private boolean open = true; + private boolean open = true; ReadOnlyWindowStoreStub(final long windowSize) { this.windowSize = windowSize; @@ -81,11 +81,28 @@ public WindowStoreIterator fetch(final K key, final long timeFrom, final long @Override public WindowStoreIterator fetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException { return fetch( - key, + key, ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")), ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to"))); } + @Override + public WindowStoreIterator backwardFetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + if (!open) { + throw new InvalidStateStoreException("Store is not open"); + } + final List> results = new ArrayList<>(); + for (long now = timeTo; now >= timeFrom; now--) { + final Map kvMap = data.get(now); + if (kvMap != null && kvMap.containsKey(key)) { + results.add(new KeyValue<>(now, kvMap.get(key))); + } + } + return new TheWindowStoreIterator<>(results.iterator()); + } + @Override public KeyValueIterator, V> all() { if (!open) { @@ -104,7 +121,47 @@ public KeyValueIterator, V> all() { return new KeyValueIterator, V>() { @Override - public void close() {} + public void close() { + } + + @Override + public Windowed peekNextKey() { + throw new UnsupportedOperationException("peekNextKey() not supported in " + getClass().getName()); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public KeyValue, V> next() { + return iterator.next(); + } + + }; + } + + @Override + public KeyValueIterator, V> backwardAll() { + if (!open) { + throw new InvalidStateStoreException("Store is not open"); + } + final List, V>> results = new ArrayList<>(); + for (final long now : data.keySet()) { + final NavigableMap kvMap = data.get(now); + if (kvMap != null) { + for (final Entry entry : kvMap.descendingMap().entrySet()) { + results.add(new KeyValue<>(new Windowed<>(entry.getKey(), new TimeWindow(now, now + windowSize)), entry.getValue())); + } + } + } + final Iterator, V>> iterator = results.iterator(); + + return new KeyValueIterator, V>() { + @Override + public void close() { + } @Override public Windowed peekNextKey() { @@ -146,7 +203,8 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, final long return new KeyValueIterator, V>() { @Override - public void close() {} + public void close() { + } @Override public Windowed peekNextKey() { @@ -173,6 +231,51 @@ public KeyValueIterator, V> fetchAll(final Instant from, final Insta ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to"))); } + @Override + public KeyValueIterator, V> backwardFetchAll(final Instant from, final Instant to) throws IllegalArgumentException { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + + if (!open) { + throw new InvalidStateStoreException("Store is not open"); + } + final List, V>> results = new ArrayList<>(); + for (final long now : data.keySet()) { + if (!(now >= timeFrom && now <= timeTo)) { + continue; + } + final NavigableMap kvMap = data.get(now); + if (kvMap != null) { + for (final Entry entry : kvMap.descendingMap().entrySet()) { + results.add(new KeyValue<>(new Windowed<>(entry.getKey(), new TimeWindow(now, now + windowSize)), entry.getValue())); + } + } + } + final Iterator, V>> iterator = results.iterator(); + + return new KeyValueIterator, V>() { + @Override + public void close() { + } + + @Override + public Windowed peekNextKey() { + throw new UnsupportedOperationException("peekNextKey() not supported in " + getClass().getName()); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public KeyValue, V> next() { + return iterator.next(); + } + + }; + } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { @@ -192,7 +295,8 @@ public KeyValueIterator, V> fetch(final K from, final K to, final lo return new KeyValueIterator, V>() { @Override - public void close() {} + public void close() { + } @Override public Windowed peekNextKey() { @@ -212,17 +316,62 @@ public KeyValue, V> next() { }; } - @Override public KeyValueIterator, V> fetch(final K from, - final K to, - final Instant fromTime, - final Instant toTime) throws IllegalArgumentException { + @Override + public KeyValueIterator, V> fetch(final K from, + final K to, + final Instant fromTime, + final Instant toTime) throws IllegalArgumentException { return fetch( from, - to, + to, ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")), ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime"))); } + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final Instant fromTimeInstant, + final Instant toTimeInstant) throws IllegalArgumentException { + final long timeFrom = ApiUtils.validateMillisecondInstant(fromTimeInstant, prepareMillisCheckFailMsgPrefix(fromTimeInstant, "fromTimeInstant")); + final long timeTo = ApiUtils.validateMillisecondInstant(toTimeInstant, prepareMillisCheckFailMsgPrefix(toTimeInstant, "toTimeInstant")); + if (!open) { + throw new InvalidStateStoreException("Store is not open"); + } + final List, V>> results = new ArrayList<>(); + for (long now = timeFrom; now <= timeTo; now++) { + final NavigableMap kvMap = data.get(now); + if (kvMap != null) { + for (final Entry entry : kvMap.subMap(from, true, to, true).descendingMap().entrySet()) { + results.add(new KeyValue<>(new Windowed<>(entry.getKey(), new TimeWindow(now, now + windowSize)), entry.getValue())); + } + } + } + final Iterator, V>> iterator = results.iterator(); + + return new KeyValueIterator, V>() { + @Override + public void close() { + } + + @Override + public Windowed peekNextKey() { + throw new UnsupportedOperationException("peekNextKey() not supported in " + getClass().getName()); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public KeyValue, V> next() { + return iterator.next(); + } + + }; + } + public void put(final K key, final V value, final long timestamp) { if (!data.containsKey(timestamp)) { data.put(timestamp, new TreeMap<>()); @@ -236,13 +385,16 @@ public String name() { } @Override - public void init(final ProcessorContext context, final StateStore root) {} + public void init(final ProcessorContext context, final StateStore root) { + } @Override - public void flush() {} + public void flush() { + } @Override - public void close() {} + public void close() { + } @Override public boolean persistent() { @@ -267,7 +419,8 @@ private class TheWindowStoreIterator implements WindowStoreIterator { } @Override - public void close() {} + public void close() { + } @Override public Long peekNextKey() { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java index f02e75f8c7859..74ef7b7a531c9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java @@ -54,14 +54,14 @@ public class SegmentIteratorTest { @Before public void before() { final InternalMockProcessorContext context = new InternalMockProcessorContext( - TestUtils.tempDirectory(), - Serdes.String(), - Serdes.String(), - new MockRecordCollector(), - new ThreadCache( - new LogContext("testCache "), - 0, - new MockStreamsMetrics(new Metrics()))); + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + new MockRecordCollector(), + new ThreadCache( + new LogContext("testCache "), + 0, + new MockStreamsMetrics(new Metrics()))); segmentOne.init(context, segmentOne); segmentTwo.init(context, segmentTwo); segmentOne.put(Bytes.wrap("a".getBytes()), "1".getBytes()); @@ -86,7 +86,7 @@ public void shouldIterateOverAllSegments() { Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("a".getBytes()), - Bytes.wrap("z".getBytes())); + Bytes.wrap("z".getBytes()), false); assertTrue(iterator.hasNext()); assertEquals("a", new String(iterator.peekNextKey().get())); @@ -107,26 +107,76 @@ public void shouldIterateOverAllSegments() { assertFalse(iterator.hasNext()); } + @Test + public void shouldIterateBackwardOverAllSegments() { + iterator = new SegmentIterator<>( + Arrays.asList(segmentOne, segmentTwo).iterator(), + hasNextCondition, + Bytes.wrap("a".getBytes()), + Bytes.wrap("z".getBytes()), + true); + + assertTrue(iterator.hasNext()); + assertEquals("b", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("b", "2"), toStringKeyValue(iterator.next())); + + assertTrue(iterator.hasNext()); + assertEquals("a", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("a", "1"), toStringKeyValue(iterator.next())); + + assertTrue(iterator.hasNext()); + assertEquals("d", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("d", "4"), toStringKeyValue(iterator.next())); + + assertTrue(iterator.hasNext()); + assertEquals("c", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("c", "3"), toStringKeyValue(iterator.next())); + + assertFalse(iterator.hasNext()); + } + @Test public void shouldNotThrowExceptionOnHasNextWhenStoreClosed() { iterator = new SegmentIterator<>( Collections.singletonList(segmentOne).iterator(), hasNextCondition, Bytes.wrap("a".getBytes()), - Bytes.wrap("z".getBytes())); + Bytes.wrap("z".getBytes()), + false); iterator.currentIterator = segmentOne.all(); segmentOne.close(); assertFalse(iterator.hasNext()); } + @Test + public void shouldOnlyIterateOverSegmentsInBackwardRange() { + iterator = new SegmentIterator<>( + Arrays.asList(segmentOne, segmentTwo).iterator(), + hasNextCondition, + Bytes.wrap("a".getBytes()), + Bytes.wrap("b".getBytes()), + true); + + assertTrue(iterator.hasNext()); + assertEquals("b", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("b", "2"), toStringKeyValue(iterator.next())); + + assertTrue(iterator.hasNext()); + assertEquals("a", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("a", "1"), toStringKeyValue(iterator.next())); + + assertFalse(iterator.hasNext()); + } + @Test public void shouldOnlyIterateOverSegmentsInRange() { iterator = new SegmentIterator<>( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("a".getBytes()), - Bytes.wrap("b".getBytes())); + Bytes.wrap("b".getBytes()), + false); assertTrue(iterator.hasNext()); assertEquals("a", new String(iterator.peekNextKey().get())); @@ -145,7 +195,8 @@ public void shouldThrowNoSuchElementOnPeekNextKeyIfNoNext() { Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("f".getBytes()), - Bytes.wrap("h".getBytes())); + Bytes.wrap("h".getBytes()), + false); iterator.peekNextKey(); } @@ -156,7 +207,8 @@ public void shouldThrowNoSuchElementOnNextIfNoNext() { Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("f".getBytes()), - Bytes.wrap("h".getBytes())); + Bytes.wrap("h".getBytes()), + false); iterator.next(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java index 2ae671769909b..a275df74e77e9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java @@ -188,13 +188,33 @@ public void shouldGetSegmentsWithinTimeRange() { segments.getOrCreateSegmentIfLive(3, context, streamTime); segments.getOrCreateSegmentIfLive(4, context, streamTime); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); assertEquals(2, segments.get(2).id); } + @Test + public void shouldGetSegmentsWithinBackwardTimeRange() { + updateStreamTimeAndCreateSegment(0); + updateStreamTimeAndCreateSegment(1); + updateStreamTimeAndCreateSegment(2); + updateStreamTimeAndCreateSegment(3); + final long streamTime = updateStreamTimeAndCreateSegment(4); + segments.getOrCreateSegmentIfLive(0, context, streamTime); + segments.getOrCreateSegmentIfLive(1, context, streamTime); + segments.getOrCreateSegmentIfLive(2, context, streamTime); + segments.getOrCreateSegmentIfLive(3, context, streamTime); + segments.getOrCreateSegmentIfLive(4, context, streamTime); + + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); + assertEquals(3, segments.size()); + assertEquals(0, segments.get(2).id); + assertEquals(1, segments.get(1).id); + assertEquals(2, segments.get(0).id); + } + @Test public void shouldGetSegmentsWithinTimeRangeOutOfOrder() { updateStreamTimeAndCreateSegment(4); @@ -203,13 +223,28 @@ public void shouldGetSegmentsWithinTimeRangeOutOfOrder() { updateStreamTimeAndCreateSegment(1); updateStreamTimeAndCreateSegment(3); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); assertEquals(2, segments.get(2).id); } + @Test + public void shouldGetSegmentsWithinBackwardTimeRangeOutOfOrder() { + updateStreamTimeAndCreateSegment(4); + updateStreamTimeAndCreateSegment(2); + updateStreamTimeAndCreateSegment(0); + updateStreamTimeAndCreateSegment(1); + updateStreamTimeAndCreateSegment(3); + + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); + assertEquals(3, segments.size()); + assertEquals(0, segments.get(2).id); + assertEquals(1, segments.get(1).id); + assertEquals(2, segments.get(0).id); + } + @Test public void shouldRollSegments() { updateStreamTimeAndCreateSegment(0); @@ -310,7 +345,7 @@ public void shouldClearSegmentsOnClose() { } private void verifyCorrectSegments(final long first, final int numSegments) { - final List result = this.segments.segments(0, Long.MAX_VALUE); + final List result = this.segments.segments(0, Long.MAX_VALUE, false); assertEquals(numSegments, result.size()); for (int i = 0; i < numSegments; i++) { assertEquals(i + first, result.get(i).id); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java index 8824a9487d383..d566b6a01dc86 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java @@ -57,12 +57,12 @@ public class WindowKeySchemaTest { @Test public void testHasNextConditionUsingNullKeys() { final List> keys = Arrays.asList( - KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[]{0, 0}), new TimeWindow(0, 1)), 0), 1), - KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[]{0}), new TimeWindow(0, 1)), 0), 2), - KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[]{0, 0, 0}), new TimeWindow(0, 1)), 0), 3), - KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[]{0}), new TimeWindow(10, 20)), 4), 4), - KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[]{0, 0}), new TimeWindow(10, 20)), 5), 5), - KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[]{0, 0, 0}), new TimeWindow(10, 20)), 6), 6)); + KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0}), new TimeWindow(0, 1)), 0), 1), + KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0}), new TimeWindow(0, 1)), 0), 2), + KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0, 0}), new TimeWindow(0, 1)), 0), 3), + KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0}), new TimeWindow(10, 20)), 4), 4), + KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0}), new TimeWindow(10, 20)), 5), 5), + KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0, 0}), new TimeWindow(10, 20)), 6), 6)); final DelegatingPeekingKeyValueIterator iterator = new DelegatingPeekingKeyValueIterator<>("foo", new KeyValueIteratorStub<>(keys.iterator())); final HasNextCondition hasNextCondition = windowKeySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE); @@ -75,13 +75,13 @@ public void testHasNextConditionUsingNullKeys() { @Test public void testUpperBoundWithLargeTimestamps() { - final Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), Long.MAX_VALUE); + final Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), Long.MAX_VALUE); assertThat( "shorter key with max timestamp should be in range", upper.compareTo( WindowKeySchema.toStoreKeyBinary( - new byte[]{0xA}, + new byte[] {0xA}, Long.MAX_VALUE, Integer.MAX_VALUE ) @@ -92,87 +92,87 @@ public void testUpperBoundWithLargeTimestamps() { "shorter key with max timestamp should be in range", upper.compareTo( WindowKeySchema.toStoreKeyBinary( - new byte[]{0xA, 0xB}, + new byte[] {0xA, 0xB}, Long.MAX_VALUE, Integer.MAX_VALUE ) ) >= 0 ); - assertThat(upper, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[]{0xA}, Long.MAX_VALUE, Integer.MAX_VALUE))); + assertThat(upper, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[] {0xA}, Long.MAX_VALUE, Integer.MAX_VALUE))); } @Test public void testUpperBoundWithKeyBytesLargerThanFirstTimestampByte() { - final Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, (byte) 0x8F, (byte) 0x9F}), Long.MAX_VALUE); + final Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[] {0xA, (byte) 0x8F, (byte) 0x9F}), Long.MAX_VALUE); assertThat( "shorter key with max timestamp should be in range", upper.compareTo( WindowKeySchema.toStoreKeyBinary( - new byte[]{0xA, (byte) 0x8F}, + new byte[] {0xA, (byte) 0x8F}, Long.MAX_VALUE, Integer.MAX_VALUE ) ) >= 0 ); - assertThat(upper, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[]{0xA, (byte) 0x8F, (byte) 0x9F}, Long.MAX_VALUE, Integer.MAX_VALUE))); + assertThat(upper, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[] {0xA, (byte) 0x8F, (byte) 0x9F}, Long.MAX_VALUE, Integer.MAX_VALUE))); } @Test public void testUpperBoundWithKeyBytesLargerAndSmallerThanFirstTimestampByte() { - final Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[]{0xC, 0xC, 0x9}), 0x0AffffffffffffffL); + final Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[] {0xC, 0xC, 0x9}), 0x0AffffffffffffffL); assertThat( "shorter key with max timestamp should be in range", upper.compareTo( WindowKeySchema.toStoreKeyBinary( - new byte[]{0xC, 0xC}, + new byte[] {0xC, 0xC}, 0x0AffffffffffffffL, Integer.MAX_VALUE ) ) >= 0 ); - assertThat(upper, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[]{0xC, 0xC}, 0x0AffffffffffffffL, Integer.MAX_VALUE))); + assertThat(upper, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[] {0xC, 0xC}, 0x0AffffffffffffffL, Integer.MAX_VALUE))); } @Test public void testUpperBoundWithZeroTimestamp() { - final Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), 0); - assertThat(upper, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[]{0xA, 0xB, 0xC}, 0, Integer.MAX_VALUE))); + final Bytes upper = windowKeySchema.upperRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), 0); + assertThat(upper, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, 0, Integer.MAX_VALUE))); } @Test public void testLowerBoundWithZeroTimestamp() { - final Bytes lower = windowKeySchema.lowerRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), 0); - assertThat(lower, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[]{0xA, 0xB, 0xC}, 0, 0))); + final Bytes lower = windowKeySchema.lowerRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), 0); + assertThat(lower, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, 0, 0))); } @Test public void testLowerBoundWithMonZeroTimestamp() { - final Bytes lower = windowKeySchema.lowerRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), 42); - assertThat(lower, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[]{0xA, 0xB, 0xC}, 0, 0))); + final Bytes lower = windowKeySchema.lowerRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), 42); + assertThat(lower, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, 0, 0))); } @Test public void testLowerBoundMatchesTrailingZeros() { - final Bytes lower = windowKeySchema.lowerRange(Bytes.wrap(new byte[]{0xA, 0xB, 0xC}), Long.MAX_VALUE - 1); + final Bytes lower = windowKeySchema.lowerRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), Long.MAX_VALUE - 1); assertThat( "appending zeros to key should still be in range", lower.compareTo( - WindowKeySchema.toStoreKeyBinary( - new byte[]{0xA, 0xB, 0xC, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, - Long.MAX_VALUE - 1, - 0 + WindowKeySchema.toStoreKeyBinary( + new byte[] {0xA, 0xB, 0xC, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, + Long.MAX_VALUE - 1, + 0 ) ) < 0 ); - assertThat(lower, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[]{0xA, 0xB, 0xC}, 0, 0))); + assertThat(lower, equalTo(WindowKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, 0, 0))); } @Test @@ -187,7 +187,7 @@ public void shouldSerializeDeserialize() { public void testSerializeDeserializeOverflowWindowSize() { final byte[] bytes = keySerde.serializer().serialize(topic, windowedKey); final Windowed result = new TimeWindowedDeserializer<>(serde.deserializer(), Long.MAX_VALUE - 1) - .deserialize(topic, bytes); + .deserialize(topic, bytes); assertEquals(new Windowed<>(key, new TimeWindow(startTime, Long.MAX_VALUE)), result); } @@ -203,9 +203,9 @@ public void shouldSerializeDeserializeExpectedWindowSize() { public void shouldSerializeDeserializeExpectedChangelogWindowSize() { // Key-value containing serialized store key binary and the key's window size final List> keys = Arrays.asList( - KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[]{0}), new TimeWindow(0, 1)), 0), 1), - KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[]{0, 0}), new TimeWindow(0, 10)), 0), 10), - KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[]{0, 0, 0}), new TimeWindow(10, 30)), 6), 20)); + KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0}), new TimeWindow(0, 1)), 0), 1), + KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0}), new TimeWindow(0, 10)), 0), 10), + KeyValue.pair(WindowKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0, 0}), new TimeWindow(10, 30)), 6), 20)); final List results = new ArrayList<>(); for (final KeyValue keyValue : keys) { diff --git a/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java b/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java deleted file mode 100644 index 17c5e6814d018..0000000000000 --- a/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.test; - -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.state.KeyValueIterator; -import org.apache.kafka.streams.state.internals.SegmentedBytesStore; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - -public class SegmentedBytesStoreStub implements SegmentedBytesStore { - private Map store = new HashMap<>(); - public boolean fetchCalled; - public boolean allUpToCalled; - public boolean flushed; - public boolean closed; - public boolean initialized; - public boolean removeCalled; - public boolean putCalled; - public boolean getCalled; - - @Override - public String name() { - return ""; - } - - @Override - public void init(final ProcessorContext context, final StateStore root) { - initialized = true; - } - - @Override - public KeyValueIterator fetch(final Bytes key, final long from, final long to) { - return fetch(key, key, from, to); - } - - @Override - public KeyValueIterator fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) { - fetchCalled = true; - return new KeyValueIteratorStub<>(Collections.>emptyIterator()); - } - - @Override - public KeyValueIterator all() { - fetchCalled = true; - return new KeyValueIteratorStub<>(Collections.>emptyIterator()); - } - - @Override - public KeyValueIterator fetchAll(final long timeFrom, final long timeTo) { - fetchCalled = true; - return new KeyValueIteratorStub<>(Collections.>emptyIterator()); - } - - @Override - public void remove(final Bytes key) { - store.put(key, null); - removeCalled = true; - } - - @Override - public void put(final Bytes key, final byte[] value) { - store.put(key, value); - putCalled = true; - } - - @Override - public boolean isOpen() { - return false; - } - - - @Override - public byte[] get(final Bytes key) { - getCalled = true; - return store.get(key); - } - - @Override - public void flush() { - flushed = true; - } - - @Override - public void close() { - closed = true; - } - - @Override - public boolean persistent() { - return false; - } -} diff --git a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java index afc378541c118..22fa79fdda744 100644 --- a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java @@ -30,6 +30,7 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Properties; @@ -121,7 +122,7 @@ public static List> toList(final Iterator> } public static Set> toSet(final Iterator> iterator) { - final Set> results = new HashSet<>(); + final Set> results = new LinkedHashSet<>(); while (iterator.hasNext()) { results.add(iterator.next()); From a8fc3a44eb0f225a40ae8a2f5385397285697253 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 5 Aug 2020 19:12:56 +0100 Subject: [PATCH 08/29] small additions --- .../internals/AbstractReadOnlyDecorator.java | 25 ------------------- .../state/internals/SessionKeySchema.java | 5 ++-- 2 files changed, 3 insertions(+), 27 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java index 466877b9adebc..52d39e6aeb8c2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java @@ -253,13 +253,6 @@ public KeyValueIterator, AGG> findSessions(final K key, return wrapped().findSessions(key, earliestSessionEndTime, latestSessionStartTime); } - @Override - public KeyValueIterator, AGG> backwardFindSessions(final K key, - final long earliestSessionEndTime, - final long latestSessionStartTime) { - return wrapped().backwardFindSessions(key, earliestSessionEndTime, latestSessionStartTime); - } - @Override public KeyValueIterator, AGG> findSessions(final K keyFrom, final K keyTo, @@ -268,14 +261,6 @@ public KeyValueIterator, AGG> findSessions(final K keyFrom, return wrapped().findSessions(keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime); } - @Override - public KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, - final K keyTo, - final long earliestSessionEndTime, - final long latestSessionStartTime) { - return wrapped().backwardFindSessions(keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime); - } - @Override public void remove(final Windowed sessionKey) { throw new UnsupportedOperationException(ERROR_MESSAGE); @@ -297,21 +282,11 @@ public KeyValueIterator, AGG> fetch(final K key) { return wrapped().fetch(key); } - @Override - public KeyValueIterator, AGG> backwardFetch(final K key) { - return wrapped().backwardFetch(key); - } - @Override public KeyValueIterator, AGG> fetch(final K from, final K to) { return wrapped().fetch(from, to); } - @Override - public KeyValueIterator, AGG> backwardFetch(final K from, - final K to) { - return wrapped().backwardFetch(from, to); - } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java index 0c80da4f80880..568297e0d7825 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java @@ -87,8 +87,9 @@ public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom, final Bytes @Override public List segmentsToSearch(final Segments segments, final long from, - final long to) { - return segments.segments(from, Long.MAX_VALUE); + final long to, + final boolean backward) { + return segments.segments(from, Long.MAX_VALUE, backward); } private static K extractKey(final byte[] binaryKey, From 39405c6adfc0d330ee80c2ac1659365ec411fb82 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 5 Aug 2020 19:18:43 +0100 Subject: [PATCH 09/29] rearrange code --- .../internals/AbstractReadOnlyDecorator.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java index 52d39e6aeb8c2..0d9b1d39a228f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java @@ -16,9 +16,6 @@ */ package org.apache.kafka.streams.processor.internals; -import java.time.Instant; -import java.util.List; - import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; @@ -33,6 +30,9 @@ import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.internals.WrappedStateStore; +import java.time.Instant; +import java.util.List; + abstract class AbstractReadOnlyDecorator extends WrappedStateStore { static final String ERROR_MESSAGE = "Global store is read only"; @@ -199,9 +199,9 @@ public KeyValueIterator, V> fetch(final K from, @Override public KeyValueIterator, V> backwardFetch(final K from, - final K to, - final Instant timeFrom, - final Instant timeTo) { + final K to, + final Instant timeFrom, + final Instant timeTo) { return wrapped().backwardFetch(from, to, timeFrom, timeTo); } From 1893fc1eaaa8792478b5ef11e4fe2412e6e73dec Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 11 Aug 2020 13:15:30 +0100 Subject: [PATCH 10/29] improve time range comments --- .../streams/state/ReadOnlyWindowStore.java | 58 +++++++++---------- 1 file changed, 29 insertions(+), 29 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java index 9c44123866dd9..0fbf82588cb4d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java @@ -68,9 +68,9 @@ public interface ReadOnlyWindowStore { * available window to the newest/latest window. * * @param key the key to fetch - * @param timeFrom time range start (inclusive) - * @param timeTo time range end (inclusive) - * @return an iterator over key-value pairs {@code } + * @param timeFrom time range start (inclusive), where iteration starts. + * @param timeTo time range end (inclusive), where iteration ends. + * @return an iterator over key-value pairs {@code }, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for key. * @deprecated Use {@link #fetch(Object, Instant, Instant)} instead @@ -105,9 +105,9 @@ public interface ReadOnlyWindowStore { * available window to the newest/latest window. * * @param key the key to fetch - * @param from time range start (inclusive) - * @param to time range end (inclusive) - * @return an iterator over key-value pairs {@code } + * @param from time range start (inclusive), where iteration starts. + * @param to time range end (inclusive), where iteration ends. + * @return an iterator over key-value pairs {@code }, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} @@ -143,9 +143,9 @@ public interface ReadOnlyWindowStore { * available window to the oldest/earliest window. * * @param key the key to fetch - * @param from time range start (inclusive) - * @param to time range end (inclusive) - * @return an iterator over key-value pairs {@code } + * @param from time range start (inclusive), where iteration ends. + * @param to time range end (inclusive), where iteration starts. + * @return an iterator over key-value pairs {@code }, from end to beginning of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} @@ -161,9 +161,9 @@ default WindowStoreIterator backwardFetch(K key, Instant from, Instant to) th * * @param from the first key in the range * @param to the last key in the range - * @param timeFrom time range start (inclusive) - * @param timeTo time range end (inclusive) - * @return an iterator over windowed key-value pairs {@code , value>} + * @param timeFrom time range start (inclusive), where iteration starts. + * @param timeTo time range end (inclusive), where iteration ends. + * @return an iterator over windowed key-value pairs {@code , value>}, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for any key. * @deprecated Use {@link #fetch(Object, Object, Instant, Instant)} instead @@ -178,9 +178,9 @@ default WindowStoreIterator backwardFetch(K key, Instant from, Instant to) th * * @param from the first key in the range * @param to the last key in the range - * @param fromTime time range start (inclusive) - * @param toTime time range end (inclusive) - * @return an iterator over windowed key-value pairs {@code , value>} + * @param fromTime time range start (inclusive), where iteration starts. + * @param toTime time range end (inclusive), where iteration ends. + * @return an iterator over windowed key-value pairs {@code , value>}, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for any key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} @@ -196,9 +196,9 @@ KeyValueIterator, V> fetch(K from, K to, Instant fromTime, Instant t * * @param from the first key in the range * @param to the last key in the range - * @param fromTime time range start (inclusive) - * @param toTime time range end (inclusive) - * @return an iterator over windowed key-value pairs {@code , value>} + * @param fromTime time range start (inclusive), where iteration ends. + * @param toTime time range end (inclusive), where iteration starts. + * @return an iterator over windowed key-value pairs {@code , value>}, from end to beginning of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for any key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} @@ -211,7 +211,7 @@ default KeyValueIterator, V> backwardFetch(K from, K to, Instant fro /** * Gets all the key-value pairs in the existing windows. * - * @return an iterator over windowed key-value pairs {@code , value>} + * @return an iterator over windowed key-value pairs {@code , value>}, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized */ KeyValueIterator, V> all(); @@ -220,7 +220,7 @@ default KeyValueIterator, V> backwardFetch(K from, K to, Instant fro * Gets all the key-value pairs in the existing windows in backward order * with respect to time (from end to beginning of time). * - * @return an iterator over windowed key-value pairs {@code , value>} + * @return an backward iterator over windowed key-value pairs {@code , value>}, from the end to beginning of time. * @throws InvalidStateStoreException if the store is not initialized */ default KeyValueIterator, V> backwardAll() { @@ -230,9 +230,9 @@ default KeyValueIterator, V> backwardAll() { /** * Gets all the key-value pairs that belong to the windows within in the given time range. * - * @param timeFrom the beginning of the time slot from which to search (inclusive) - * @param timeTo the end of the time slot from which to search (inclusive) - * @return an iterator over windowed key-value pairs {@code , value>} + * @param timeFrom the beginning of the time slot from which to search (inclusive), where iteration starts. + * @param timeTo the end of the time slot from which to search (inclusive), where iteration ends. + * @return an iterator over windowed key-value pairs {@code , value>}, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException if {@code null} is used for any key * @deprecated Use {@link #fetchAll(Instant, Instant)} instead @@ -243,9 +243,9 @@ default KeyValueIterator, V> backwardAll() { /** * Gets all the key-value pairs that belong to the windows within in the given time range. * - * @param from the beginning of the time slot from which to search (inclusive) - * @param to the end of the time slot from which to search (inclusive) - * @return an iterator over windowed key-value pairs {@code , value>} + * @param from the beginning of the time slot from which to search (inclusive), where iteration starts. + * @param to the end of the time slot from which to search (inclusive), where iteration ends. + * @return an iterator over windowed key-value pairs {@code , value>}, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException if {@code null} is used for any key * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} @@ -256,9 +256,9 @@ default KeyValueIterator, V> backwardAll() { * Gets all the key-value pairs that belong to the windows within in the given time range in backward order * with respect to time (from end to beginning of time). * - * @param from the beginning of the time slot from which to search (inclusive) - * @param to the end of the time slot from which to search (inclusive) - * @return an iterator over windowed key-value pairs {@code , value>} + * @param from the beginning of the time slot from which to search (inclusive), where iteration ends. + * @param to the end of the time slot from which to search (inclusive), where iteration starts. + * @return an backward iterator over windowed key-value pairs {@code , value>}, from end to beginning of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException if {@code null} is used for any key * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} From f24e38ba26b5fd945df04e36afba12df08005783 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 11 Aug 2020 14:13:53 +0100 Subject: [PATCH 11/29] fix bytes range validator not needed --- .../kafka/streams/state/internals/CachingWindowStoreTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java index c083bd4bbbfb9..357179cd50281 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java @@ -797,7 +797,7 @@ public void shouldNotThrowInvalidRangeExceptionWithNegativeFromKey() { final Bytes keyFrom = Bytes.wrap(Serdes.Integer().serializer().serialize("", -1)); final Bytes keyTo = Bytes.wrap(Serdes.Integer().serializer().serialize("", 1)); - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(BytesRangeValidator.class)) { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(CachingWindowStore.class)) { final KeyValueIterator, byte[]> iterator = cachingStore.fetch(keyFrom, keyTo, 0L, 10L); assertFalse(iterator.hasNext()); @@ -817,7 +817,7 @@ public void shouldNotThrowInvalidBackwardRangeExceptionWithNegativeFromKey() { final Bytes keyFrom = Bytes.wrap(Serdes.Integer().serializer().serialize("", -1)); final Bytes keyTo = Bytes.wrap(Serdes.Integer().serializer().serialize("", 1)); - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(BytesRangeValidator.class)) { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(CachingWindowStore.class)) { final KeyValueIterator, byte[]> iterator = cachingStore.backwardFetch(keyFrom, keyTo, Instant.ofEpochMilli(0L), Instant.ofEpochMilli(10L)); assertFalse(iterator.hasNext()); From 2cf2297e96bed7d5af34983b5445fca9c7d7d35e Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 11 Aug 2020 14:33:54 +0100 Subject: [PATCH 12/29] fix cache iterator --- .../kafka/streams/state/internals/CachingWindowStore.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index cde0a2e55411f..8e6c5ed69b8c8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -448,12 +448,12 @@ public synchronized void close() { private class CacheIteratorWrapper implements PeekingKeyValueIterator { private final long segmentInterval; - private final Bytes keyFrom; private final Bytes keyTo; private final long timeTo; - private long lastSegmentId; + private final boolean reverse; + private long lastSegmentId; private long currentSegmentId; private Bytes cacheKeyFrom; private Bytes cacheKeyTo; @@ -476,6 +476,7 @@ private CacheIteratorWrapper(final Bytes keyFrom, this.keyTo = keyTo; this.timeTo = timeTo; this.lastSegmentId = cacheFunction.segmentId(Math.min(timeTo, maxObservedTimestamp.get())); + this.reverse = reverse; this.segmentInterval = cacheFunction.getSegmentInterval(); this.currentSegmentId = cacheFunction.segmentId(timeFrom); @@ -554,7 +555,8 @@ private void getNextSegmentIterator() { setCacheKeyRange(currentSegmentBeginTime(), currentSegmentLastTime()); current.close(); - current = context.cache().range(cacheName, cacheKeyFrom, cacheKeyTo); + if (reverse) current = context.cache().reverseRange(cacheName, cacheKeyFrom, cacheKeyTo); + else current = context.cache().range(cacheName, cacheKeyFrom, cacheKeyTo); } private void setCacheKeyRange(final long lowerRangeEndTime, final long upperRangeEndTime) { From 24b6e71cddcebb96645f7e7ba36f64403d9cc3fe Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 18 Aug 2020 10:04:31 +0100 Subject: [PATCH 13/29] fix syntax --- .../state/internals/AbstractSegments.java | 26 ++++++---- .../state/internals/CachingWindowStore.java | 15 ++++-- .../state/internals/InMemoryWindowStore.java | 52 ++++++++++++++----- .../state/internals/SegmentIterator.java | 14 +++-- 4 files changed, 77 insertions(+), 30 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java index 666a773374f0d..1b29c6f13ae43 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java @@ -121,14 +121,17 @@ public void openExisting(final ProcessorContext context, final long streamTime) public List segments(final long timeFrom, final long timeTo, final boolean backward) { final List result = new ArrayList<>(); final NavigableMap segmentsInRange; - if (backward) segmentsInRange = segments.subMap( - segmentId(timeFrom), true, - segmentId(timeTo), true - ).descendingMap(); - else segmentsInRange = segments.subMap( - segmentId(timeFrom), true, - segmentId(timeTo), true - ); + if (backward) { + segmentsInRange = segments.subMap( + segmentId(timeFrom), true, + segmentId(timeTo), true + ).descendingMap(); + } else { + segmentsInRange = segments.subMap( + segmentId(timeFrom), true, + segmentId(timeTo), true + ); + } for (final S segment : segmentsInRange.values()) { if (segment.isOpen()) { result.add(segment); @@ -141,8 +144,11 @@ public List segments(final long timeFrom, final long timeTo, final boolean ba public List allSegments(final boolean backward) { final List result = new ArrayList<>(); final Collection values; - if (backward) values = segments.descendingMap().values(); - else values = segments.values(); + if (backward) { + values = segments.descendingMap().values(); + } else { + values = segments.values(); + } for (final S segment : values) { if (segment.isOpen()) { result.add(segment); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index 8e6c5ed69b8c8..917792daa9db3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -483,8 +483,11 @@ private CacheIteratorWrapper(final Bytes keyFrom, setCacheKeyRange(timeFrom, currentSegmentLastTime()); - if (reverse) this.current = context.cache().reverseRange(cacheName, cacheKeyFrom, cacheKeyTo); - else this.current = context.cache().range(cacheName, cacheKeyFrom, cacheKeyTo); + if (reverse) { + this.current = context.cache().reverseRange(cacheName, cacheKeyFrom, cacheKeyTo); + } else { + this.current = context.cache().range(cacheName, cacheKeyFrom, cacheKeyTo); + } } @Override @@ -555,8 +558,12 @@ private void getNextSegmentIterator() { setCacheKeyRange(currentSegmentBeginTime(), currentSegmentLastTime()); current.close(); - if (reverse) current = context.cache().reverseRange(cacheName, cacheKeyFrom, cacheKeyTo); - else current = context.cache().range(cacheName, cacheKeyFrom, cacheKeyTo); + + if (reverse) { + current = context.cache().reverseRange(cacheName, cacheKeyFrom, cacheKeyTo); + } else { + current = context.cache().range(cacheName, cacheKeyFrom, cacheKeyTo); + } } private void setCacheKeyRange(final long lowerRangeEndTime, final long upperRangeEndTime) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java index aa6d4007e32fd..b3fb397bc466d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java @@ -187,10 +187,19 @@ WindowStoreIterator fetch(final Bytes key, final long timeFrom, final lo return WrappedInMemoryWindowStoreIterator.emptyIterator(); } - if (backward) return registerNewWindowStoreIterator( - key, segmentMap.subMap(minTime, true, timeTo, true).descendingMap().entrySet().iterator()); - else return registerNewWindowStoreIterator( - key, segmentMap.subMap(minTime, true, timeTo, true).entrySet().iterator()); + if (backward) { + return registerNewWindowStoreIterator( + key, + segmentMap.subMap(minTime, true, timeTo, true) + .descendingMap().entrySet().iterator() + ); + } else { + return registerNewWindowStoreIterator( + key, + segmentMap.subMap(minTime, true, timeTo, true) + .entrySet().iterator() + ); + } } @Deprecated @@ -237,10 +246,20 @@ KeyValueIterator, byte[]> fetch(final Bytes from, return KeyValueIterators.emptyIterator(); } - if (backward) return registerNewWindowedKeyValueIterator( - from, to, segmentMap.subMap(minTime, true, timeTo, true).descendingMap().entrySet().iterator()); - else return registerNewWindowedKeyValueIterator( - from, to, segmentMap.subMap(minTime, true, timeTo, true).entrySet().iterator()); + if (backward) { + return registerNewWindowedKeyValueIterator( + from, + to, + segmentMap.subMap(minTime, true, timeTo, true) + .descendingMap().entrySet().iterator()); + } else { + return registerNewWindowedKeyValueIterator( + from, + to, + segmentMap.subMap(minTime, true, timeTo, true) + .entrySet().iterator() + ); + } } @Deprecated @@ -266,10 +285,19 @@ KeyValueIterator, byte[]> fetchAll(final long timeFrom, final lo return KeyValueIterators.emptyIterator(); } - if (backward) return registerNewWindowedKeyValueIterator( - null, null, segmentMap.subMap(minTime, true, timeTo, true).descendingMap().entrySet().iterator()); - else return registerNewWindowedKeyValueIterator( - null, null, segmentMap.subMap(minTime, true, timeTo, true).entrySet().iterator()); + if (backward) { + return registerNewWindowedKeyValueIterator( + null, + null, + segmentMap.subMap(minTime, true, timeTo, true) + .descendingMap().entrySet().iterator()); + } else { + return registerNewWindowedKeyValueIterator( + null, + null, + segmentMap.subMap(minTime, true, timeTo, true) + .entrySet().iterator()); + } } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java index c0fbefbc4a7a7..edaef688ab36a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java @@ -75,11 +75,17 @@ public boolean hasNext() { currentSegment = segments.next(); try { if (from == null || to == null) { - if (reverse) currentIterator = currentSegment.reverseAll(); - else currentIterator = currentSegment.all(); + if (reverse) { + currentIterator = currentSegment.reverseAll(); + } else { + currentIterator = currentSegment.all(); + } } else { - if (reverse) currentIterator = currentSegment.reverseRange(from, to); - else currentIterator = currentSegment.range(from, to); + if (reverse) { + currentIterator = currentSegment.reverseRange(from, to); + } else { + currentIterator = currentSegment.range(from, to); + } } } catch (final InvalidStateStoreException e) { // segment may have been closed so we ignore it. From 56630d2a81cd4731382ebc1c12c971a3115c2620 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 18 Aug 2020 10:44:50 +0100 Subject: [PATCH 14/29] improve tests exception handling --- .../AbstractWindowBytesStoreTest.java | 17 +++++------ .../internals/CachingWindowStoreTest.java | 28 +++++++++---------- .../state/internals/SegmentIteratorTest.java | 9 +++--- .../apache/kafka/test/StreamsTestUtils.java | 7 ++--- 4 files changed, 31 insertions(+), 30 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java index c57576c42414d..84622150e56a2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java @@ -70,6 +70,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; public abstract class AbstractWindowBytesStoreTest { @@ -854,25 +855,25 @@ public void shouldReturnNullOnWindowNotFound() { assertNull(windowStore.fetch(1, 0L)); } - @Test(expected = NullPointerException.class) + @Test @SuppressWarnings("deprecation") public void shouldThrowNullPointerExceptionOnPutNullKey() { - windowStore.put(null, "anyValue"); + assertThrows(NullPointerException.class, () -> windowStore.put(null, "anyValue")); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerExceptionOnGetNullKey() { - windowStore.fetch(null, ofEpochMilli(1L), ofEpochMilli(2L)); + assertThrows(NullPointerException.class, () -> windowStore.fetch(null, ofEpochMilli(1L), ofEpochMilli(2L))); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerExceptionOnRangeNullFromKey() { - windowStore.fetch(null, 2, ofEpochMilli(1L), ofEpochMilli(2L)); + assertThrows(NullPointerException.class, () -> windowStore.fetch(null, 2, ofEpochMilli(1L), ofEpochMilli(2L))); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerExceptionOnRangeNullToKey() { - windowStore.fetch(1, null, ofEpochMilli(1L), ofEpochMilli(2L)); + assertThrows(NullPointerException.class, () -> windowStore.fetch(1, null, ofEpochMilli(1L), ofEpochMilli(2L))); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java index 357179cd50281..d4e5dab292543 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java @@ -604,23 +604,23 @@ public void shouldClearNamespaceCacheOnClose() { assertEquals(0, cache.size()); } - @Test(expected = InvalidStateStoreException.class) + @Test public void shouldThrowIfTryingToFetchFromClosedCachingStore() { cachingStore.close(); - cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(10)); + assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(10))); } - @Test(expected = InvalidStateStoreException.class) + @Test public void shouldThrowIfTryingToFetchRangeFromClosedCachingStore() { cachingStore.close(); - cachingStore.fetch(bytesKey("a"), bytesKey("b"), ofEpochMilli(0), ofEpochMilli(10)); + assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), bytesKey("b"), ofEpochMilli(0), ofEpochMilli(10))); } - @Test(expected = InvalidStateStoreException.class) + @Test @SuppressWarnings("deprecation") public void shouldThrowIfTryingToWriteToClosedCachingStore() { cachingStore.close(); - cachingStore.put(bytesKey("a"), bytesValue("a")); + assertThrows(InvalidStateStoreException.class, () -> cachingStore.put(bytesKey("a"), bytesValue("a"))); } @Test @@ -765,10 +765,10 @@ public void shouldReturnSameResultsForSingleKeyFetchAndEqualKeyRangeBackwardFetc assertFalse(keyRangeIterator.hasNext()); } - @Test(expected = NullPointerException.class) + @Test @SuppressWarnings("deprecation") public void shouldThrowNullPointerExceptionOnPutNullKey() { - cachingStore.put(null, bytesValue("anyValue")); + assertThrows(NullPointerException.class, () -> cachingStore.put(null, bytesValue("anyValue"))); } @Test @@ -777,19 +777,19 @@ public void shouldNotThrowNullPointerExceptionOnPutNullValue() { cachingStore.put(bytesKey("a"), null); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerExceptionOnFetchNullKey() { - cachingStore.fetch(null, ofEpochMilli(1L), ofEpochMilli(2L)); + assertThrows(NullPointerException.class, () -> cachingStore.fetch(null, ofEpochMilli(1L), ofEpochMilli(2L))); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerExceptionOnRangeNullFromKey() { - cachingStore.fetch(null, bytesKey("anyTo"), ofEpochMilli(1L), ofEpochMilli(2L)); + assertThrows(NullPointerException.class, () -> cachingStore.fetch(null, bytesKey("anyTo"), ofEpochMilli(1L), ofEpochMilli(2L))); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerExceptionOnRangeNullToKey() { - cachingStore.fetch(bytesKey("anyFrom"), null, ofEpochMilli(1L), ofEpochMilli(2L)); + assertThrows(NullPointerException.class, () -> cachingStore.fetch(bytesKey("anyFrom"), null, ofEpochMilli(1L), ofEpochMilli(2L))); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java index 74ef7b7a531c9..b639e4aef9540 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java @@ -37,6 +37,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; public class SegmentIteratorTest { @@ -189,7 +190,7 @@ public void shouldOnlyIterateOverSegmentsInRange() { assertFalse(iterator.hasNext()); } - @Test(expected = NoSuchElementException.class) + @Test public void shouldThrowNoSuchElementOnPeekNextKeyIfNoNext() { iterator = new SegmentIterator<>( Arrays.asList(segmentOne, segmentTwo).iterator(), @@ -198,10 +199,10 @@ public void shouldThrowNoSuchElementOnPeekNextKeyIfNoNext() { Bytes.wrap("h".getBytes()), false); - iterator.peekNextKey(); + assertThrows(NoSuchElementException.class, () -> iterator.peekNextKey()); } - @Test(expected = NoSuchElementException.class) + @Test public void shouldThrowNoSuchElementOnNextIfNoNext() { iterator = new SegmentIterator<>( Arrays.asList(segmentOne, segmentTwo).iterator(), @@ -210,7 +211,7 @@ public void shouldThrowNoSuchElementOnNextIfNoNext() { Bytes.wrap("h".getBytes()), false); - iterator.next(); + assertThrows(NoSuchElementException.class, () -> iterator.next()); } private KeyValue toStringKeyValue(final KeyValue binaryKv) { diff --git a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java index 22fa79fdda744..b0360f0f572ca 100644 --- a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java @@ -30,7 +30,6 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Properties; @@ -64,8 +63,8 @@ public static Properties getStreamsConfig(final String applicationId, return props; } - public static Properties getStreamsConfig(final Serde keyDeserializer, - final Serde valueDeserializer) { + public static Properties getStreamsConfig(final Serde keyDeserializer, + final Serde valueDeserializer) { return getStreamsConfig( UUID.randomUUID().toString(), "localhost:9091", @@ -122,7 +121,7 @@ public static List> toList(final Iterator> } public static Set> toSet(final Iterator> iterator) { - final Set> results = new LinkedHashSet<>(); + final Set> results = new HashSet<>(); while (iterator.hasNext()) { results.add(iterator.next()); From 92e39ff2375e0b4ebd7aca9eeacfb208b34ef0da Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 18 Aug 2020 10:58:49 +0100 Subject: [PATCH 15/29] fix ordered set --- .../src/test/java/org/apache/kafka/test/StreamsTestUtils.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java index b0360f0f572ca..26822a4a9631d 100644 --- a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java @@ -30,6 +30,7 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Properties; @@ -121,7 +122,7 @@ public static List> toList(final Iterator> } public static Set> toSet(final Iterator> iterator) { - final Set> results = new HashSet<>(); + final Set> results = new LinkedHashSet<>(); while (iterator.hasNext()) { results.add(iterator.next()); From 2a428d3d6abc5c2bc4f8210950269cd3a1f65941 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 20 Aug 2020 17:01:06 +0100 Subject: [PATCH 16/29] replace reverse with forward --- .../AbstractRocksDBSegmentedBytesStore.java | 20 ++++++------- .../state/internals/CachingWindowStore.java | 30 +++++++++---------- .../state/internals/SegmentIterator.java | 18 +++++------ ...rappedWindowStoreKeyValueIteratorTest.java | 4 +-- .../state/internals/SegmentIteratorTest.java | 15 +++++----- 5 files changed, 44 insertions(+), 43 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java index 28107dc0e97e6..60d0c65f253b3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java @@ -76,8 +76,8 @@ public KeyValueIterator backwardFetch(final Bytes key, KeyValueIterator fetch(final Bytes key, final long from, final long to, - final boolean backward) { - final List searchSpace = keySchema.segmentsToSearch(segments, from, to, backward); + final boolean forward) { + final List searchSpace = keySchema.segmentsToSearch(segments, from, to, forward); final Bytes binaryFrom = keySchema.lowerRangeFixedSize(key, from); final Bytes binaryTo = keySchema.upperRangeFixedSize(key, to); @@ -87,7 +87,7 @@ KeyValueIterator fetch(final Bytes key, keySchema.hasNextCondition(key, key, from, to), binaryFrom, binaryTo, - backward); + forward); } @Override @@ -110,7 +110,7 @@ KeyValueIterator fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to, - final boolean backward) { + final boolean forward) { if (keyFrom.compareTo(keyTo) > 0) { LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + "This may be due to range arguments set in the wrong order, " + @@ -119,7 +119,7 @@ KeyValueIterator fetch(final Bytes keyFrom, return KeyValueIterators.emptyIterator(); } - final List searchSpace = keySchema.segmentsToSearch(segments, from, to, backward); + final List searchSpace = keySchema.segmentsToSearch(segments, from, to, forward); final Bytes binaryFrom = keySchema.lowerRange(keyFrom, from); final Bytes binaryTo = keySchema.upperRange(keyTo, to); @@ -129,7 +129,7 @@ KeyValueIterator fetch(final Bytes keyFrom, keySchema.hasNextCondition(keyFrom, keyTo, from, to), binaryFrom, binaryTo, - backward); + forward); } @Override @@ -141,7 +141,7 @@ public KeyValueIterator all() { keySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE), null, null, - false); + true); } @Override @@ -153,7 +153,7 @@ public KeyValueIterator backwardAll() { keySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE), null, null, - true); + false); } @Override @@ -166,7 +166,7 @@ public KeyValueIterator fetchAll(final long timeFrom, keySchema.hasNextCondition(null, null, timeFrom, timeTo), null, null, - false); + true); } @Override @@ -179,7 +179,7 @@ public KeyValueIterator backwardFetchAll(final long timeFrom, keySchema.hasNextCondition(null, null, timeFrom, timeTo), null, null, - true); + false); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index 917792daa9db3..f2756b6f22124 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -204,7 +204,7 @@ public synchronized WindowStoreIterator fetch(final Bytes key, } final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? - new CacheIteratorWrapper(key, timeFrom, timeTo, false) : + new CacheIteratorWrapper(key, timeFrom, timeTo, true) : context.cache().range( cacheName, cacheFunction.cacheKey(keySchema.lowerRangeFixedSize(key, timeFrom)), @@ -235,7 +235,7 @@ public synchronized WindowStoreIterator backwardFetch(final Bytes key, } final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? - new CacheIteratorWrapper(key, timeFrom, timeTo, true) : + new CacheIteratorWrapper(key, timeFrom, timeTo, false) : context.cache().reverseRange( cacheName, cacheFunction.cacheKey(keySchema.lowerRangeFixedSize(key, timeFrom)), @@ -274,7 +274,7 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, } final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? - new CacheIteratorWrapper(from, to, timeFrom, timeTo, false) : + new CacheIteratorWrapper(from, to, timeFrom, timeTo, true) : context.cache().range( cacheName, cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), @@ -320,7 +320,7 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes from, } final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? - new CacheIteratorWrapper(from, to, timeFrom, timeTo, true) : + new CacheIteratorWrapper(from, to, timeFrom, timeTo, false) : context.cache().reverseRange( cacheName, cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), @@ -451,7 +451,7 @@ private class CacheIteratorWrapper implements PeekingKeyValueIterator implements KeyValueIterator segments; protected final HasNextCondition hasNextCondition; @@ -42,12 +42,12 @@ class SegmentIterator implements KeyValueIterator convertWindowedKey(final Windowed bytesWindowed) private MergedSortedCacheWindowStoreKeyValueIterator createIterator( final Iterator, byte[]>> storeKvs, final Iterator> cacheKvs, - final boolean reverse + final boolean forward ) { final DelegatingPeekingKeyValueIterator, byte[]> storeIterator = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(storeKvs)); @@ -187,7 +187,7 @@ private MergedSortedCacheWindowStoreKeyValueIterator createIterator( new StateSerdes<>("name", Serdes.Bytes(), Serdes.ByteArray()), WINDOW_SIZE, SINGLE_SEGMENT_CACHE_FUNCTION, - reverse + forward ); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java index b639e4aef9540..56ccb501682ef 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java @@ -87,7 +87,8 @@ public void shouldIterateOverAllSegments() { Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("a".getBytes()), - Bytes.wrap("z".getBytes()), false); + Bytes.wrap("z".getBytes()), + true); assertTrue(iterator.hasNext()); assertEquals("a", new String(iterator.peekNextKey().get())); @@ -115,7 +116,7 @@ public void shouldIterateBackwardOverAllSegments() { hasNextCondition, Bytes.wrap("a".getBytes()), Bytes.wrap("z".getBytes()), - true); + false); assertTrue(iterator.hasNext()); assertEquals("b", new String(iterator.peekNextKey().get())); @@ -143,7 +144,7 @@ public void shouldNotThrowExceptionOnHasNextWhenStoreClosed() { hasNextCondition, Bytes.wrap("a".getBytes()), Bytes.wrap("z".getBytes()), - false); + true); iterator.currentIterator = segmentOne.all(); segmentOne.close(); @@ -157,7 +158,7 @@ public void shouldOnlyIterateOverSegmentsInBackwardRange() { hasNextCondition, Bytes.wrap("a".getBytes()), Bytes.wrap("b".getBytes()), - true); + false); assertTrue(iterator.hasNext()); assertEquals("b", new String(iterator.peekNextKey().get())); @@ -177,7 +178,7 @@ public void shouldOnlyIterateOverSegmentsInRange() { hasNextCondition, Bytes.wrap("a".getBytes()), Bytes.wrap("b".getBytes()), - false); + true); assertTrue(iterator.hasNext()); assertEquals("a", new String(iterator.peekNextKey().get())); @@ -197,7 +198,7 @@ public void shouldThrowNoSuchElementOnPeekNextKeyIfNoNext() { hasNextCondition, Bytes.wrap("f".getBytes()), Bytes.wrap("h".getBytes()), - false); + true); assertThrows(NoSuchElementException.class, () -> iterator.peekNextKey()); } @@ -209,7 +210,7 @@ public void shouldThrowNoSuchElementOnNextIfNoNext() { hasNextCondition, Bytes.wrap("f".getBytes()), Bytes.wrap("h".getBytes()), - false); + true); assertThrows(NoSuchElementException.class, () -> iterator.next()); } From 0871d19e1ef14a5fd56a56c8ec9725cb8fcbd29a Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 20 Aug 2020 17:36:46 +0100 Subject: [PATCH 17/29] replace backward with forward flag --- .../AbstractRocksDBSegmentedBytesStore.java | 16 ++++---- .../state/internals/AbstractSegments.java | 16 ++++---- .../state/internals/CachingWindowStore.java | 16 ++++---- .../state/internals/InMemoryWindowStore.java | 39 +++++++++---------- .../state/internals/SegmentedBytesStore.java | 2 +- .../streams/state/internals/Segments.java | 4 +- .../state/internals/SessionKeySchema.java | 4 +- .../state/internals/WindowKeySchema.java | 4 +- .../state/internals/KeyValueSegmentsTest.java | 10 ++--- ...edCacheWrappedWindowStoreIteratorTest.java | 36 ++++++++++++----- ...rappedWindowStoreKeyValueIteratorTest.java | 4 +- .../internals/TimestampedSegmentsTest.java | 10 ++--- 12 files changed, 88 insertions(+), 73 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java index 60d0c65f253b3..b740bd93d4ca9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java @@ -63,14 +63,14 @@ public class AbstractRocksDBSegmentedBytesStore implements Se public KeyValueIterator fetch(final Bytes key, final long from, final long to) { - return fetch(key, from, to, false); + return fetch(key, from, to, true); } @Override public KeyValueIterator backwardFetch(final Bytes key, final long from, final long to) { - return fetch(key, from, to, true); + return fetch(key, from, to, false); } KeyValueIterator fetch(final Bytes key, @@ -95,7 +95,7 @@ public KeyValueIterator fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) { - return fetch(keyFrom, keyTo, from, to, false); + return fetch(keyFrom, keyTo, from, to, true); } @Override @@ -103,7 +103,7 @@ public KeyValueIterator backwardFetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) { - return fetch(keyFrom, keyTo, from, to, true); + return fetch(keyFrom, keyTo, from, to, false); } KeyValueIterator fetch(final Bytes keyFrom, @@ -134,7 +134,7 @@ KeyValueIterator fetch(final Bytes keyFrom, @Override public KeyValueIterator all() { - final List searchSpace = segments.allSegments(false); + final List searchSpace = segments.allSegments(true); return new SegmentIterator<>( searchSpace.iterator(), @@ -146,7 +146,7 @@ public KeyValueIterator all() { @Override public KeyValueIterator backwardAll() { - final List searchSpace = segments.allSegments(true); + final List searchSpace = segments.allSegments(false); return new SegmentIterator<>( searchSpace.iterator(), @@ -159,7 +159,7 @@ public KeyValueIterator backwardAll() { @Override public KeyValueIterator fetchAll(final long timeFrom, final long timeTo) { - final List searchSpace = segments.segments(timeFrom, timeTo, false); + final List searchSpace = segments.segments(timeFrom, timeTo, true); return new SegmentIterator<>( searchSpace.iterator(), @@ -172,7 +172,7 @@ public KeyValueIterator fetchAll(final long timeFrom, @Override public KeyValueIterator backwardFetchAll(final long timeFrom, final long timeTo) { - final List searchSpace = segments.segments(timeFrom, timeTo, true); + final List searchSpace = segments.segments(timeFrom, timeTo, false); return new SegmentIterator<>( searchSpace.iterator(), diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java index 1b29c6f13ae43..4b59c95ff2c91 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java @@ -118,19 +118,19 @@ public void openExisting(final ProcessorContext context, final long streamTime) } @Override - public List segments(final long timeFrom, final long timeTo, final boolean backward) { + public List segments(final long timeFrom, final long timeTo, final boolean forward) { final List result = new ArrayList<>(); final NavigableMap segmentsInRange; - if (backward) { + if (forward) { segmentsInRange = segments.subMap( segmentId(timeFrom), true, segmentId(timeTo), true - ).descendingMap(); + ); } else { segmentsInRange = segments.subMap( segmentId(timeFrom), true, segmentId(timeTo), true - ); + ).descendingMap(); } for (final S segment : segmentsInRange.values()) { if (segment.isOpen()) { @@ -141,13 +141,13 @@ public List segments(final long timeFrom, final long timeTo, final boolean ba } @Override - public List allSegments(final boolean backward) { + public List allSegments(final boolean forward) { final List result = new ArrayList<>(); final Collection values; - if (backward) { - values = segments.descendingMap().values(); - } else { + if (forward) { values = segments.values(); + } else { + values = segments.descendingMap().values(); } for (final S segment : values) { if (segment.isOpen()) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index f2756b6f22124..cfcd15f3c007d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -216,7 +216,7 @@ public synchronized WindowStoreIterator fetch(final Bytes key, cacheIterator, hasNextCondition, cacheFunction ); - return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator, false); + return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator, true); } @Override @@ -246,7 +246,7 @@ public synchronized WindowStoreIterator backwardFetch(final Bytes key, final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); - return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator, true); + return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator, false); } @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @@ -290,7 +290,7 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, bytesSerdes, windowSize, cacheFunction, - false + true ); } @@ -336,7 +336,7 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes from, bytesSerdes, windowSize, cacheFunction, - true + false ); } @@ -358,7 +358,7 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, bytesSerdes, windowSize, cacheFunction, - false + true ); } @@ -382,7 +382,7 @@ public KeyValueIterator, byte[]> backwardFetchAll(final Instant bytesSerdes, windowSize, cacheFunction, - true + false ); } @@ -399,7 +399,7 @@ public KeyValueIterator, byte[]> all() { bytesSerdes, windowSize, cacheFunction, - false + true ); } @@ -416,7 +416,7 @@ public KeyValueIterator, byte[]> backwardAll() { bytesSerdes, windowSize, cacheFunction, - true + false ); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java index b3fb397bc466d..e614cd3eea3ae 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java @@ -145,7 +145,6 @@ public void put(final Bytes key, final byte[] value, final long windowStartTimes @Override public byte[] fetch(final Bytes key, final long windowStartTimestamp) { - Objects.requireNonNull(key, "key cannot be null"); removeExpiredSegments(); @@ -165,17 +164,17 @@ public byte[] fetch(final Bytes key, final long windowStartTimestamp) { @Deprecated @Override public WindowStoreIterator fetch(final Bytes key, final long timeFrom, final long timeTo) { - return fetch(key, timeFrom, timeTo, false); + return fetch(key, timeFrom, timeTo, true); } @Override public WindowStoreIterator backwardFetch(final Bytes key, final Instant from, final Instant to) { final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); - return fetch(key, timeFrom, timeTo, true); + return fetch(key, timeFrom, timeTo, false); } - WindowStoreIterator fetch(final Bytes key, final long timeFrom, final long timeTo, final boolean backward) { + WindowStoreIterator fetch(final Bytes key, final long timeFrom, final long timeTo, final boolean forward) { Objects.requireNonNull(key, "key cannot be null"); removeExpiredSegments(); @@ -187,17 +186,17 @@ WindowStoreIterator fetch(final Bytes key, final long timeFrom, final lo return WrappedInMemoryWindowStoreIterator.emptyIterator(); } - if (backward) { + if (forward) { return registerNewWindowStoreIterator( key, segmentMap.subMap(minTime, true, timeTo, true) - .descendingMap().entrySet().iterator() + .entrySet().iterator() ); } else { return registerNewWindowStoreIterator( key, segmentMap.subMap(minTime, true, timeTo, true) - .entrySet().iterator() + .descendingMap().entrySet().iterator() ); } } @@ -208,7 +207,7 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to, final long timeFrom, final long timeTo) { - return fetch(from, to, timeFrom, timeTo, false); + return fetch(from, to, timeFrom, timeTo, true); } @Override @@ -218,14 +217,14 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Instant toTime) { final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")); final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime")); - return fetch(from, to, timeFrom, timeTo, true); + return fetch(from, to, timeFrom, timeTo, false); } KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to, final long timeFrom, final long timeTo, - final boolean backward) { + final boolean forward) { Objects.requireNonNull(from, "from key cannot be null"); Objects.requireNonNull(to, "to key cannot be null"); @@ -246,36 +245,36 @@ KeyValueIterator, byte[]> fetch(final Bytes from, return KeyValueIterators.emptyIterator(); } - if (backward) { + if (forward) { return registerNewWindowedKeyValueIterator( from, to, segmentMap.subMap(minTime, true, timeTo, true) - .descendingMap().entrySet().iterator()); + .entrySet().iterator() + ); } else { return registerNewWindowedKeyValueIterator( from, to, segmentMap.subMap(minTime, true, timeTo, true) - .entrySet().iterator() - ); + .descendingMap().entrySet().iterator()); } } @Deprecated @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { - return fetchAll(timeFrom, timeTo, false); + return fetchAll(timeFrom, timeTo, true); } @Override public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, final Instant to) { final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); - return fetchAll(timeFrom, timeTo, true); + return fetchAll(timeFrom, timeTo, false); } - KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo, final boolean backward) { + KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo, final boolean forward) { removeExpiredSegments(); // add one b/c records expire exactly retentionPeriod ms after created @@ -285,18 +284,18 @@ KeyValueIterator, byte[]> fetchAll(final long timeFrom, final lo return KeyValueIterators.emptyIterator(); } - if (backward) { + if (forward) { return registerNewWindowedKeyValueIterator( null, null, segmentMap.subMap(minTime, true, timeTo, true) - .descendingMap().entrySet().iterator()); + .entrySet().iterator()); } else { return registerNewWindowedKeyValueIterator( null, null, segmentMap.subMap(minTime, true, timeTo, true) - .entrySet().iterator()); + .descendingMap().entrySet().iterator()); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java index 07082e8ea3631..79ada1f07db05 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java @@ -202,6 +202,6 @@ interface KeySchema { * @param to * @return List of segments to search */ - List segmentsToSearch(Segments segments, long from, long to, boolean backward); + List segmentsToSearch(Segments segments, long from, long to, boolean forward); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java index 38a5d318bcf66..7e50b98452165 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java @@ -34,9 +34,9 @@ interface Segments { void openExisting(final ProcessorContext context, final long streamTime); - List segments(final long timeFrom, final long timeTo, final boolean backward); + List segments(final long timeFrom, final long timeTo, final boolean forward); - List allSegments(final boolean backward); + List allSegments(final boolean forward); void flush(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java index 568297e0d7825..326b86945f8e4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java @@ -88,8 +88,8 @@ public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom, final Bytes public List segmentsToSearch(final Segments segments, final long from, final long to, - final boolean backward) { - return segments.segments(from, Long.MAX_VALUE, backward); + final boolean forward) { + return segments.segments(from, Long.MAX_VALUE, forward); } private static K extractKey(final byte[] binaryKey, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java index a675ae490f8c9..a467af8cf5a1f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java @@ -94,8 +94,8 @@ public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom, public List segmentsToSearch(final Segments segments, final long from, final long to, - final boolean backward) { - return segments.segments(from, to, backward); + final boolean forward) { + return segments.segments(from, to, forward); } /** diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java index 442723d0c7459..aeef8ce8e13f5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java @@ -187,7 +187,7 @@ public void shouldGetSegmentsWithinTimeRange() { segments.getOrCreateSegmentIfLive(3, context, streamTime); segments.getOrCreateSegmentIfLive(4, context, streamTime); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); @@ -207,7 +207,7 @@ public void shouldGetSegmentsWithinBackwardTimeRange() { segments.getOrCreateSegmentIfLive(3, context, streamTime); segments.getOrCreateSegmentIfLive(4, context, streamTime); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); assertEquals(3, segments.size()); assertEquals(0, segments.get(2).id); assertEquals(1, segments.get(1).id); @@ -222,7 +222,7 @@ public void shouldGetSegmentsWithinTimeRangeOutOfOrder() { updateStreamTimeAndCreateSegment(1); updateStreamTimeAndCreateSegment(3); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); @@ -237,7 +237,7 @@ public void shouldGetSegmentsWithinTimeBackwardRangeOutOfOrder() { updateStreamTimeAndCreateSegment(1); updateStreamTimeAndCreateSegment(3); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); assertEquals(3, segments.size()); assertEquals(2, segments.get(0).id); assertEquals(1, segments.get(1).id); @@ -344,7 +344,7 @@ public void shouldClearSegmentsOnClose() { } private void verifyCorrectSegments(final long first, final int numSegments) { - final List result = this.segments.segments(0, Long.MAX_VALUE, false); + final List result = this.segments.segments(0, Long.MAX_VALUE, true); assertEquals(numSegments, result.size()); for (int i = 0; i < numSegments; i++) { assertEquals(i + first, result.get(i).id); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java index 7559b846badbf..0d69d933cf9f6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java @@ -73,7 +73,7 @@ public void shouldIterateOverValueFromBothIterators() { ); final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( - cacheIterator, storeIterator, false + cacheIterator, storeIterator, true ); int index = 0; while (iterator.hasNext()) { @@ -111,7 +111,7 @@ public void shouldReverseIterateOverValueFromBothIterators() { ); final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( - cacheIterator, storeIterator, true + cacheIterator, storeIterator, false ); int index = 0; Collections.reverse(expectedKvPairs); @@ -135,7 +135,7 @@ public void shouldPeekNextStoreKey() { namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes) ); final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( - cacheIterator, storeIterator, false + cacheIterator, storeIterator, true ); assertThat(iterator.peekNextKey(), equalTo(0L)); iterator.next(); @@ -156,7 +156,7 @@ public void shouldPeekNextStoreKeyReverse() { SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes) ); final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( - cacheIterator, storeIterator, true + cacheIterator, storeIterator, false ); assertThat(iterator.peekNextKey(), equalTo(10L)); iterator.next(); @@ -170,9 +170,18 @@ public void shouldPeekNextCacheKey() { cache.put(namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(WindowKeySchema.toStoreKeyBinary("a", 10L, 0, stateSerdes)), new LRUCacheEntry("b".getBytes())); final Bytes fromBytes = WindowKeySchema.toStoreKeyBinary("a", 0, 0, stateSerdes); final Bytes toBytes = WindowKeySchema.toStoreKeyBinary("a", 100, 0, stateSerdes); - final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); - final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes)); - final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator(cacheIterator, storeIterator, false); + final KeyValueIterator storeIterator = + new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range( + namespace, + SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), + SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes) + ); + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( + cacheIterator, + storeIterator, + true + ); assertThat(iterator.peekNextKey(), equalTo(0L)); iterator.next(); assertThat(iterator.peekNextKey(), equalTo(10L)); @@ -187,9 +196,16 @@ public void shouldPeekNextCacheKeyReverse() { final Bytes toBytes = WindowKeySchema.toStoreKeyBinary("a", 100, 0, stateSerdes); final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); - final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = - cache.reverseRange(namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes)); - final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator(cacheIterator, storeIterator, true); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.reverseRange( + namespace, + SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), + SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes) + ); + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( + cacheIterator, + storeIterator, + false + ); assertThat(iterator.peekNextKey(), equalTo(10L)); iterator.next(); assertThat(iterator.peekNextKey(), equalTo(0L)); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java index 1fc6c07507312..63cc7f7f2db5a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java @@ -146,7 +146,7 @@ public void shouldPeekNextKeyFromReverseCache() { @Test public void shouldIterateBothStoreAndCache() { - final MergedSortedCacheWindowStoreKeyValueIterator iterator = createIterator(storeKvs, cacheKvs, false); + final MergedSortedCacheWindowStoreKeyValueIterator iterator = createIterator(storeKvs, cacheKvs, true); assertThat(convertKeyValuePair(iterator.next()), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); assertThat(convertKeyValuePair(iterator.next()), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); assertFalse(iterator.hasNext()); @@ -154,7 +154,7 @@ public void shouldIterateBothStoreAndCache() { @Test public void shouldReverseIterateBothStoreAndCache() { - final MergedSortedCacheWindowStoreKeyValueIterator iterator = createIterator(storeKvs, cacheKvs, true); + final MergedSortedCacheWindowStoreKeyValueIterator iterator = createIterator(storeKvs, cacheKvs, false); assertThat(convertKeyValuePair(iterator.next()), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); assertThat(convertKeyValuePair(iterator.next()), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); assertFalse(iterator.hasNext()); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java index a275df74e77e9..558f1c9471b1e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java @@ -188,7 +188,7 @@ public void shouldGetSegmentsWithinTimeRange() { segments.getOrCreateSegmentIfLive(3, context, streamTime); segments.getOrCreateSegmentIfLive(4, context, streamTime); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); @@ -208,7 +208,7 @@ public void shouldGetSegmentsWithinBackwardTimeRange() { segments.getOrCreateSegmentIfLive(3, context, streamTime); segments.getOrCreateSegmentIfLive(4, context, streamTime); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); assertEquals(3, segments.size()); assertEquals(0, segments.get(2).id); assertEquals(1, segments.get(1).id); @@ -223,7 +223,7 @@ public void shouldGetSegmentsWithinTimeRangeOutOfOrder() { updateStreamTimeAndCreateSegment(1); updateStreamTimeAndCreateSegment(3); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); @@ -238,7 +238,7 @@ public void shouldGetSegmentsWithinBackwardTimeRangeOutOfOrder() { updateStreamTimeAndCreateSegment(1); updateStreamTimeAndCreateSegment(3); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); assertEquals(3, segments.size()); assertEquals(0, segments.get(2).id); assertEquals(1, segments.get(1).id); @@ -345,7 +345,7 @@ public void shouldClearSegmentsOnClose() { } private void verifyCorrectSegments(final long first, final int numSegments) { - final List result = this.segments.segments(0, Long.MAX_VALUE, false); + final List result = this.segments.segments(0, Long.MAX_VALUE, true); assertEquals(numSegments, result.size()); for (int i = 0; i < numSegments; i++) { assertEquals(i + first, result.get(i).id); From 07557b40127cfbb457686dc1bc262492546bface Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 24 Aug 2020 22:18:01 +0100 Subject: [PATCH 18/29] apply suggestions --- .../internals/AbstractReadOnlyDecorator.java | 1 - ...bstractMergedSortedCacheStoreIterator.java | 2 +- .../AbstractRocksDBSegmentedBytesStore.java | 5 ++- .../state/internals/CachingWindowStore.java | 15 ++++----- .../internals/InMemoryKeyValueStore.java | 2 +- .../state/internals/InMemoryWindowStore.java | 14 +++++---- .../streams/kstream/SessionWindowsTest.java | 9 +++--- .../internals/AbstractKeyValueStoreTest.java | 31 ++++++++++--------- .../internals/CachingWindowStoreTest.java | 5 +-- .../CompositeReadOnlyWindowStoreTest.java | 21 +++++++------ ...DelegatingPeekingKeyValueIteratorTest.java | 9 +++--- ...rappedWindowStoreKeyValueIteratorTest.java | 10 +++--- .../state/internals/SegmentIteratorTest.java | 18 +++++------ .../state/internals/ThreadCacheTest.java | 10 +++--- .../TimestampedKeyValueStoreBuilderTest.java | 21 +++++++------ .../TimestampedWindowStoreBuilderTest.java | 17 +++++----- 16 files changed, 98 insertions(+), 92 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java index 0d9b1d39a228f..de4e1fa7cc018 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java @@ -287,6 +287,5 @@ public KeyValueIterator, AGG> fetch(final K from, final K to) { return wrapped().fetch(from, to); } - } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java index 40ab65dafccc3..819c58ca20e99 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java @@ -191,4 +191,4 @@ public void close() { cacheIterator.close(); storeIterator.close(); } -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java index b740bd93d4ca9..2e9882d0049e9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java @@ -40,15 +40,18 @@ public class AbstractRocksDBSegmentedBytesStore implements SegmentedBytesStore { private static final Logger LOG = LoggerFactory.getLogger(AbstractRocksDBSegmentedBytesStore.class); + private final String name; private final AbstractSegments segments; private final String metricScope; private final KeySchema keySchema; + private ProcessorContext context; - private volatile boolean open; private Sensor expiredRecordSensor; private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP; + private volatile boolean open; + AbstractRocksDBSegmentedBytesStore(final String name, final String metricScope, final KeySchema keySchema, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index cfcd15f3c007d..77a862221842d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -81,7 +81,6 @@ public void init(final ProcessorContext context, final StateStore root) { super.init(context, root); } - @SuppressWarnings("unchecked") private void initInternal(final InternalProcessorContext context) { this.context = context; final String topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), name()); @@ -212,9 +211,8 @@ public synchronized WindowStoreIterator fetch(final Bytes key, ); final HasNextCondition hasNextCondition = keySchema.hasNextCondition(key, key, timeFrom, timeTo); - final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator( - cacheIterator, hasNextCondition, cacheFunction - ); + final PeekingKeyValueIterator filteredCacheIterator = + new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator, true); } @@ -282,7 +280,8 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, ); final HasNextCondition hasNextCondition = keySchema.hasNextCondition(from, to, timeFrom, timeTo); - final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); + final PeekingKeyValueIterator filteredCacheIterator = + new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); return new MergedSortedCacheWindowStoreKeyValueIterator( filteredCacheIterator, @@ -328,7 +327,8 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes from, ); final HasNextCondition hasNextCondition = keySchema.hasNextCondition(from, to, timeFrom, timeTo); - final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); + final PeekingKeyValueIterator filteredCacheIterator = + new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); return new MergedSortedCacheWindowStoreKeyValueIterator( filteredCacheIterator, @@ -376,6 +376,7 @@ public KeyValueIterator, byte[]> backwardFetchAll(final Instant final HasNextCondition hasNextCondition = keySchema.hasNextCondition(null, null, timeFrom, timeTo); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); + return new MergedSortedCacheWindowStoreKeyValueIterator( filteredCacheIterator, underlyingIterator, @@ -571,7 +572,7 @@ private void setCacheKeyRange(final long lowerRangeEndTime, final long upperRang throw new IllegalStateException("Error iterating over segments: segment interval has changed"); } - if (keyFrom == keyTo) { + if (keyFrom.equals(keyTo)) { cacheKeyFrom = cacheFunction.cacheKey(segmentLowerRangeFixedSize(keyFrom, lowerRangeEndTime)); cacheKeyTo = cacheFunction.cacheKey(segmentUpperRangeFixedSize(keyTo, upperRangeEndTime)); } else { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java index adac4a4729557..b02459dc57a67 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java @@ -196,4 +196,4 @@ public Bytes peekNextKey() { throw new UnsupportedOperationException("peekNextKey() not supported in " + getClass().getName()); } } -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java index e614cd3eea3ae..e510cffa7bf58 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java @@ -57,11 +57,6 @@ public class InMemoryWindowStore implements WindowStore { private final String name; private final String metricScope; - private ProcessorContext context; - private Sensor expiredRecordSensor; - private int seqnum = 0; - private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP; - private final long retentionPeriod; private final long windowSize; private final boolean retainDuplicates; @@ -69,6 +64,11 @@ public class InMemoryWindowStore implements WindowStore { private final ConcurrentNavigableMap> segmentMap = new ConcurrentSkipListMap<>(); private final Set openIterators = ConcurrentHashMap.newKeySet(); + private ProcessorContext context; + private Sensor expiredRecordSensor; + private int seqnum = 0; + private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP; + private volatile boolean open = false; InMemoryWindowStore(final String name, @@ -539,7 +539,9 @@ public static WrappedInMemoryWindowStoreIterator emptyIterator() { } } - private static class WrappedWindowedKeyValueIterator extends InMemoryWindowStoreIteratorWrapper implements KeyValueIterator, byte[]> { + private static class WrappedWindowedKeyValueIterator + extends InMemoryWindowStoreIteratorWrapper + implements KeyValueIterator, byte[]> { private final long windowSize; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java index eac978cfeafb6..65d3229083ccb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java @@ -22,6 +22,7 @@ import static org.apache.kafka.streams.EqualityCheck.verifyEquality; import static org.apache.kafka.streams.EqualityCheck.verifyInEquality; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.fail; @SuppressWarnings("deprecation") @@ -52,14 +53,14 @@ public void gracePeriodShouldEnforceBoundaries() { } } - @Test(expected = IllegalArgumentException.class) + @Test public void windowSizeMustNotBeNegative() { - SessionWindows.with(ofMillis(-1)); + assertThrows(IllegalArgumentException.class, () -> SessionWindows.with(ofMillis(-1))); } - @Test(expected = IllegalArgumentException.class) + @Test public void windowSizeMustNotBeZero() { - SessionWindows.with(ofMillis(0)); + assertThrows(IllegalArgumentException.class, () -> SessionWindows.with(ofMillis(0))); } @SuppressWarnings("deprecation") // specifically testing deprecated apis diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java index 61f317d495416..60f3523ab097b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java @@ -45,6 +45,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -339,19 +340,19 @@ public void testPutIfAbsent() { assertFalse(driver.flushedEntryRemoved(4)); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerExceptionOnPutNullKey() { - store.put(null, "anyValue"); + assertThrows(NullPointerException.class, () -> store.put(null, "anyValue")); } @Test public void shouldNotThrowNullPointerExceptionOnPutNullValue() { - store.put(1, null); + assertThrows(NullPointerException.class, () -> store.put(1, null)); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerExceptionOnPutIfAbsentNullKey() { - store.putIfAbsent(null, "anyValue"); + assertThrows(NullPointerException.class, () -> store.putIfAbsent(null, "anyValue")); } @Test @@ -359,9 +360,9 @@ public void shouldNotThrowNullPointerExceptionOnPutIfAbsentNullValue() { store.putIfAbsent(1, null); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerExceptionOnPutAllNullKey() { - store.putAll(Collections.singletonList(new KeyValue<>(null, "anyValue"))); + assertThrows(NullPointerException.class, () -> store.putAll(Collections.singletonList(new KeyValue<>(null, "anyValue")))); } @Test @@ -369,24 +370,24 @@ public void shouldNotThrowNullPointerExceptionOnPutAllNullKey() { store.putAll(Collections.singletonList(new KeyValue<>(1, null))); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerExceptionOnDeleteNullKey() { - store.delete(null); + assertThrows(NullPointerException.class, () -> store.delete(null)); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerExceptionOnGetNullKey() { - store.get(null); + assertThrows(NullPointerException.class, () -> store.get(null)); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerExceptionOnRangeNullFromKey() { - store.range(null, 2); + assertThrows(NullPointerException.class, () -> store.range(null, 2)); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerExceptionOnRangeNullToKey() { - store.range(2, null); + assertThrows(NullPointerException.class, () -> store.range(2, null)); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java index d4e5dab292543..6c4ddf6ef6496 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java @@ -95,10 +95,7 @@ public class CachingWindowStoreTest { public void setUp() { keySchema = new WindowKeySchema(); bytesStore = new RocksDBSegmentedBytesStore("test", "metrics-scope", 0, SEGMENT_INTERVAL, keySchema); - underlyingStore = new RocksDBWindowStore( - bytesStore, - false, - WINDOW_SIZE); + underlyingStore = new RocksDBWindowStore(bytesStore, false, WINDOW_SIZE); final TimeWindowedDeserializer keyDeserializer = new TimeWindowedDeserializer<>(new StringDeserializer(), WINDOW_SIZE); keyDeserializer.setIsChangelogTopic(true); cacheListener = new CachingKeyValueStoreTest.CacheFlushListenerStub<>(keyDeserializer, new StringDeserializer()); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java index 39313e17e3c0c..7bb7268052100 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java @@ -173,7 +173,7 @@ public void shouldNotGetValuesBackwardFromOtherStores() { assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results); } - @Test(expected = InvalidStateStoreException.class) + @Test public void shouldThrowInvalidStateStoreExceptionOnRebalance() { final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); EasyMock.expect(storeProvider.stores(anyString(), anyObject())) @@ -185,10 +185,11 @@ public void shouldThrowInvalidStateStoreExceptionOnRebalance() { QueryableStoreTypes.windowStore(), "foo" ); - store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); + + assertThrows(InvalidStateStoreException.class, () -> store.fetch("key", ofEpochMilli(1), ofEpochMilli(10))); } - @Test(expected = InvalidStateStoreException.class) + @Test public void shouldThrowInvalidStateStoreExceptionOnRebalanceWhenBackwards() { final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); EasyMock.expect(storeProvider.stores(anyString(), anyObject())) @@ -200,7 +201,7 @@ public void shouldThrowInvalidStateStoreExceptionOnRebalanceWhenBackwards() { QueryableStoreTypes.windowStore(), "foo" ); - store.backwardFetch("key", ofEpochMilli(1), ofEpochMilli(10)); + assertThrows(InvalidStateStoreException.class, () -> store.backwardFetch("key", ofEpochMilli(1), ofEpochMilli(10))); } @Test @@ -429,19 +430,19 @@ public void shouldBackwardFetchAllAcrossStores() { KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNPEIfKeyIsNull() { - windowStore.fetch(null, ofEpochMilli(0), ofEpochMilli(0)); + assertThrows(NullPointerException.class, () -> windowStore.fetch(null, ofEpochMilli(0), ofEpochMilli(0))); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNPEIfFromKeyIsNull() { - windowStore.fetch(null, "a", ofEpochMilli(0), ofEpochMilli(0)); + assertThrows(NullPointerException.class, () -> windowStore.fetch(null, "a", ofEpochMilli(0), ofEpochMilli(0))); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNPEIfToKeyIsNull() { - windowStore.fetch("a", null, ofEpochMilli(0), ofEpochMilli(0)); + assertThrows(NullPointerException.class, () -> windowStore.fetch("a", null, ofEpochMilli(0), ofEpochMilli(0))); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java index 593b265aa02c3..d2aa9431b2196 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java @@ -25,6 +25,7 @@ import java.util.NoSuchElementException; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; public class DelegatingPeekingKeyValueIteratorTest { @@ -77,18 +78,18 @@ public void shouldPeekAndIterate() { peekingIterator.close(); } - @Test(expected = NoSuchElementException.class) + @Test public void shouldThrowNoSuchElementWhenNoMoreItemsLeftAndNextCalled() { final DelegatingPeekingKeyValueIterator peekingIterator = new DelegatingPeekingKeyValueIterator<>(name, store.all()); peekingIterator.next(); - peekingIterator.close(); + assertThrows(NoSuchElementException.class, peekingIterator::close); } - @Test(expected = NoSuchElementException.class) + @Test public void shouldThrowNoSuchElementWhenNoMoreItemsLeftAndPeekNextCalled() { final DelegatingPeekingKeyValueIterator peekingIterator = new DelegatingPeekingKeyValueIterator<>(name, store.all()); peekingIterator.peekNextKey(); - peekingIterator.close(); + assertThrows(NoSuchElementException.class, peekingIterator::close); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java index 63cc7f7f2db5a..8cfe9b87e7e8f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java @@ -58,7 +58,7 @@ public long segmentId(final Bytes key) { ), new LRUCacheEntry(cacheKey.getBytes()) )).iterator(); - private Deserializer deserializer = Serdes.String().deserializer(); + final private Deserializer deserializer = Serdes.String().deserializer(); @Test public void shouldHaveNextFromStore() { @@ -171,11 +171,9 @@ private Windowed convertWindowedKey(final Windowed bytesWindowed) } - private MergedSortedCacheWindowStoreKeyValueIterator createIterator( - final Iterator, byte[]>> storeKvs, - final Iterator> cacheKvs, - final boolean forward - ) { + private MergedSortedCacheWindowStoreKeyValueIterator createIterator(final Iterator, byte[]>> storeKvs, + final Iterator> cacheKvs, + final boolean forward) { final DelegatingPeekingKeyValueIterator, byte[]> storeIterator = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(storeKvs)); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java index 56ccb501682ef..b831a298fefbe 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java @@ -112,20 +112,12 @@ public void shouldIterateOverAllSegments() { @Test public void shouldIterateBackwardOverAllSegments() { iterator = new SegmentIterator<>( - Arrays.asList(segmentOne, segmentTwo).iterator(), + Arrays.asList(segmentTwo, segmentOne).iterator(), //store should pass the segments in the right order hasNextCondition, Bytes.wrap("a".getBytes()), Bytes.wrap("z".getBytes()), false); - assertTrue(iterator.hasNext()); - assertEquals("b", new String(iterator.peekNextKey().get())); - assertEquals(KeyValue.pair("b", "2"), toStringKeyValue(iterator.next())); - - assertTrue(iterator.hasNext()); - assertEquals("a", new String(iterator.peekNextKey().get())); - assertEquals(KeyValue.pair("a", "1"), toStringKeyValue(iterator.next())); - assertTrue(iterator.hasNext()); assertEquals("d", new String(iterator.peekNextKey().get())); assertEquals(KeyValue.pair("d", "4"), toStringKeyValue(iterator.next())); @@ -134,6 +126,14 @@ public void shouldIterateBackwardOverAllSegments() { assertEquals("c", new String(iterator.peekNextKey().get())); assertEquals(KeyValue.pair("c", "3"), toStringKeyValue(iterator.next())); + assertTrue(iterator.hasNext()); + assertEquals("b", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("b", "2"), toStringKeyValue(iterator.next())); + + assertTrue(iterator.hasNext()); + assertEquals("a", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("a", "1"), toStringKeyValue(iterator.next())); + assertFalse(iterator.hasNext()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java index c9c5789516232..84431ad11c2dd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java @@ -24,7 +24,6 @@ import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.junit.Test; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -35,6 +34,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; public class ThreadCacheTest { @@ -44,7 +44,7 @@ public class ThreadCacheTest { private final LogContext logContext = new LogContext("testCache "); @Test - public void basicPutGet() throws IOException { + public void basicPutGet() { final List> toInsert = Arrays.asList( new KeyValue<>("K1", "V1"), new KeyValue<>("K2", "V2"), @@ -65,7 +65,7 @@ public void basicPutGet() throws IOException { for (final KeyValue kvToInsert : toInsert) { final Bytes key = Bytes.wrap(kvToInsert.key.getBytes()); final LRUCacheEntry entry = cache.get(namespace, key); - assertEquals(entry.isDirty(), true); + assertTrue(entry.isDirty()); assertEquals(new String(entry.value()), kvToInsert.value); } assertEquals(cache.gets(), 5); @@ -252,11 +252,11 @@ public void shouldGetSameKeyAsPeekNext() { assertEquals(iterator.peekNextKey(), iterator.next().key); } - @Test(expected = NoSuchElementException.class) + @Test public void shouldThrowIfNoPeekNextKey() { final ThreadCache cache = new ThreadCache(logContext, 10000L, new MockStreamsMetrics(new Metrics())); final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.range(namespace, Bytes.wrap(new byte[]{0}), Bytes.wrap(new byte[]{1})); - iterator.peekNextKey(); + assertThrows(NoSuchElementException.class, iterator::peekNextKey); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedKeyValueStoreBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedKeyValueStoreBuilderTest.java index a72d733e221ef..c3ad6b329b444 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedKeyValueStoreBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedKeyValueStoreBuilderTest.java @@ -37,6 +37,7 @@ import static org.easymock.EasyMock.reset; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThrows; @RunWith(EasyMockRunner.class) public class TimestampedKeyValueStoreBuilderTest { @@ -145,29 +146,29 @@ public void shouldWrapPlainKeyValueStoreAsTimestampStore() { } @SuppressWarnings("all") - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerIfInnerIsNull() { - new TimestampedKeyValueStoreBuilder<>(null, Serdes.String(), Serdes.String(), new MockTime()); + assertThrows(NullPointerException.class, () -> new TimestampedKeyValueStoreBuilder<>(null, Serdes.String(), Serdes.String(), new MockTime())); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerIfKeySerdeIsNull() { - new TimestampedKeyValueStoreBuilder<>(supplier, null, Serdes.String(), new MockTime()); + assertThrows(NullPointerException.class, () -> new TimestampedKeyValueStoreBuilder<>(supplier, null, Serdes.String(), new MockTime())); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerIfValueSerdeIsNull() { - new TimestampedKeyValueStoreBuilder<>(supplier, Serdes.String(), null, new MockTime()); + assertThrows(NullPointerException.class, () -> new TimestampedKeyValueStoreBuilder<>(supplier, Serdes.String(), null, new MockTime())); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerIfTimeIsNull() { - new TimestampedKeyValueStoreBuilder<>(supplier, Serdes.String(), Serdes.String(), null); + assertThrows(NullPointerException.class, () -> new TimestampedKeyValueStoreBuilder<>(supplier, Serdes.String(), Serdes.String(), null)); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerIfMetricsScopeIsNull() { - new TimestampedKeyValueStoreBuilder<>(supplier, Serdes.String(), Serdes.String(), new MockTime()); + assertThrows(NullPointerException.class, () -> new TimestampedKeyValueStoreBuilder<>(supplier, Serdes.String(), Serdes.String(), new MockTime())); } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilderTest.java index a57134421fe5b..539df95d73f20 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilderTest.java @@ -41,6 +41,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; @RunWith(EasyMockRunner.class) public class TimestampedWindowStoreBuilderTest { @@ -179,24 +180,24 @@ public void shouldDisableCachingWithRetainDuplicates() { } @SuppressWarnings("all") - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerIfInnerIsNull() { - new TimestampedWindowStoreBuilder<>(null, Serdes.String(), Serdes.String(), new MockTime()); + assertThrows(NullPointerException.class, () -> new TimestampedWindowStoreBuilder<>(null, Serdes.String(), Serdes.String(), new MockTime())); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerIfKeySerdeIsNull() { - new TimestampedWindowStoreBuilder<>(supplier, null, Serdes.String(), new MockTime()); + assertThrows(NullPointerException.class, () -> new TimestampedWindowStoreBuilder<>(supplier, null, Serdes.String(), new MockTime())); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerIfValueSerdeIsNull() { - new TimestampedWindowStoreBuilder<>(supplier, Serdes.String(), null, new MockTime()); + assertThrows(NullPointerException.class, () -> new TimestampedWindowStoreBuilder<>(supplier, Serdes.String(), null, new MockTime())); } - @Test(expected = NullPointerException.class) + @Test public void shouldThrowNullPointerIfTimeIsNull() { - new TimestampedWindowStoreBuilder<>(supplier, Serdes.String(), Serdes.String(), null); + assertThrows(NullPointerException.class, () -> new TimestampedWindowStoreBuilder<>(supplier, Serdes.String(), Serdes.String(), null)); } } \ No newline at end of file From 33ee2a1039d922fb91af8b9d75c94be8fdbbbb5e Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 25 Aug 2020 00:38:27 +0100 Subject: [PATCH 19/29] fix: iterate segments backwards --- .../state/internals/CachingWindowStore.java | 42 +++++++++++++------ 1 file changed, 29 insertions(+), 13 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index 77a862221842d..8e0b274972ef0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -476,17 +476,21 @@ private CacheIteratorWrapper(final Bytes keyFrom, this.keyFrom = keyFrom; this.keyTo = keyTo; this.timeTo = timeTo; - this.lastSegmentId = cacheFunction.segmentId(Math.min(timeTo, maxObservedTimestamp.get())); this.forward = forward; this.segmentInterval = cacheFunction.getSegmentInterval(); - this.currentSegmentId = cacheFunction.segmentId(timeFrom); - - setCacheKeyRange(timeFrom, currentSegmentLastTime()); if (forward) { + this.lastSegmentId = cacheFunction.segmentId(Math.min(timeTo, maxObservedTimestamp.get())); + this.currentSegmentId = cacheFunction.segmentId(timeFrom); + + setCacheKeyRange(timeFrom, currentSegmentLastTime()); this.current = context.cache().range(cacheName, cacheKeyFrom, cacheKeyTo); } else { + this.currentSegmentId = cacheFunction.segmentId(Math.min(timeTo, maxObservedTimestamp.get())); + this.lastSegmentId = cacheFunction.segmentId(timeFrom); + + setCacheKeyRange(currentSegmentBeginTime(), Math.min(timeTo, maxObservedTimestamp.get())); this.current = context.cache().reverseRange(cacheName, cacheKeyFrom, cacheKeyTo); } } @@ -548,21 +552,33 @@ private long currentSegmentLastTime() { } private void getNextSegmentIterator() { - ++currentSegmentId; - lastSegmentId = cacheFunction.segmentId(Math.min(timeTo, maxObservedTimestamp.get())); + if (forward) { + ++currentSegmentId; + lastSegmentId = cacheFunction.segmentId(Math.min(timeTo, maxObservedTimestamp.get())); - if (currentSegmentId > lastSegmentId) { - current = null; - return; - } + if (currentSegmentId > lastSegmentId) { + current = null; + return; + } - setCacheKeyRange(currentSegmentBeginTime(), currentSegmentLastTime()); + setCacheKeyRange(currentSegmentBeginTime(), currentSegmentLastTime()); - current.close(); + current.close(); - if (forward) { current = context.cache().range(cacheName, cacheKeyFrom, cacheKeyTo); } else { + --currentSegmentId; +// lastSegmentId = cacheFunction.segmentId(Math.min(timeTo, maxObservedTimestamp.get())); + + if (currentSegmentId < lastSegmentId) { + current = null; + return; + } + + setCacheKeyRange(currentSegmentBeginTime(), currentSegmentLastTime()); + + current.close(); + current = context.cache().reverseRange(cacheName, cacheKeyFrom, cacheKeyTo); } } From 8725a7ea27db07273e9c19ec196f5b66c2d69f83 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 25 Aug 2020 09:50:02 +0100 Subject: [PATCH 20/29] fix iteration --- .../state/internals/CachingWindowStoreTest.java | 10 ++++++++-- .../state/internals/ReadOnlyWindowStoreStub.java | 4 ++-- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java index 6c4ddf6ef6496..21cb3a26328a5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java @@ -663,6 +663,7 @@ public void shouldFetchAndIterateOverKeyRange() { cachingStore.put(bytesKey("a"), bytesValue("0003"), 1); cachingStore.put(bytesKey("aa"), bytesValue("0004"), 1); cachingStore.put(bytesKey("a"), bytesValue("0005"), SEGMENT_INTERVAL); + cachingStore.put(bytesKey("aa"), bytesValue("0006"), SEGMENT_INTERVAL + 1); verifyKeyValueList( asList( @@ -676,7 +677,8 @@ public void shouldFetchAndIterateOverKeyRange() { verifyKeyValueList( asList( windowedPair("aa", "0002", 0), - windowedPair("aa", "0004", 1)), + windowedPair("aa", "0004", 1), + windowedPair("aa", "0006", SEGMENT_INTERVAL + 1)), toList(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) ); @@ -686,7 +688,8 @@ public void shouldFetchAndIterateOverKeyRange() { windowedPair("a", "0003", 1), windowedPair("aa", "0002", 0), windowedPair("aa", "0004", 1), - windowedPair("a", "0005", SEGMENT_INTERVAL) + windowedPair("a", "0005", SEGMENT_INTERVAL), + windowedPair("aa", "0006", SEGMENT_INTERVAL + 1) ), toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) ); @@ -699,6 +702,7 @@ public void shouldFetchAndIterateOverKeyBackwardRange() { cachingStore.put(bytesKey("a"), bytesValue("0003"), 1); cachingStore.put(bytesKey("aa"), bytesValue("0004"), 1); cachingStore.put(bytesKey("a"), bytesValue("0005"), SEGMENT_INTERVAL); + cachingStore.put(bytesKey("aa"), bytesValue("0006"), SEGMENT_INTERVAL + 1); verifyKeyValueList( asList( @@ -711,6 +715,7 @@ public void shouldFetchAndIterateOverKeyBackwardRange() { verifyKeyValueList( asList( + windowedPair("aa", "0006", SEGMENT_INTERVAL + 1), windowedPair("aa", "0004", 1), windowedPair("aa", "0002", 0)), toList(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) @@ -718,6 +723,7 @@ public void shouldFetchAndIterateOverKeyBackwardRange() { verifyKeyValueList( asList( + windowedPair("aa", "0006", SEGMENT_INTERVAL + 1), windowedPair("a", "0005", SEGMENT_INTERVAL), windowedPair("aa", "0004", 1), windowedPair("aa", "0002", 0), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java index cc26a454ba7a7..9407dab54c9d6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java @@ -339,7 +339,7 @@ public KeyValueIterator, V> backwardFetch(final K from, throw new InvalidStateStoreException("Store is not open"); } final List, V>> results = new ArrayList<>(); - for (long now = timeFrom; now <= timeTo; now++) { + for (long now = timeTo; now >= timeFrom; now--) { final NavigableMap kvMap = data.get(now); if (kvMap != null) { for (final Entry entry : kvMap.subMap(from, true, to, true).descendingMap().entrySet()) { @@ -410,7 +410,7 @@ void setOpen(final boolean open) { this.open = open; } - private class TheWindowStoreIterator implements WindowStoreIterator { + private static class TheWindowStoreIterator implements WindowStoreIterator { private final Iterator> underlying; From 86932775070f2c33eedacbb9341c5f819bdd0cf1 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 26 Aug 2020 17:32:07 +0100 Subject: [PATCH 21/29] remove commented code --- .../apache/kafka/streams/state/internals/CachingWindowStore.java | 1 - 1 file changed, 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index 8e0b274972ef0..f04ebbde4cc36 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -568,7 +568,6 @@ private void getNextSegmentIterator() { current = context.cache().range(cacheName, cacheKeyFrom, cacheKeyTo); } else { --currentSegmentId; -// lastSegmentId = cacheFunction.segmentId(Math.min(timeTo, maxObservedTimestamp.get())); if (currentSegmentId < lastSegmentId) { current = null; From 4f81ba3985722efb1469dd653c44e8a2c17d1d8d Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 27 Aug 2020 10:50:11 +0100 Subject: [PATCH 22/29] adding long variant to backward methods. --- .../internals/AbstractReadOnlyDecorator.java | 13 ++- .../internals/AbstractReadWriteDecorator.java | 26 ++++++ .../streams/state/ReadOnlyWindowStore.java | 51 ++++++------ .../kafka/streams/state/WindowStore.java | 68 ++++++++++++--- .../state/internals/CachingWindowStore.java | 31 +++---- .../ChangeLoggingWindowBytesStore.java | 22 +++-- .../CompositeReadOnlyWindowStore.java | 82 ++++++++++++------ .../state/internals/InMemoryWindowStore.java | 17 +--- .../state/internals/MeteredWindowStore.java | 14 ++-- .../internals/ReadOnlyWindowStoreFacade.java | 65 +++++++++------ .../state/internals/RocksDBWindowStore.java | 22 ++--- .../TimestampedWindowStoreBuilder.java | 13 ++- ...owToTimestampedWindowByteStoreAdapter.java | 36 ++++---- .../kafka/streams/state/NoOpWindowStore.java | 59 +++++++++++-- .../internals/AbstractKeyValueStoreTest.java | 2 +- .../AbstractWindowBytesStoreTest.java | 83 ++++++++++--------- .../internals/CachingWindowStoreTest.java | 10 +-- .../ChangeLoggingWindowBytesStoreTest.java | 7 +- ...DelegatingPeekingKeyValueIteratorTest.java | 14 ++-- .../internals/ReadOnlyWindowStoreStub.java | 64 +++++++++----- 20 files changed, 423 insertions(+), 276 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java index de4e1fa7cc018..ecd7903abd1ab 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java @@ -30,7 +30,6 @@ import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.internals.WrappedStateStore; -import java.time.Instant; import java.util.List; abstract class AbstractReadOnlyDecorator extends WrappedStateStore { @@ -183,8 +182,8 @@ public WindowStoreIterator fetch(final K key, @Override public WindowStoreIterator backwardFetch(final K key, - final Instant timeFrom, - final Instant timeTo) { + final long timeFrom, + final long timeTo) { return wrapped().backwardFetch(key, timeFrom, timeTo); } @@ -200,8 +199,8 @@ public KeyValueIterator, V> fetch(final K from, @Override public KeyValueIterator, V> backwardFetch(final K from, final K to, - final Instant timeFrom, - final Instant timeTo) { + final long timeFrom, + final long timeTo) { return wrapped().backwardFetch(from, to, timeFrom, timeTo); } @@ -223,8 +222,8 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, } @Override - public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, - final Instant timeTo) { + public KeyValueIterator, V> backwardFetchAll(final long timeFrom, + final long timeTo) { return wrapped().backwardFetchAll(timeFrom, timeTo); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java index 61e47f6cf6c6e..be2d097218dc1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java @@ -173,6 +173,13 @@ public WindowStoreIterator fetch(final K key, return wrapped().fetch(key, timeFrom, timeTo); } + @Override + public WindowStoreIterator backwardFetch(final K key, + final long timeFrom, + final long timeTo) { + return wrapped().backwardFetch(key, timeFrom, timeTo); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, V> fetch(final K from, @@ -182,6 +189,14 @@ public KeyValueIterator, V> fetch(final K from, return wrapped().fetch(from, to, timeFrom, timeTo); } + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final long timeFrom, + final long timeTo) { + return wrapped().backwardFetch(from, to, timeFrom, timeTo); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, V> fetchAll(final long timeFrom, @@ -189,10 +204,21 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, return wrapped().fetchAll(timeFrom, timeTo); } + @Override + public KeyValueIterator, V> backwardFetchAll(final long timeFrom, + final long timeTo) { + return wrapped().backwardFetchAll(timeFrom, timeTo); + } + @Override public KeyValueIterator, V> all() { return wrapped().all(); } + + @Override + public KeyValueIterator, V> backwardAll() { + return wrapped().backwardAll(); + } } static class TimestampedWindowStoreReadWriteDecorator diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java index 0fbf82588cb4d..83104d21fd90d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java @@ -105,14 +105,14 @@ public interface ReadOnlyWindowStore { * available window to the newest/latest window. * * @param key the key to fetch - * @param from time range start (inclusive), where iteration starts. - * @param to time range end (inclusive), where iteration ends. + * @param timeFrom time range start (inclusive), where iteration starts. + * @param timeTo time range end (inclusive), where iteration ends. * @return an iterator over key-value pairs {@code }, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ - WindowStoreIterator fetch(K key, Instant from, Instant to) throws IllegalArgumentException; + WindowStoreIterator fetch(K key, Instant timeFrom, Instant timeTo) throws IllegalArgumentException; /** * Get all the key-value pairs with the given key and the time range from all the existing windows @@ -143,16 +143,16 @@ public interface ReadOnlyWindowStore { * available window to the oldest/earliest window. * * @param key the key to fetch - * @param from time range start (inclusive), where iteration ends. - * @param to time range end (inclusive), where iteration starts. + * @param timeFrom time range start (inclusive), where iteration ends. + * @param timeTo time range end (inclusive), where iteration starts. * @return an iterator over key-value pairs {@code }, from end to beginning of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ - default WindowStoreIterator backwardFetch(K key, Instant from, Instant to) throws IllegalArgumentException { - throw new UnsupportedOperationException(); - } + WindowStoreIterator backwardFetch(K key, long timeFrom, long timeTo); + + WindowStoreIterator backwardFetch(K key, Instant timeFrom, Instant timeTo) throws IllegalArgumentException; /** * Get all the key-value pairs in the given key range and time range from all the existing windows. @@ -178,14 +178,14 @@ default WindowStoreIterator backwardFetch(K key, Instant from, Instant to) th * * @param from the first key in the range * @param to the last key in the range - * @param fromTime time range start (inclusive), where iteration starts. - * @param toTime time range end (inclusive), where iteration ends. + * @param timeFrom time range start (inclusive), where iteration starts. + * @param timeTo time range end (inclusive), where iteration ends. * @return an iterator over windowed key-value pairs {@code , value>}, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for any key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ - KeyValueIterator, V> fetch(K from, K to, Instant fromTime, Instant toTime) + KeyValueIterator, V> fetch(K from, K to, Instant timeFrom, Instant timeTo) throws IllegalArgumentException; /** @@ -196,17 +196,18 @@ KeyValueIterator, V> fetch(K from, K to, Instant fromTime, Instant t * * @param from the first key in the range * @param to the last key in the range - * @param fromTime time range start (inclusive), where iteration ends. - * @param toTime time range end (inclusive), where iteration starts. + * @param timeFrom time range start (inclusive), where iteration ends. + * @param timeTo time range end (inclusive), where iteration starts. * @return an iterator over windowed key-value pairs {@code , value>}, from end to beginning of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for any key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ - default KeyValueIterator, V> backwardFetch(K from, K to, Instant fromTime, Instant toTime) - throws IllegalArgumentException { - throw new UnsupportedOperationException(); - } + KeyValueIterator, V> backwardFetch(K from, K to, long timeFrom, long timeTo); + + KeyValueIterator, V> backwardFetch(K from, K to, Instant timeFrom, Instant timeTo) + throws IllegalArgumentException; + /** * Gets all the key-value pairs in the existing windows. @@ -243,27 +244,27 @@ default KeyValueIterator, V> backwardAll() { /** * Gets all the key-value pairs that belong to the windows within in the given time range. * - * @param from the beginning of the time slot from which to search (inclusive), where iteration starts. - * @param to the end of the time slot from which to search (inclusive), where iteration ends. + * @param timeFrom the beginning of the time slot from which to search (inclusive), where iteration starts. + * @param timeTo the end of the time slot from which to search (inclusive), where iteration ends. * @return an iterator over windowed key-value pairs {@code , value>}, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException if {@code null} is used for any key * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ - KeyValueIterator, V> fetchAll(Instant from, Instant to) throws IllegalArgumentException; + KeyValueIterator, V> fetchAll(Instant timeFrom, Instant timeTo) throws IllegalArgumentException; /** * Gets all the key-value pairs that belong to the windows within in the given time range in backward order * with respect to time (from end to beginning of time). * - * @param from the beginning of the time slot from which to search (inclusive), where iteration ends. - * @param to the end of the time slot from which to search (inclusive), where iteration starts. + * @param timeFrom the beginning of the time slot from which to search (inclusive), where iteration ends. + * @param timeTo the end of the time slot from which to search (inclusive), where iteration starts. * @return an backward iterator over windowed key-value pairs {@code , value>}, from end to beginning of time. * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException if {@code null} is used for any key * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ - default KeyValueIterator, V> backwardFetchAll(Instant from, Instant to) throws IllegalArgumentException { - throw new UnsupportedOperationException(); - } + KeyValueIterator, V> backwardFetchAll(long timeFrom, long timeTo); + + KeyValueIterator, V> backwardFetchAll(Instant timeFrom, Instant timeTo) throws IllegalArgumentException; } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java index e9f1db0507c2b..83254a08256e8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java @@ -105,12 +105,28 @@ public interface WindowStore extends StateStore, ReadOnlyWindowStore @Override default WindowStoreIterator fetch(final K key, - final Instant from, - final Instant to) { + final Instant timeFrom, + final Instant timeTo) { return fetch( key, - ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")), - ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to"))); + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); + } + + default WindowStoreIterator backwardFetch(final K key, + final long timeFrom, + final long timeTo) throws IllegalArgumentException { + throw new UnsupportedOperationException(); + } + + @Override + default WindowStoreIterator backwardFetch(final K key, + final Instant timeFrom, + final Instant timeTo) { + return backwardFetch( + key, + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } /** @@ -133,13 +149,32 @@ default WindowStoreIterator fetch(final K key, @Override default KeyValueIterator, V> fetch(final K from, final K to, - final Instant fromTime, - final Instant toTime) { + final Instant timeFrom, + final Instant timeTo) { return fetch( from, to, - ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")), - ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime"))); + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); + } + + default KeyValueIterator, V> backwardFetch(final K from, + final K to, + final long timeFrom, + final long timeTo) { + throw new UnsupportedOperationException(); + } + + @Override + default KeyValueIterator, V> backwardFetch(final K from, + final K to, + final Instant timeFrom, + final Instant timeTo) { + return backwardFetch( + from, + to, + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } /** @@ -155,9 +190,20 @@ default KeyValueIterator, V> fetch(final K from, KeyValueIterator, V> fetchAll(long timeFrom, long timeTo); @Override - default KeyValueIterator, V> fetchAll(final Instant from, final Instant to) { + default KeyValueIterator, V> fetchAll(final Instant timeFrom, final Instant timeTo) { return fetchAll( - ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")), - ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to"))); + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); + } + + default KeyValueIterator, V> backwardFetchAll(final long timeFrom, final long timeTo) { + throw new UnsupportedOperationException(); + } + + @Override + default KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, final Instant timeTo) { + return backwardFetchAll( + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index f04ebbde4cc36..fbabfc2c617a6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; @@ -34,12 +33,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.time.Instant; import java.util.LinkedList; import java.util.NoSuchElementException; import java.util.concurrent.atomic.AtomicLong; -import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; import static org.apache.kafka.streams.state.internals.ExceptionUtils.executeAll; import static org.apache.kafka.streams.state.internals.ExceptionUtils.throwSuppressed; @@ -219,15 +216,13 @@ public synchronized WindowStoreIterator fetch(final Bytes key, @Override public synchronized WindowStoreIterator backwardFetch(final Bytes key, - final Instant from, - final Instant to) { - final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); - final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + final long timeFrom, + final long timeTo) { // since this function may not access the underlying inner store, we need to validate // if store is open outside as well. validateStoreOpen(); - final WindowStoreIterator underlyingIterator = wrapped().backwardFetch(key, from, to); + final WindowStoreIterator underlyingIterator = wrapped().backwardFetch(key, timeFrom, timeTo); if (context.cache() == null) { return underlyingIterator; } @@ -296,8 +291,8 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, @Override public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Bytes to, - final Instant fromTime, - final Instant toTime) { + final long timeFrom, + final long timeTo) { if (from.compareTo(to) > 0) { LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + @@ -305,15 +300,12 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes from, return KeyValueIterators.emptyIterator(); } - final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")); - final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime")); - // since this function may not access the underlying inner store, we need to validate // if store is open outside as well. validateStoreOpen(); final KeyValueIterator, byte[]> underlyingIterator = - wrapped().backwardFetch(from, to, fromTime, toTime); + wrapped().backwardFetch(from, to, timeFrom, timeTo); if (context.cache() == null) { return underlyingIterator; } @@ -363,14 +355,11 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, } @Override - public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, - final Instant to) { + public KeyValueIterator, byte[]> backwardFetchAll(final long timeFrom, + final long timeTo) { validateStoreOpen(); - final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); - final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); - - final KeyValueIterator, byte[]> underlyingIterator = wrapped().backwardFetchAll(from, to); + final KeyValueIterator, byte[]> underlyingIterator = wrapped().backwardFetchAll(timeFrom, timeTo); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = context.cache().reverseAll(cacheName); final HasNextCondition hasNextCondition = keySchema.hasNextCondition(null, null, timeFrom, timeTo); @@ -554,6 +543,7 @@ private long currentSegmentLastTime() { private void getNextSegmentIterator() { if (forward) { ++currentSegmentId; + // updating as maxObservedTimestamp can change while iterating lastSegmentId = cacheFunction.segmentId(Math.min(timeTo, maxObservedTimestamp.get())); if (currentSegmentId > lastSegmentId) { @@ -569,6 +559,7 @@ private void getNextSegmentIterator() { } else { --currentSegmentId; + // last segment id is stable when iterating backward, therefore no need to update if (currentSegmentId < lastSegmentId) { current = null; return; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java index 374deca9f8841..8da413cab24bb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java @@ -25,8 +25,6 @@ import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; -import java.time.Instant; - /** * Simple wrapper around a {@link WindowStore} to support writing * updates to a changelog @@ -73,26 +71,26 @@ public WindowStoreIterator fetch(final Bytes key, @Override public WindowStoreIterator backwardFetch(final Bytes key, - final Instant from, - final Instant to) { - return wrapped().backwardFetch(key, from, to); + final long timeFrom, + final long timeTo) { + return wrapped().backwardFetch(key, timeFrom, timeTo); } @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo, - final long from, + final long timeFrom, final long to) { - return wrapped().fetch(keyFrom, keyTo, from, to); + return wrapped().fetch(keyFrom, keyTo, timeFrom, to); } @Override public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, final Bytes keyTo, - final Instant from, - final Instant to) { - return wrapped().backwardFetch(keyFrom, keyTo, from, to); + final long timeFrom, + final long timeTo) { + return wrapped().backwardFetch(keyFrom, keyTo, timeFrom, timeTo); } @Override @@ -114,8 +112,8 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, } @Override - public KeyValueIterator, byte[]> backwardFetchAll(final Instant timeFrom, - final Instant timeTo) { + public KeyValueIterator, byte[]> backwardFetchAll(final long timeFrom, + final long timeTo) { return wrapped().backwardFetchAll(timeFrom, timeTo); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java index 81368fe9bb8d2..d5b22e6ba35cc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java @@ -91,10 +91,21 @@ public WindowStoreIterator fetch(final K key, return KeyValueIterators.emptyWindowStoreIterator(); } + @SuppressWarnings("deprecation") // removing fetch(K from, long from, long to) will fix this + @Override + public WindowStoreIterator fetch(final K key, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return fetch( + key, + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "from")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "to"))); + } + @Override public WindowStoreIterator backwardFetch(final K key, - final Instant timeFrom, - final Instant timeTo) { + final long timeFrom, + final long timeTo) { Objects.requireNonNull(key, "key can't be null"); final List> stores = provider.stores(storeName, windowStoreType); for (final ReadOnlyWindowStore windowStore : stores) { @@ -114,15 +125,14 @@ public WindowStoreIterator backwardFetch(final K key, return KeyValueIterators.emptyWindowStoreIterator(); } - @SuppressWarnings("deprecation") // removing fetch(K from, long from, long to) will fix this @Override - public WindowStoreIterator fetch(final K key, - final Instant from, - final Instant to) throws IllegalArgumentException { - return fetch( + public WindowStoreIterator backwardFetch(final K key, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return backwardFetch( key, - ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")), - ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to"))); + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } @SuppressWarnings("deprecation") // removing fetch(K from, K to, long from, long to) will fix this @@ -142,11 +152,23 @@ public KeyValueIterator, V> fetch(final K from, nextIteratorFunction)); } + @Override + public KeyValueIterator, V> fetch(final K from, + final K to, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return fetch( + from, + to, + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); + } + @Override public KeyValueIterator, V> backwardFetch(final K from, final K to, - final Instant timeFrom, - final Instant timeTo) { + final long timeFrom, + final long timeTo) throws IllegalArgumentException { Objects.requireNonNull(from, "from can't be null"); Objects.requireNonNull(to, "to can't be null"); final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = @@ -159,15 +181,15 @@ public KeyValueIterator, V> backwardFetch(final K from, } @Override - public KeyValueIterator, V> fetch(final K from, - final K to, - final Instant fromTime, - final Instant toTime) throws IllegalArgumentException { - return fetch( + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final Instant timeFrom, + final Instant timeTo) { + return backwardFetch( from, to, - ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")), - ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime"))); + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } @Override @@ -205,9 +227,18 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, nextIteratorFunction)); } + @SuppressWarnings("deprecation") // removing fetchAll(long from, long to) will fix this + @Override + public KeyValueIterator, V> fetchAll(final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return fetchAll( + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "from")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "to"))); + } + @Override - public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, - final Instant timeTo) { + public KeyValueIterator, V> backwardFetchAll(final long timeFrom, + final long timeTo) throws IllegalArgumentException { final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = store -> store.backwardFetchAll(timeFrom, timeTo); return new DelegatingPeekingKeyValueIterator<>( @@ -217,12 +248,11 @@ public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, nextIteratorFunction)); } - @SuppressWarnings("deprecation") // removing fetchAll(long from, long to) will fix this @Override - public KeyValueIterator, V> fetchAll(final Instant from, - final Instant to) throws IllegalArgumentException { - return fetchAll( - ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")), - ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to"))); + public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) { + return backwardFetchAll( + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java index e510cffa7bf58..8217c9745e137 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.TimeWindow; import org.apache.kafka.streams.processor.ProcessorContext; @@ -35,7 +34,6 @@ import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; -import java.time.Instant; import java.util.Iterator; import java.util.Map; import java.util.NoSuchElementException; @@ -45,7 +43,6 @@ import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; -import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; import static org.apache.kafka.streams.state.internals.WindowKeySchema.extractStoreKeyBytes; import static org.apache.kafka.streams.state.internals.WindowKeySchema.extractStoreTimestamp; @@ -168,9 +165,7 @@ public WindowStoreIterator fetch(final Bytes key, final long timeFrom, f } @Override - public WindowStoreIterator backwardFetch(final Bytes key, final Instant from, final Instant to) { - final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); - final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + public WindowStoreIterator backwardFetch(final Bytes key, final long timeFrom, final long timeTo) { return fetch(key, timeFrom, timeTo, false); } @@ -213,10 +208,8 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, @Override public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Bytes to, - final Instant fromTime, - final Instant toTime) { - final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")); - final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime")); + final long timeFrom, + final long timeTo) { return fetch(from, to, timeFrom, timeTo, false); } @@ -268,9 +261,7 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, f } @Override - public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, final Instant to) { - final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); - final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + public KeyValueIterator, byte[]> backwardFetchAll(final long timeFrom, final long timeTo) { return fetchAll(timeFrom, timeTo, false); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index fcd761016d2ee..e7315dcdf5354 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -33,8 +33,6 @@ import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics; -import java.time.Instant; - import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; public class MeteredWindowStore @@ -173,8 +171,8 @@ public WindowStoreIterator fetch(final K key, @Override public WindowStoreIterator backwardFetch(final K key, - final Instant timeFrom, - final Instant timeTo) { + final long timeFrom, + final long timeTo) { return new MeteredWindowStoreIterator<>( wrapped().backwardFetch(keyBytes(key), timeFrom, timeTo), fetchSensor, @@ -201,8 +199,8 @@ public KeyValueIterator, V> fetch(final K from, @Override public KeyValueIterator, V> backwardFetch(final K from, final K to, - final Instant timeFrom, - final Instant timeTo) { + final long timeFrom, + final long timeTo) { return new MeteredWindowedKeyValueIterator<>( wrapped().backwardFetch(keyBytes(from), keyBytes(to), timeFrom, timeTo), fetchSensor, @@ -224,8 +222,8 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, } @Override - public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, - final Instant timeTo) { + public KeyValueIterator, V> backwardFetchAll(final long timeFrom, + final long timeTo) { return new MeteredWindowedKeyValueIterator<>( wrapped().backwardFetchAll(timeFrom, timeTo), fetchSensor, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java index ff84d4e2da041..b5464b1db0983 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java @@ -51,16 +51,23 @@ public WindowStoreIterator fetch(final K key, @Override public WindowStoreIterator fetch(final K key, - final Instant from, - final Instant to) throws IllegalArgumentException { - return new WindowStoreIteratorFacade<>(inner.fetch(key, from, to)); + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return new WindowStoreIteratorFacade<>(inner.fetch(key, timeFrom, timeTo)); + } + + @Override + public WindowStoreIterator backwardFetch(final K key, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return new WindowStoreIteratorFacade<>(inner.backwardFetch(key, timeFrom, timeTo)); } @Override public WindowStoreIterator backwardFetch(final K key, - final Instant from, - final Instant to) throws IllegalArgumentException { - return new WindowStoreIteratorFacade<>(inner.backwardFetch(key, from, to)); + final long timeFrom, + final long timeTo) throws IllegalArgumentException { + return new WindowStoreIteratorFacade<>(inner.backwardFetch(key, timeFrom, timeTo)); } @Override @@ -75,17 +82,25 @@ public KeyValueIterator, V> fetch(final K from, @Override public KeyValueIterator, V> fetch(final K from, final K to, - final Instant fromTime, - final Instant toTime) throws IllegalArgumentException { - return new KeyValueIteratorFacade<>(inner.fetch(from, to, fromTime, toTime)); + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return new KeyValueIteratorFacade<>(inner.fetch(from, to, timeFrom, timeTo)); } @Override public KeyValueIterator, V> backwardFetch(final K from, final K to, - final Instant fromTime, - final Instant toTime) throws IllegalArgumentException { - return new KeyValueIteratorFacade<>(inner.backwardFetch(from, to, fromTime, toTime)); + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return new KeyValueIteratorFacade<>(inner.backwardFetch(from, to, timeFrom, timeTo)); + } + + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final long timeFrom, + final long timeTo) throws IllegalArgumentException { + return new KeyValueIteratorFacade<>(inner.backwardFetch(from, to, timeFrom, timeTo)); } @Override @@ -96,29 +111,31 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, } @Override - public KeyValueIterator, V> fetchAll(final Instant from, - final Instant to) throws IllegalArgumentException { - final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.fetchAll(from, to); - return new KeyValueIteratorFacade<>(innerIterator); + public KeyValueIterator, V> fetchAll(final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return new KeyValueIteratorFacade<>(inner.fetchAll(timeFrom, timeTo)); + } + + @Override + public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return new KeyValueIteratorFacade<>(inner.backwardFetchAll(timeFrom, timeTo)); } @Override - public KeyValueIterator, V> backwardFetchAll(final Instant from, - final Instant to) throws IllegalArgumentException { - final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.backwardFetchAll(from, to); - return new KeyValueIteratorFacade<>(innerIterator); + public KeyValueIterator, V> backwardFetchAll(final long timeFrom, + final long timeTo) throws IllegalArgumentException { + return new KeyValueIteratorFacade<>(inner.backwardFetchAll(timeFrom, timeTo)); } @Override public KeyValueIterator, V> all() { - final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.all(); - return new KeyValueIteratorFacade<>(innerIterator); + return new KeyValueIteratorFacade<>(inner.all()); } @Override public KeyValueIterator, V> backwardAll() { - final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.backwardAll(); - return new KeyValueIteratorFacade<>(innerIterator); + return new KeyValueIteratorFacade<>(inner.backwardAll()); } private static class WindowStoreIteratorFacade implements WindowStoreIterator { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index fbe03fef29fa0..3a278e0bc52b7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; @@ -25,10 +24,6 @@ import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; -import java.time.Instant; - -import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; - public class RocksDBWindowStore extends WrappedStateStore implements WindowStore { @@ -70,8 +65,7 @@ public void put(final Bytes key, final byte[] value, final long windowStartTimes @Override public byte[] fetch(final Bytes key, final long timestamp) { - final byte[] bytesValue = wrapped().get(WindowKeySchema.toStoreKeyBinary(key, timestamp, seqnum)); - return bytesValue; + return wrapped().get(WindowKeySchema.toStoreKeyBinary(key, timestamp, seqnum)); } @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @@ -82,9 +76,7 @@ public WindowStoreIterator fetch(final Bytes key, final long timeFrom, f } @Override - public WindowStoreIterator backwardFetch(final Bytes key, final Instant from, final Instant to) { - final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); - final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + public WindowStoreIterator backwardFetch(final Bytes key, final long timeFrom, final long timeTo) { final KeyValueIterator bytesIterator = wrapped().backwardFetch(key, timeFrom, timeTo); return new WindowStoreIteratorWrapper(bytesIterator, windowSize).valuesIterator(); } @@ -102,10 +94,8 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, @Override public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Bytes to, - final Instant fromTime, - final Instant toTime) { - final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "from")); - final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "to")); + final long timeFrom, + final long timeTo) { final KeyValueIterator bytesIterator = wrapped().backwardFetch(from, to, timeFrom, timeTo); return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); } @@ -130,9 +120,7 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, f } @Override - public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, final Instant to) { - final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); - final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + public KeyValueIterator, byte[]> backwardFetchAll(final long timeFrom, final long timeTo) { final KeyValueIterator bytesIterator = wrapped().backwardFetchAll(timeFrom, timeTo); return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java index ebb1be5aa77b0..6da6a6df2a3e0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java @@ -32,7 +32,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.time.Instant; import java.util.Objects; public class TimestampedWindowStoreBuilder @@ -145,8 +144,8 @@ public WindowStoreIterator fetch(final Bytes key, @Override public WindowStoreIterator backwardFetch(final Bytes key, - final Instant timeFrom, - final Instant timeTo) { + final long timeFrom, + final long timeTo) { return wrapped.backwardFetch(key, timeFrom, timeTo); } @@ -162,8 +161,8 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, @Override public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Bytes to, - final Instant timeFrom, - final Instant timeTo) { + final long timeFrom, + final long timeTo) { return wrapped.backwardFetch(from, to, timeFrom, timeTo); } @@ -175,8 +174,8 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, } @Override - public KeyValueIterator, byte[]> backwardFetchAll(final Instant timeFrom, - final Instant timeTo) { + public KeyValueIterator, byte[]> backwardFetchAll(final long timeFrom, + final long timeTo) { return wrapped.backwardFetchAll(timeFrom, timeTo); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java index c36c4146f8dc2..931f565885a4b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java @@ -69,16 +69,16 @@ public WindowStoreIterator fetch(final Bytes key, @Override public WindowStoreIterator fetch(final Bytes key, - final Instant from, - final Instant to) { - return new WindowToTimestampedWindowIteratorAdapter(store.fetch(key, from, to)); + final Instant timeFrom, + final Instant timeTo) { + return new WindowToTimestampedWindowIteratorAdapter(store.fetch(key, timeFrom, timeTo)); } @Override public WindowStoreIterator backwardFetch(final Bytes key, - final Instant from, - final Instant to) { - return new WindowToTimestampedWindowIteratorAdapter(store.backwardFetch(key, from, to)); + final Instant timeFrom, + final Instant timeTo) { + return new WindowToTimestampedWindowIteratorAdapter(store.backwardFetch(key, timeFrom, timeTo)); } @Override @@ -93,17 +93,17 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, @Override public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Bytes to, - final Instant fromTime, - final Instant toTime) { - return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardFetch(from, to, fromTime, toTime)); + final Instant timeFrom, + final Instant timeTo) { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardFetch(from, to, timeFrom, timeTo)); } @Override public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to, - final Instant fromTime, - final Instant toTime) { - return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetch(from, to, fromTime, toTime)); + final Instant timeFrom, + final Instant timeTo) { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetch(from, to, timeFrom, timeTo)); } @Override @@ -124,15 +124,15 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, } @Override - public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, - final Instant to) { - return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardFetchAll(from, to)); + public KeyValueIterator, byte[]> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardFetchAll(timeFrom, timeTo)); } @Override - public KeyValueIterator, byte[]> fetchAll(final Instant from, - final Instant to) { - return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetchAll(from, to)); + public KeyValueIterator, byte[]> fetchAll(final Instant timeFrom, + final Instant timeTo) { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetchAll(timeFrom, timeTo)); } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java index 0e8a7b9b30475..033e1a927c8d9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java @@ -91,21 +91,52 @@ public WindowStoreIterator fetch(final Object key, final long timeFrom, final lo } @Override - public WindowStoreIterator fetch(final Object key, final Instant from, final Instant to) { + public WindowStoreIterator fetch(final Object key, final Instant timeFrom, final Instant timeTo) { + return EMPTY_WINDOW_STORE_ITERATOR; + } + + @Override + public WindowStoreIterator backwardFetch(final Object key, final long timeFrom, final long timeTo) { + return EMPTY_WINDOW_STORE_ITERATOR; + } + + @Override + public WindowStoreIterator backwardFetch(final Object key, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { return EMPTY_WINDOW_STORE_ITERATOR; } @Override @SuppressWarnings("deprecation") - public WindowStoreIterator fetch(final Object from, final Object to, final long timeFrom, final long timeTo) { + public WindowStoreIterator fetch(final Object from, + final Object to, + final long timeFrom, + final long timeTo) { return EMPTY_WINDOW_STORE_ITERATOR; } @Override public KeyValueIterator fetch(final Object from, final Object to, - final Instant fromTime, - final Instant toTime) throws IllegalArgumentException { + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return EMPTY_WINDOW_STORE_ITERATOR; + } + + @Override + public WindowStoreIterator backwardFetch(final Object from, + final Object to, + final long timeFrom, + final long timeTo) { + return EMPTY_WINDOW_STORE_ITERATOR; + } + + @Override + public KeyValueIterator backwardFetch(final Object from, + final Object to, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { return EMPTY_WINDOW_STORE_ITERATOR; } @@ -113,7 +144,12 @@ public KeyValueIterator fetch(final Object from, public WindowStoreIterator all() { return EMPTY_WINDOW_STORE_ITERATOR; } - + + @Override + public WindowStoreIterator backwardAll() { + return EMPTY_WINDOW_STORE_ITERATOR; + } + @Override @SuppressWarnings("deprecation") public WindowStoreIterator fetchAll(final long timeFrom, final long timeTo) { @@ -121,7 +157,18 @@ public WindowStoreIterator fetchAll(final long timeFrom, final long ti } @Override - public KeyValueIterator fetchAll(final Instant from, final Instant to) { + public KeyValueIterator fetchAll(final Instant timeFrom, final Instant timeTo) { + return EMPTY_WINDOW_STORE_ITERATOR; + } + + @Override + public WindowStoreIterator backwardFetchAll(final long timeFrom, final long timeTo) { + return EMPTY_WINDOW_STORE_ITERATOR; + } + + @Override + public KeyValueIterator backwardFetchAll(final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { return EMPTY_WINDOW_STORE_ITERATOR; } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java index 60f3523ab097b..4c9c0440eda15 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java @@ -347,7 +347,7 @@ public void shouldThrowNullPointerExceptionOnPutNullKey() { @Test public void shouldNotThrowNullPointerExceptionOnPutNullValue() { - assertThrows(NullPointerException.class, () -> store.put(1, null)); + store.put(1, null); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java index 84622150e56a2..0366fe57b6cbe 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java @@ -60,6 +60,7 @@ import static java.util.Arrays.asList; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.toSet; import static org.apache.kafka.test.StreamsTestUtils.valuesToSet; import static org.hamcrest.CoreMatchers.equalTo; @@ -334,17 +335,17 @@ public void shouldBackwardFetchAllInTimeRange() { final KeyValue, String> four = windowedPair(4, "four", startTime + 4); final KeyValue, String> five = windowedPair(5, "five", startTime + 5); - assertArrayEquals( - new LinkedHashSet<>(asList(four, two, one)).toArray(), - toSet(windowStore.backwardFetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))).toArray() + assertEquals( + asList(four, two, one), + toList(windowStore.backwardFetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))) ); - assertArrayEquals( - new LinkedHashSet<>(asList(two, one, zero)).toArray(), - toSet(windowStore.backwardFetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))).toArray() + assertEquals( + asList(two, one, zero), + toList(windowStore.backwardFetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))) ); - assertArrayEquals( - new LinkedHashSet<>(asList(five, four, two, one)).toArray(), - toSet(windowStore.backwardFetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))).toArray() + assertEquals( + asList(five, four, two, one), + toList(windowStore.backwardFetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))) ); } @@ -438,69 +439,69 @@ public void testBackwardFetchRange() { final KeyValue, String> four = windowedPair(4, "four", startTime + 4); final KeyValue, String> five = windowedPair(5, "five", startTime + 5); - assertArrayEquals( - new LinkedHashSet<>(asList(one, zero)).toArray(), - toSet(windowStore.backwardFetch( + assertEquals( + asList(one, zero), + toList(windowStore.backwardFetch( 0, 1, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ofEpochMilli(startTime + 0L + WINDOW_SIZE))) ); - assertArrayEquals( - new LinkedHashSet<>(Collections.singletonList(one)).toArray(), - toSet(windowStore.backwardFetch( + assertEquals( + Collections.singletonList(one), + toList(windowStore.backwardFetch( 1, 1, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ofEpochMilli(startTime + 0L + WINDOW_SIZE))) ); - assertArrayEquals( - new LinkedHashSet<>(asList(two, one)).toArray(), - toSet(windowStore.backwardFetch( + assertEquals( + asList(two, one), + toList(windowStore.backwardFetch( 1, 3, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ofEpochMilli(startTime + 0L + WINDOW_SIZE))) ); - assertArrayEquals( - new LinkedHashSet<>(asList(two, one, zero)).toArray(), - toSet(windowStore.backwardFetch( + assertEquals( + asList(two, one, zero), + toList(windowStore.backwardFetch( 0, 5, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ofEpochMilli(startTime + 0L + WINDOW_SIZE))) ); - assertArrayEquals( - new LinkedHashSet<>(asList(five, four, two, one, zero)).toArray(), - toSet(windowStore.backwardFetch( + assertEquals( + asList(five, four, two, one, zero), + toList(windowStore.backwardFetch( 0, 5, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))).toArray() + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))) ); - assertArrayEquals( - new LinkedHashSet<>(asList(five, four, two)).toArray(), - toSet(windowStore.backwardFetch( + assertEquals( + asList(five, four, two), + toList(windowStore.backwardFetch( 0, 5, ofEpochMilli(startTime + 2L), - ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))).toArray() + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))) ); - assertArrayEquals( - new LinkedHashSet<>(Collections.emptyList()).toArray(), - toSet(windowStore.backwardFetch( + assertEquals( + Collections.emptyList(), + toList(windowStore.backwardFetch( 4, 5, ofEpochMilli(startTime + 2L), - ofEpochMilli(startTime + WINDOW_SIZE))).toArray() + ofEpochMilli(startTime + WINDOW_SIZE))) ); - assertArrayEquals( - new LinkedHashSet<>(Collections.emptyList()).toArray(), - toSet(windowStore.backwardFetch( + assertEquals( + Collections.emptyList(), + toList(windowStore.backwardFetch( 0, 3, ofEpochMilli(startTime + 3L), - ofEpochMilli(startTime + WINDOW_SIZE + 5))).toArray() + ofEpochMilli(startTime + WINDOW_SIZE + 5))) ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java index 21cb3a26328a5..6c4ddf6ef6496 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java @@ -663,7 +663,6 @@ public void shouldFetchAndIterateOverKeyRange() { cachingStore.put(bytesKey("a"), bytesValue("0003"), 1); cachingStore.put(bytesKey("aa"), bytesValue("0004"), 1); cachingStore.put(bytesKey("a"), bytesValue("0005"), SEGMENT_INTERVAL); - cachingStore.put(bytesKey("aa"), bytesValue("0006"), SEGMENT_INTERVAL + 1); verifyKeyValueList( asList( @@ -677,8 +676,7 @@ public void shouldFetchAndIterateOverKeyRange() { verifyKeyValueList( asList( windowedPair("aa", "0002", 0), - windowedPair("aa", "0004", 1), - windowedPair("aa", "0006", SEGMENT_INTERVAL + 1)), + windowedPair("aa", "0004", 1)), toList(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) ); @@ -688,8 +686,7 @@ public void shouldFetchAndIterateOverKeyRange() { windowedPair("a", "0003", 1), windowedPair("aa", "0002", 0), windowedPair("aa", "0004", 1), - windowedPair("a", "0005", SEGMENT_INTERVAL), - windowedPair("aa", "0006", SEGMENT_INTERVAL + 1) + windowedPair("a", "0005", SEGMENT_INTERVAL) ), toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) ); @@ -702,7 +699,6 @@ public void shouldFetchAndIterateOverKeyBackwardRange() { cachingStore.put(bytesKey("a"), bytesValue("0003"), 1); cachingStore.put(bytesKey("aa"), bytesValue("0004"), 1); cachingStore.put(bytesKey("a"), bytesValue("0005"), SEGMENT_INTERVAL); - cachingStore.put(bytesKey("aa"), bytesValue("0006"), SEGMENT_INTERVAL + 1); verifyKeyValueList( asList( @@ -715,7 +711,6 @@ public void shouldFetchAndIterateOverKeyBackwardRange() { verifyKeyValueList( asList( - windowedPair("aa", "0006", SEGMENT_INTERVAL + 1), windowedPair("aa", "0004", 1), windowedPair("aa", "0002", 0)), toList(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) @@ -723,7 +718,6 @@ public void shouldFetchAndIterateOverKeyBackwardRange() { verifyKeyValueList( asList( - windowedPair("aa", "0006", SEGMENT_INTERVAL + 1), windowedPair("a", "0005", SEGMENT_INTERVAL), windowedPair("aa", "0004", 1), windowedPair("aa", "0002", 0), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java index 7bc1e92cd3472..0728a1ebf876d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java @@ -30,8 +30,6 @@ import org.junit.Test; import org.junit.runner.RunWith; -import java.time.Instant; - import static java.time.Instant.ofEpochMilli; @RunWith(EasyMockRunner.class) @@ -49,7 +47,6 @@ public class ChangeLoggingWindowBytesStoreTest { private ProcessorContextImpl context; private ChangeLoggingWindowBytesStore store; - @Before public void setUp() { store = new ChangeLoggingWindowBytesStore(inner, false); @@ -100,7 +97,7 @@ public void shouldDelegateToUnderlyingStoreWhenFetching() { @Test public void shouldDelegateToUnderlyingStoreWhenBackwardFetching() { EasyMock - .expect(inner.backwardFetch(bytesKey, Instant.ofEpochMilli(0), Instant.ofEpochMilli(10))) + .expect(inner.backwardFetch(bytesKey, 0, 10)) .andReturn(KeyValueIterators.emptyWindowStoreIterator()); init(); @@ -124,7 +121,7 @@ public void shouldDelegateToUnderlyingStoreWhenFetchingRange() { @Test public void shouldDelegateToUnderlyingStoreWhenBackwardFetchingRange() { EasyMock - .expect(inner.backwardFetch(bytesKey, bytesKey, Instant.ofEpochMilli(0), Instant.ofEpochMilli(1))) + .expect(inner.backwardFetch(bytesKey, bytesKey, 0, 1)) .andReturn(KeyValueIterators.emptyIterator()); init(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java index d2aa9431b2196..9e0ef8d2a2ac3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java @@ -80,16 +80,18 @@ public void shouldPeekAndIterate() { @Test public void shouldThrowNoSuchElementWhenNoMoreItemsLeftAndNextCalled() { - final DelegatingPeekingKeyValueIterator peekingIterator = new DelegatingPeekingKeyValueIterator<>(name, store.all()); - peekingIterator.next(); - assertThrows(NoSuchElementException.class, peekingIterator::close); + try (final DelegatingPeekingKeyValueIterator peekingIterator = + new DelegatingPeekingKeyValueIterator<>(name, store.all())) { + assertThrows(NoSuchElementException.class, peekingIterator::next); + } } @Test public void shouldThrowNoSuchElementWhenNoMoreItemsLeftAndPeekNextCalled() { - final DelegatingPeekingKeyValueIterator peekingIterator = new DelegatingPeekingKeyValueIterator<>(name, store.all()); - peekingIterator.peekNextKey(); - assertThrows(NoSuchElementException.class, peekingIterator::close); + try (final DelegatingPeekingKeyValueIterator peekingIterator = + new DelegatingPeekingKeyValueIterator<>(name, store.all())) { + assertThrows(NoSuchElementException.class, peekingIterator::peekNextKey); + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java index 9407dab54c9d6..7c477217a3e4f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java @@ -79,17 +79,15 @@ public WindowStoreIterator fetch(final K key, final long timeFrom, final long } @Override - public WindowStoreIterator fetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException { + public WindowStoreIterator fetch(final K key, final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { return fetch( key, - ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")), - ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to"))); + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "from")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "to"))); } @Override - public WindowStoreIterator backwardFetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException { - final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); - final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + public WindowStoreIterator backwardFetch(final K key, final long timeFrom, final long timeTo) throws IllegalArgumentException { if (!open) { throw new InvalidStateStoreException("Store is not open"); } @@ -103,6 +101,16 @@ public WindowStoreIterator backwardFetch(final K key, final Instant from, fin return new TheWindowStoreIterator<>(results.iterator()); } + @Override + public WindowStoreIterator backwardFetch(final K key, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return backwardFetch( + key, + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); + } + @Override public KeyValueIterator, V> all() { if (!open) { @@ -225,17 +233,14 @@ public KeyValue, V> next() { } @Override - public KeyValueIterator, V> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException { + public KeyValueIterator, V> fetchAll(final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { return fetchAll( - ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")), - ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to"))); + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "from")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "to"))); } @Override - public KeyValueIterator, V> backwardFetchAll(final Instant from, final Instant to) throws IllegalArgumentException { - final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); - final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); - + public KeyValueIterator, V> backwardFetchAll(final long timeFrom, final long timeTo) throws IllegalArgumentException { if (!open) { throw new InvalidStateStoreException("Store is not open"); } @@ -276,6 +281,13 @@ public KeyValue, V> next() { }; } + @Override + public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, final Instant timeTo) { + return backwardFetchAll( + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); + } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { @@ -319,22 +331,20 @@ public KeyValue, V> next() { @Override public KeyValueIterator, V> fetch(final K from, final K to, - final Instant fromTime, - final Instant toTime) throws IllegalArgumentException { + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { return fetch( from, to, - ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")), - ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime"))); + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "fromTime")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "toTime"))); } @Override public KeyValueIterator, V> backwardFetch(final K from, final K to, - final Instant fromTimeInstant, - final Instant toTimeInstant) throws IllegalArgumentException { - final long timeFrom = ApiUtils.validateMillisecondInstant(fromTimeInstant, prepareMillisCheckFailMsgPrefix(fromTimeInstant, "fromTimeInstant")); - final long timeTo = ApiUtils.validateMillisecondInstant(toTimeInstant, prepareMillisCheckFailMsgPrefix(toTimeInstant, "toTimeInstant")); + final long timeFrom, + final long timeTo) throws IllegalArgumentException { if (!open) { throw new InvalidStateStoreException("Store is not open"); } @@ -372,6 +382,18 @@ public KeyValue, V> next() { }; } + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final Instant timeFrom, + final Instant timeTo) { + return backwardFetch( + from, + to, + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "fromTime")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "toTime"))); + } + public void put(final K key, final V value, final long timestamp) { if (!data.containsKey(timestamp)) { data.put(timestamp, new TreeMap<>()); From a98827659a461e32641ea9781476b829f41e1c14 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 27 Aug 2020 15:18:39 +0100 Subject: [PATCH 23/29] fix: remove redundant array/set/list equals assertions --- .../state/internals/InMemoryWindowStore.java | 63 ++++++++---- .../AbstractWindowBytesStoreTest.java | 95 +++++++++---------- .../internals/ReadOnlyWindowStoreStub.java | 7 +- 3 files changed, 94 insertions(+), 71 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java index 8217c9745e137..41a824d5325b6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java @@ -185,13 +185,15 @@ WindowStoreIterator fetch(final Bytes key, final long timeFrom, final lo return registerNewWindowStoreIterator( key, segmentMap.subMap(minTime, true, timeTo, true) - .entrySet().iterator() + .entrySet().iterator(), + true ); } else { return registerNewWindowStoreIterator( key, segmentMap.subMap(minTime, true, timeTo, true) - .descendingMap().entrySet().iterator() + .descendingMap().entrySet().iterator(), + false ); } } @@ -243,14 +245,17 @@ KeyValueIterator, byte[]> fetch(final Bytes from, from, to, segmentMap.subMap(minTime, true, timeTo, true) - .entrySet().iterator() + .entrySet().iterator(), + true ); } else { return registerNewWindowedKeyValueIterator( from, to, segmentMap.subMap(minTime, true, timeTo, true) - .descendingMap().entrySet().iterator()); + .descendingMap().entrySet().iterator(), + false + ); } } @@ -280,13 +285,17 @@ KeyValueIterator, byte[]> fetchAll(final long timeFrom, final lo null, null, segmentMap.subMap(minTime, true, timeTo, true) - .entrySet().iterator()); + .entrySet().iterator(), + true + ); } else { return registerNewWindowedKeyValueIterator( null, null, segmentMap.subMap(minTime, true, timeTo, true) - .descendingMap().entrySet().iterator()); + .descendingMap().entrySet().iterator(), + false + ); } } @@ -297,7 +306,11 @@ public KeyValueIterator, byte[]> all() { final long minTime = observedStreamTime - retentionPeriod; return registerNewWindowedKeyValueIterator( - null, null, segmentMap.tailMap(minTime, false).entrySet().iterator()); + null, + null, + segmentMap.tailMap(minTime, false).entrySet().iterator(), + true + ); } @Override @@ -307,7 +320,11 @@ public KeyValueIterator, byte[]> backwardAll() { final long minTime = observedStreamTime - retentionPeriod; return registerNewWindowedKeyValueIterator( - null, null, segmentMap.tailMap(minTime, false).descendingMap().entrySet().iterator()); + null, + null, + segmentMap.tailMap(minTime, false).descendingMap().entrySet().iterator(), + false + ); } @Override @@ -367,12 +384,13 @@ private static Bytes getKey(final Bytes keyBytes) { } private WrappedInMemoryWindowStoreIterator registerNewWindowStoreIterator(final Bytes key, - final Iterator>> segmentIterator) { + final Iterator>> segmentIterator, + final boolean forward) { final Bytes keyFrom = retainDuplicates ? wrapForDups(key, 0) : key; final Bytes keyTo = retainDuplicates ? wrapForDups(key, Integer.MAX_VALUE) : key; final WrappedInMemoryWindowStoreIterator iterator = - new WrappedInMemoryWindowStoreIterator(keyFrom, keyTo, segmentIterator, openIterators::remove, retainDuplicates); + new WrappedInMemoryWindowStoreIterator(keyFrom, keyTo, segmentIterator, openIterators::remove, retainDuplicates, forward); openIterators.add(iterator); return iterator; @@ -380,7 +398,8 @@ private WrappedInMemoryWindowStoreIterator registerNewWindowStoreIterator(final private WrappedWindowedKeyValueIterator registerNewWindowedKeyValueIterator(final Bytes keyFrom, final Bytes keyTo, - final Iterator>> segmentIterator) { + final Iterator>> segmentIterator, + final boolean forward) { final Bytes from = (retainDuplicates && keyFrom != null) ? wrapForDups(keyFrom, 0) : keyFrom; final Bytes to = (retainDuplicates && keyTo != null) ? wrapForDups(keyTo, Integer.MAX_VALUE) : keyTo; @@ -390,7 +409,8 @@ private WrappedWindowedKeyValueIterator registerNewWindowedKeyValueIterator(fina segmentIterator, openIterators::remove, retainDuplicates, - windowSize); + windowSize, + forward); openIterators.add(iterator); return iterator; } @@ -410,18 +430,21 @@ private static abstract class InMemoryWindowStoreIteratorWrapper { private final Bytes keyFrom; private final Bytes keyTo; private final Iterator>> segmentIterator; - private final boolean retainDuplicates; private final ClosingCallback callback; + private final boolean retainDuplicates; + private final boolean forward; InMemoryWindowStoreIteratorWrapper(final Bytes keyFrom, final Bytes keyTo, final Iterator>> segmentIterator, final ClosingCallback callback, - final boolean retainDuplicates) { + final boolean retainDuplicates, + final boolean forward) { this.keyFrom = keyFrom; this.keyTo = keyTo; allKeys = (keyFrom == null) && (keyTo == null); this.retainDuplicates = retainDuplicates; + this.forward = forward; this.segmentIterator = segmentIterator; this.callback = callback; @@ -501,8 +524,9 @@ private static class WrappedInMemoryWindowStoreIterator extends InMemoryWindowSt final Bytes keyTo, final Iterator>> segmentIterator, final ClosingCallback callback, - final boolean retainDuplicates) { - super(keyFrom, keyTo, segmentIterator, callback, retainDuplicates); + final boolean retainDuplicates, + final boolean forward) { + super(keyFrom, keyTo, segmentIterator, callback, retainDuplicates, forward); } @Override @@ -526,7 +550,7 @@ public KeyValue next() { public static WrappedInMemoryWindowStoreIterator emptyIterator() { return new WrappedInMemoryWindowStoreIterator(null, null, null, it -> { - }, false); + }, false, true); } } @@ -541,8 +565,9 @@ private static class WrappedWindowedKeyValueIterator final Iterator>> segmentIterator, final ClosingCallback callback, final boolean retainDuplicates, - final long windowSize) { - super(keyFrom, keyTo, segmentIterator, callback, retainDuplicates); + final long windowSize, + final boolean forward) { + super(keyFrom, keyTo, segmentIterator, callback, retainDuplicates, forward); this.windowSize = windowSize; } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java index 0366fe57b6cbe..3261ce8d56334 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java @@ -66,7 +66,6 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; @@ -273,9 +272,9 @@ public void shouldGetAll() { final KeyValue, String> four = windowedPair(4, "four", startTime + 4); final KeyValue, String> five = windowedPair(5, "five", startTime + 5); - assertArrayEquals( - new LinkedHashSet<>(asList(zero, one, two, four, five)).toArray(), - toSet(windowStore.all()).toArray() + assertEquals( + asList(zero, one, two, four, five), + toList(windowStore.all()) ); } @@ -291,9 +290,9 @@ public void shouldGetBackwardAll() { final KeyValue, String> four = windowedPair(4, "four", startTime + 4); final KeyValue, String> five = windowedPair(5, "five", startTime + 5); - assertArrayEquals( - new LinkedHashSet<>(asList(five, four, two, one, zero)).toArray(), - toSet(windowStore.backwardAll()).toArray() + assertEquals( + asList(five, four, two, one, zero), + toList(windowStore.backwardAll()) ); } @@ -309,17 +308,17 @@ public void shouldFetchAllInTimeRange() { final KeyValue, String> four = windowedPair(4, "four", startTime + 4); final KeyValue, String> five = windowedPair(5, "five", startTime + 5); - assertArrayEquals( - new LinkedHashSet<>(asList(one, two, four)).toArray(), - toSet(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))).toArray() + assertEquals( + asList(one, two, four), + toList(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))) ); - assertArrayEquals( - new LinkedHashSet<>(asList(zero, one, two)).toArray(), - toSet(windowStore.fetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))).toArray() + assertEquals( + asList(zero, one, two), + toList(windowStore.fetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))) ); - assertArrayEquals( - new LinkedHashSet<>(asList(one, two, four, five)).toArray(), - toSet(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))).toArray() + assertEquals( + asList(one, two, four, five), + toList(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))) ); } @@ -361,69 +360,69 @@ public void testFetchRange() { final KeyValue, String> four = windowedPair(4, "four", startTime + 4); final KeyValue, String> five = windowedPair(5, "five", startTime + 5); - assertArrayEquals( - new LinkedHashSet<>(asList(zero, one)).toArray(), - toSet(windowStore.fetch( + assertEquals( + asList(zero, one), + toList(windowStore.fetch( 0, 1, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ofEpochMilli(startTime + 0L + WINDOW_SIZE))) ); - assertArrayEquals( - new LinkedHashSet<>(Collections.singletonList(one)).toArray(), - toSet(windowStore.fetch( + assertEquals( + Collections.singletonList(one), + toList(windowStore.fetch( 1, 1, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ofEpochMilli(startTime + 0L + WINDOW_SIZE))) ); - assertArrayEquals( - new LinkedHashSet<>(asList(one, two)).toArray(), - toSet(windowStore.fetch( + assertEquals( + asList(one, two), + toList(windowStore.fetch( 1, 3, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ofEpochMilli(startTime + 0L + WINDOW_SIZE))) ); - assertArrayEquals( - new LinkedHashSet<>(asList(zero, one, two)).toArray(), - toSet(windowStore.fetch( + assertEquals( + asList(zero, one, two), + toList(windowStore.fetch( 0, 5, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ofEpochMilli(startTime + 0L + WINDOW_SIZE))) ); - assertArrayEquals( - new LinkedHashSet<>(asList(zero, one, two, four, five)).toArray(), - toSet(windowStore.fetch( + assertEquals( + asList(zero, one, two, four, five), + toList(windowStore.fetch( 0, 5, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))).toArray() + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))) ); - assertArrayEquals( - new LinkedHashSet<>(asList(two, four, five)).toArray(), - toSet(windowStore.fetch( + assertEquals( + asList(two, four, five), + toList(windowStore.fetch( 0, 5, ofEpochMilli(startTime + 2L), - ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))).toArray() + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))) ); - assertArrayEquals( - new LinkedHashSet<>(Collections.emptyList()).toArray(), - toSet(windowStore.fetch( + assertEquals( + Collections.emptyList(), + toList(windowStore.fetch( 4, 5, ofEpochMilli(startTime + 2L), - ofEpochMilli(startTime + WINDOW_SIZE))).toArray() + ofEpochMilli(startTime + WINDOW_SIZE))) ); - assertArrayEquals( - new LinkedHashSet<>(Collections.emptyList()).toArray(), - toSet(windowStore.fetch( + assertEquals( + Collections.emptyList(), + toList(windowStore.fetch( 0, 3, ofEpochMilli(startTime + 3L), - ofEpochMilli(startTime + WINDOW_SIZE + 5))).toArray() + ofEpochMilli(startTime + WINDOW_SIZE + 5))) ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java index 7c477217a3e4f..f724047498fe6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java @@ -29,7 +29,6 @@ import java.time.Instant; import java.util.ArrayList; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -45,7 +44,7 @@ public class ReadOnlyWindowStoreStub implements ReadOnlyWindowStore, StateStore { private final long windowSize; - private final Map> data = new HashMap<>(); + private final NavigableMap> data = new TreeMap<>(); private boolean open = true; ReadOnlyWindowStoreStub(final long windowSize) { @@ -156,7 +155,7 @@ public KeyValueIterator, V> backwardAll() { throw new InvalidStateStoreException("Store is not open"); } final List, V>> results = new ArrayList<>(); - for (final long now : data.keySet()) { + for (final long now : data.descendingKeySet()) { final NavigableMap kvMap = data.get(now); if (kvMap != null) { for (final Entry entry : kvMap.descendingMap().entrySet()) { @@ -245,7 +244,7 @@ public KeyValueIterator, V> backwardFetchAll(final long timeFrom, fi throw new InvalidStateStoreException("Store is not open"); } final List, V>> results = new ArrayList<>(); - for (final long now : data.keySet()) { + for (final long now : data.descendingKeySet()) { if (!(now >= timeFrom && now <= timeTo)) { continue; } From 4ec40063d629db053853b5c77caa535b5644edd9 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 1 Sep 2020 19:13:40 +0100 Subject: [PATCH 24/29] fix exception comment case --- .../kafka/streams/state/ReadOnlyWindowStore.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java index 83104d21fd90d..e94439d432b03 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java @@ -37,7 +37,7 @@ public interface ReadOnlyWindowStore { * @param time start timestamp (inclusive) of the window * @return The value or {@code null} if no value is found in the window * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for any key. + * @throws NullPointerException if {@code null} is used for any key. */ V fetch(K key, long time); @@ -72,7 +72,7 @@ public interface ReadOnlyWindowStore { * @param timeTo time range end (inclusive), where iteration ends. * @return an iterator over key-value pairs {@code }, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for key. + * @throws NullPointerException if {@code null} is used for key. * @deprecated Use {@link #fetch(Object, Instant, Instant)} instead */ @Deprecated @@ -109,7 +109,7 @@ public interface ReadOnlyWindowStore { * @param timeTo time range end (inclusive), where iteration ends. * @return an iterator over key-value pairs {@code }, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for key. + * @throws NullPointerException if {@code null} is used for key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ WindowStoreIterator fetch(K key, Instant timeFrom, Instant timeTo) throws IllegalArgumentException; @@ -147,7 +147,7 @@ public interface ReadOnlyWindowStore { * @param timeTo time range end (inclusive), where iteration starts. * @return an iterator over key-value pairs {@code }, from end to beginning of time. * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for key. + * @throws NullPointerException if {@code null} is used for key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ WindowStoreIterator backwardFetch(K key, long timeFrom, long timeTo); @@ -165,7 +165,7 @@ public interface ReadOnlyWindowStore { * @param timeTo time range end (inclusive), where iteration ends. * @return an iterator over windowed key-value pairs {@code , value>}, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for any key. + * @throws NullPointerException if {@code null} is used for any key. * @deprecated Use {@link #fetch(Object, Object, Instant, Instant)} instead */ @Deprecated @@ -182,7 +182,7 @@ public interface ReadOnlyWindowStore { * @param timeTo time range end (inclusive), where iteration ends. * @return an iterator over windowed key-value pairs {@code , value>}, from beginning to end of time. * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for any key. + * @throws NullPointerException if {@code null} is used for any key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ KeyValueIterator, V> fetch(K from, K to, Instant timeFrom, Instant timeTo) @@ -200,7 +200,7 @@ KeyValueIterator, V> fetch(K from, K to, Instant timeFrom, Instant t * @param timeTo time range end (inclusive), where iteration starts. * @return an iterator over windowed key-value pairs {@code , value>}, from end to beginning of time. * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for any key. + * @throws NullPointerException if {@code null} is used for any key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ KeyValueIterator, V> backwardFetch(K from, K to, long timeFrom, long timeTo); From 814ef3bcc82bc1241d5b732c2ce1e00c2308cb73 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 2 Sep 2020 09:59:18 +0100 Subject: [PATCH 25/29] align var names and exceptions --- .../internals/AbstractReadOnlyDecorator.java | 12 ++-- .../internals/AbstractReadWriteDecorator.java | 12 ++-- .../streams/state/ReadOnlyWindowStore.java | 24 +++---- .../kafka/streams/state/WindowStore.java | 58 +++++++-------- .../state/internals/CachingWindowStore.java | 32 ++++----- .../CompositeReadOnlyWindowStore.java | 70 ++++++------------- .../state/internals/InMemoryWindowStore.java | 12 ++-- .../state/internals/MeteredWindowStore.java | 12 ++-- .../internals/ReadOnlyWindowStoreFacade.java | 39 +++-------- .../state/internals/RocksDBWindowStore.java | 12 ++-- .../TimestampedWindowStoreBuilder.java | 12 ++-- ...owToTimestampedWindowByteStoreAdapter.java | 56 ++++++++++----- .../kafka/streams/state/NoOpWindowStore.java | 32 ++------- .../AbstractWindowBytesStoreTest.java | 1 - .../internals/ReadOnlyWindowStoreStub.java | 61 +++++----------- .../streams/internals/WindowStoreFacade.java | 20 ++++++ 16 files changed, 200 insertions(+), 265 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java index ecd7903abd1ab..74227af3fa20e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java @@ -189,19 +189,19 @@ public WindowStoreIterator backwardFetch(final K key, @Override @Deprecated - public KeyValueIterator, V> fetch(final K from, - final K to, + public KeyValueIterator, V> fetch(final K keyFrom, + final K keyTo, final long timeFrom, final long timeTo) { - return wrapped().fetch(from, to, timeFrom, timeTo); + return wrapped().fetch(keyFrom, keyTo, timeFrom, timeTo); } @Override - public KeyValueIterator, V> backwardFetch(final K from, - final K to, + public KeyValueIterator, V> backwardFetch(final K keyFrom, + final K keyTo, final long timeFrom, final long timeTo) { - return wrapped().backwardFetch(from, to, timeFrom, timeTo); + return wrapped().backwardFetch(keyFrom, keyTo, timeFrom, timeTo); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java index be2d097218dc1..41026c4eb5283 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java @@ -182,19 +182,19 @@ public WindowStoreIterator backwardFetch(final K key, @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override - public KeyValueIterator, V> fetch(final K from, - final K to, + public KeyValueIterator, V> fetch(final K keyFrom, + final K keyTo, final long timeFrom, final long timeTo) { - return wrapped().fetch(from, to, timeFrom, timeTo); + return wrapped().fetch(keyFrom, keyTo, timeFrom, timeTo); } @Override - public KeyValueIterator, V> backwardFetch(final K from, - final K to, + public KeyValueIterator, V> backwardFetch(final K keyFrom, + final K keyTo, final long timeFrom, final long timeTo) { - return wrapped().backwardFetch(from, to, timeFrom, timeTo); + return wrapped().backwardFetch(keyFrom, keyTo, timeFrom, timeTo); } @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java index e94439d432b03..71a31f7dd303f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java @@ -150,8 +150,6 @@ public interface ReadOnlyWindowStore { * @throws NullPointerException if {@code null} is used for key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ - WindowStoreIterator backwardFetch(K key, long timeFrom, long timeTo); - WindowStoreIterator backwardFetch(K key, Instant timeFrom, Instant timeTo) throws IllegalArgumentException; /** @@ -159,8 +157,8 @@ public interface ReadOnlyWindowStore { *

* This iterator must be closed after use. * - * @param from the first key in the range - * @param to the last key in the range + * @param keyFrom the first key in the range + * @param keyTo the last key in the range * @param timeFrom time range start (inclusive), where iteration starts. * @param timeTo time range end (inclusive), where iteration ends. * @return an iterator over windowed key-value pairs {@code , value>}, from beginning to end of time. @@ -169,15 +167,15 @@ public interface ReadOnlyWindowStore { * @deprecated Use {@link #fetch(Object, Object, Instant, Instant)} instead */ @Deprecated - KeyValueIterator, V> fetch(K from, K to, long timeFrom, long timeTo); + KeyValueIterator, V> fetch(K keyFrom, K keyTo, long timeFrom, long timeTo); /** * Get all the key-value pairs in the given key range and time range from all the existing windows. *

* This iterator must be closed after use. * - * @param from the first key in the range - * @param to the last key in the range + * @param keyFrom the first key in the range + * @param keyTo the last key in the range * @param timeFrom time range start (inclusive), where iteration starts. * @param timeTo time range end (inclusive), where iteration ends. * @return an iterator over windowed key-value pairs {@code , value>}, from beginning to end of time. @@ -185,7 +183,7 @@ public interface ReadOnlyWindowStore { * @throws NullPointerException if {@code null} is used for any key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ - KeyValueIterator, V> fetch(K from, K to, Instant timeFrom, Instant timeTo) + KeyValueIterator, V> fetch(K keyFrom, K keyTo, Instant timeFrom, Instant timeTo) throws IllegalArgumentException; /** @@ -194,8 +192,8 @@ KeyValueIterator, V> fetch(K from, K to, Instant timeFrom, Instant t *

* This iterator must be closed after use. * - * @param from the first key in the range - * @param to the last key in the range + * @param keyFrom the first key in the range + * @param keyTo the last key in the range * @param timeFrom time range start (inclusive), where iteration ends. * @param timeTo time range end (inclusive), where iteration starts. * @return an iterator over windowed key-value pairs {@code , value>}, from end to beginning of time. @@ -203,9 +201,7 @@ KeyValueIterator, V> fetch(K from, K to, Instant timeFrom, Instant t * @throws NullPointerException if {@code null} is used for any key. * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ - KeyValueIterator, V> backwardFetch(K from, K to, long timeFrom, long timeTo); - - KeyValueIterator, V> backwardFetch(K from, K to, Instant timeFrom, Instant timeTo) + KeyValueIterator, V> backwardFetch(K keyFrom, K keyTo, Instant timeFrom, Instant timeTo) throws IllegalArgumentException; @@ -264,7 +260,5 @@ default KeyValueIterator, V> backwardAll() { * @throws NullPointerException if {@code null} is used for any key * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ - KeyValueIterator, V> backwardFetchAll(long timeFrom, long timeTo); - KeyValueIterator, V> backwardFetchAll(Instant timeFrom, Instant timeTo) throws IllegalArgumentException; } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java index 83254a08256e8..7da9ce0172b79 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java @@ -28,7 +28,7 @@ /** * Interface for storing the aggregated values of fixed-size time windows. *

- * Note, that the stores's physical key type is {@link Windowed Windowed<K>}. + * Note, that the stores' physical key type is {@link Windowed Windowed<K>}. * * @param Type of keys * @param Type of values @@ -106,23 +106,21 @@ public interface WindowStore extends StateStore, ReadOnlyWindowStore @Override default WindowStoreIterator fetch(final K key, final Instant timeFrom, - final Instant timeTo) { + final Instant timeTo) throws IllegalArgumentException { return fetch( key, ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } - default WindowStoreIterator backwardFetch(final K key, - final long timeFrom, - final long timeTo) throws IllegalArgumentException { - throw new UnsupportedOperationException(); - } + WindowStoreIterator backwardFetch(final K key, + final long timeFrom, + final long timeTo); @Override default WindowStoreIterator backwardFetch(final K key, final Instant timeFrom, - final Instant timeTo) { + final Instant timeTo) throws IllegalArgumentException { return backwardFetch( key, ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), @@ -134,8 +132,8 @@ default WindowStoreIterator backwardFetch(final K key, *

* This iterator must be closed after use. * - * @param from the first key in the range - * @param to the last key in the range + * @param keyFrom the first key in the range + * @param keyTo the last key in the range * @param timeFrom time range start (inclusive) * @param timeTo time range end (inclusive) * @return an iterator over windowed key-value pairs {@code , value>} @@ -144,35 +142,33 @@ default WindowStoreIterator backwardFetch(final K key, */ // note, this method must be kept if super#fetch(...) is removed @SuppressWarnings("deprecation") - KeyValueIterator, V> fetch(K from, K to, long timeFrom, long timeTo); + KeyValueIterator, V> fetch(K keyFrom, K keyTo, long timeFrom, long timeTo); @Override - default KeyValueIterator, V> fetch(final K from, - final K to, + default KeyValueIterator, V> fetch(final K keyFrom, + final K keyTo, final Instant timeFrom, - final Instant timeTo) { + final Instant timeTo) throws IllegalArgumentException { return fetch( - from, - to, + keyFrom, + keyTo, ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } - default KeyValueIterator, V> backwardFetch(final K from, - final K to, - final long timeFrom, - final long timeTo) { - throw new UnsupportedOperationException(); - } + KeyValueIterator, V> backwardFetch(final K keyFrom, + final K keyTo, + final long timeFrom, + final long timeTo); @Override - default KeyValueIterator, V> backwardFetch(final K from, - final K to, + default KeyValueIterator, V> backwardFetch(final K keyFrom, + final K keyTo, final Instant timeFrom, - final Instant timeTo) { + final Instant timeTo) throws IllegalArgumentException { return backwardFetch( - from, - to, + keyFrom, + keyTo, ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } @@ -190,18 +186,16 @@ default KeyValueIterator, V> backwardFetch(final K from, KeyValueIterator, V> fetchAll(long timeFrom, long timeTo); @Override - default KeyValueIterator, V> fetchAll(final Instant timeFrom, final Instant timeTo) { + default KeyValueIterator, V> fetchAll(final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { return fetchAll( ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } - default KeyValueIterator, V> backwardFetchAll(final long timeFrom, final long timeTo) { - throw new UnsupportedOperationException(); - } + KeyValueIterator, V> backwardFetchAll(final long timeFrom, final long timeTo); @Override - default KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, final Instant timeTo) { + default KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { return backwardFetchAll( ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index fbabfc2c617a6..20be3a3a33bd4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -244,11 +244,11 @@ public synchronized WindowStoreIterator backwardFetch(final Bytes key, @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override - public KeyValueIterator, byte[]> fetch(final Bytes from, - final Bytes to, + public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, + final Bytes keyTo, final long timeFrom, final long timeTo) { - if (from.compareTo(to) > 0) { + if (keyFrom.compareTo(keyTo) > 0) { LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + "This may be due to range arguments set in the wrong order, " + "or serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + @@ -261,20 +261,20 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, validateStoreOpen(); final KeyValueIterator, byte[]> underlyingIterator = - wrapped().fetch(from, to, timeFrom, timeTo); + wrapped().fetch(keyFrom, keyTo, timeFrom, timeTo); if (context.cache() == null) { return underlyingIterator; } final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? - new CacheIteratorWrapper(from, to, timeFrom, timeTo, true) : + new CacheIteratorWrapper(keyFrom, keyTo, timeFrom, timeTo, true) : context.cache().range( cacheName, - cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), - cacheFunction.cacheKey(keySchema.upperRange(to, timeTo)) + cacheFunction.cacheKey(keySchema.lowerRange(keyFrom, timeFrom)), + cacheFunction.cacheKey(keySchema.upperRange(keyTo, timeTo)) ); - final HasNextCondition hasNextCondition = keySchema.hasNextCondition(from, to, timeFrom, timeTo); + final HasNextCondition hasNextCondition = keySchema.hasNextCondition(keyFrom, keyTo, timeFrom, timeTo); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); @@ -289,11 +289,11 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, } @Override - public KeyValueIterator, byte[]> backwardFetch(final Bytes from, - final Bytes to, + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, + final Bytes keyTo, final long timeFrom, final long timeTo) { - if (from.compareTo(to) > 0) { + if (keyFrom.compareTo(keyTo) > 0) { LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + "Note that the built-in numerical serdes do not follow this for negative numbers"); @@ -305,20 +305,20 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes from, validateStoreOpen(); final KeyValueIterator, byte[]> underlyingIterator = - wrapped().backwardFetch(from, to, timeFrom, timeTo); + wrapped().backwardFetch(keyFrom, keyTo, timeFrom, timeTo); if (context.cache() == null) { return underlyingIterator; } final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? - new CacheIteratorWrapper(from, to, timeFrom, timeTo, false) : + new CacheIteratorWrapper(keyFrom, keyTo, timeFrom, timeTo, false) : context.cache().reverseRange( cacheName, - cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), - cacheFunction.cacheKey(keySchema.upperRange(to, timeTo)) + cacheFunction.cacheKey(keySchema.lowerRange(keyFrom, timeFrom)), + cacheFunction.cacheKey(keySchema.upperRange(keyTo, timeTo)) ); - final HasNextCondition hasNextCondition = keySchema.hasNextCondition(from, to, timeFrom, timeTo); + final HasNextCondition hasNextCondition = keySchema.hasNextCondition(keyFrom, keyTo, timeFrom, timeTo); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java index d5b22e6ba35cc..f062d758571fc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java @@ -104,8 +104,8 @@ public WindowStoreIterator fetch(final K key, @Override public WindowStoreIterator backwardFetch(final K key, - final long timeFrom, - final long timeTo) { + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { Objects.requireNonNull(key, "key can't be null"); final List> stores = provider.stores(storeName, windowStoreType); for (final ReadOnlyWindowStore windowStore : stores) { @@ -125,26 +125,16 @@ public WindowStoreIterator backwardFetch(final K key, return KeyValueIterators.emptyWindowStoreIterator(); } - @Override - public WindowStoreIterator backwardFetch(final K key, - final Instant timeFrom, - final Instant timeTo) throws IllegalArgumentException { - return backwardFetch( - key, - ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), - ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); - } - @SuppressWarnings("deprecation") // removing fetch(K from, K to, long from, long to) will fix this @Override - public KeyValueIterator, V> fetch(final K from, - final K to, + public KeyValueIterator, V> fetch(final K keyFrom, + final K keyTo, final long timeFrom, final long timeTo) { - Objects.requireNonNull(from, "from can't be null"); - Objects.requireNonNull(to, "to can't be null"); + Objects.requireNonNull(keyFrom, "from can't be null"); + Objects.requireNonNull(keyTo, "to can't be null"); final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = - store -> store.fetch(from, to, timeFrom, timeTo); + store -> store.fetch(keyFrom, keyTo, timeFrom, timeTo); return new DelegatingPeekingKeyValueIterator<>( storeName, new CompositeKeyValueIterator<>( @@ -153,26 +143,26 @@ public KeyValueIterator, V> fetch(final K from, } @Override - public KeyValueIterator, V> fetch(final K from, - final K to, + public KeyValueIterator, V> fetch(final K keyFrom, + final K keyTo, final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { return fetch( - from, - to, + keyFrom, + keyTo, ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } @Override - public KeyValueIterator, V> backwardFetch(final K from, - final K to, - final long timeFrom, - final long timeTo) throws IllegalArgumentException { - Objects.requireNonNull(from, "from can't be null"); - Objects.requireNonNull(to, "to can't be null"); + public KeyValueIterator, V> backwardFetch(final K keyFrom, + final K keyTo, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + Objects.requireNonNull(keyFrom, "keyFrom can't be null"); + Objects.requireNonNull(keyTo, "keyTo can't be null"); final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = - store -> store.backwardFetch(from, to, timeFrom, timeTo); + store -> store.backwardFetch(keyFrom, keyTo, timeFrom, timeTo); return new DelegatingPeekingKeyValueIterator<>( storeName, new CompositeKeyValueIterator<>( @@ -180,18 +170,6 @@ public KeyValueIterator, V> backwardFetch(final K from, nextIteratorFunction)); } - @Override - public KeyValueIterator, V> backwardFetch(final K from, - final K to, - final Instant timeFrom, - final Instant timeTo) { - return backwardFetch( - from, - to, - ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), - ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); - } - @Override public KeyValueIterator, V> all() { final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = @@ -237,8 +215,8 @@ public KeyValueIterator, V> fetchAll(final Instant timeFrom, } @Override - public KeyValueIterator, V> backwardFetchAll(final long timeFrom, - final long timeTo) throws IllegalArgumentException { + public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = store -> store.backwardFetchAll(timeFrom, timeTo); return new DelegatingPeekingKeyValueIterator<>( @@ -247,12 +225,4 @@ public KeyValueIterator, V> backwardFetchAll(final long timeFrom, provider.stores(storeName, windowStoreType).iterator(), nextIteratorFunction)); } - - @Override - public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, - final Instant timeTo) { - return backwardFetchAll( - ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), - ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); - } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java index 41a824d5325b6..b538edc762c29 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java @@ -200,19 +200,19 @@ WindowStoreIterator fetch(final Bytes key, final long timeFrom, final lo @Deprecated @Override - public KeyValueIterator, byte[]> fetch(final Bytes from, - final Bytes to, + public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, + final Bytes keyTo, final long timeFrom, final long timeTo) { - return fetch(from, to, timeFrom, timeTo, true); + return fetch(keyFrom, keyTo, timeFrom, timeTo, true); } @Override - public KeyValueIterator, byte[]> backwardFetch(final Bytes from, - final Bytes to, + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, + final Bytes keyTo, final long timeFrom, final long timeTo) { - return fetch(from, to, timeFrom, timeTo, false); + return fetch(keyFrom, keyTo, timeFrom, timeTo, false); } KeyValueIterator, byte[]> fetch(final Bytes from, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index e7315dcdf5354..6e3dc58fb4273 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -184,12 +184,12 @@ public WindowStoreIterator backwardFetch(final K key, @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed @Override - public KeyValueIterator, V> fetch(final K from, - final K to, + public KeyValueIterator, V> fetch(final K keyFrom, + final K keyTo, final long timeFrom, final long timeTo) { return new MeteredWindowedKeyValueIterator<>( - wrapped().fetch(keyBytes(from), keyBytes(to), timeFrom, timeTo), + wrapped().fetch(keyBytes(keyFrom), keyBytes(keyTo), timeFrom, timeTo), fetchSensor, streamsMetrics, serdes, @@ -197,12 +197,12 @@ public KeyValueIterator, V> fetch(final K from, } @Override - public KeyValueIterator, V> backwardFetch(final K from, - final K to, + public KeyValueIterator, V> backwardFetch(final K keyFrom, + final K keyTo, final long timeFrom, final long timeTo) { return new MeteredWindowedKeyValueIterator<>( - wrapped().backwardFetch(keyBytes(from), keyBytes(to), timeFrom, timeTo), + wrapped().backwardFetch(keyBytes(keyFrom), keyBytes(keyTo), timeFrom, timeTo), fetchSensor, streamsMetrics, serdes, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java index b5464b1db0983..b9fdebeca7c20 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java @@ -63,44 +63,29 @@ public WindowStoreIterator backwardFetch(final K key, return new WindowStoreIteratorFacade<>(inner.backwardFetch(key, timeFrom, timeTo)); } - @Override - public WindowStoreIterator backwardFetch(final K key, - final long timeFrom, - final long timeTo) throws IllegalArgumentException { - return new WindowStoreIteratorFacade<>(inner.backwardFetch(key, timeFrom, timeTo)); - } - @Override @SuppressWarnings("deprecation") - public KeyValueIterator, V> fetch(final K from, - final K to, + public KeyValueIterator, V> fetch(final K keyFrom, + final K keyTo, final long timeFrom, final long timeTo) { - return new KeyValueIteratorFacade<>(inner.fetch(from, to, timeFrom, timeTo)); + return new KeyValueIteratorFacade<>(inner.fetch(keyFrom, keyTo, timeFrom, timeTo)); } @Override - public KeyValueIterator, V> fetch(final K from, - final K to, + public KeyValueIterator, V> fetch(final K keyFrom, + final K keyTo, final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { - return new KeyValueIteratorFacade<>(inner.fetch(from, to, timeFrom, timeTo)); + return new KeyValueIteratorFacade<>(inner.fetch(keyFrom, keyTo, timeFrom, timeTo)); } @Override - public KeyValueIterator, V> backwardFetch(final K from, - final K to, + public KeyValueIterator, V> backwardFetch(final K keyFrom, + final K keyTo, final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { - return new KeyValueIteratorFacade<>(inner.backwardFetch(from, to, timeFrom, timeTo)); - } - - @Override - public KeyValueIterator, V> backwardFetch(final K from, - final K to, - final long timeFrom, - final long timeTo) throws IllegalArgumentException { - return new KeyValueIteratorFacade<>(inner.backwardFetch(from, to, timeFrom, timeTo)); + return new KeyValueIteratorFacade<>(inner.backwardFetch(keyFrom, keyTo, timeFrom, timeTo)); } @Override @@ -122,12 +107,6 @@ public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, return new KeyValueIteratorFacade<>(inner.backwardFetchAll(timeFrom, timeTo)); } - @Override - public KeyValueIterator, V> backwardFetchAll(final long timeFrom, - final long timeTo) throws IllegalArgumentException { - return new KeyValueIteratorFacade<>(inner.backwardFetchAll(timeFrom, timeTo)); - } - @Override public KeyValueIterator, V> all() { return new KeyValueIteratorFacade<>(inner.all()); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index 3a278e0bc52b7..53f061511481c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -83,20 +83,20 @@ public WindowStoreIterator backwardFetch(final Bytes key, final long tim @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override - public KeyValueIterator, byte[]> fetch(final Bytes from, - final Bytes to, + public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, + final Bytes keyTo, final long timeFrom, final long timeTo) { - final KeyValueIterator bytesIterator = wrapped().fetch(from, to, timeFrom, timeTo); + final KeyValueIterator bytesIterator = wrapped().fetch(keyFrom, keyTo, timeFrom, timeTo); return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); } @Override - public KeyValueIterator, byte[]> backwardFetch(final Bytes from, - final Bytes to, + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, + final Bytes keyTo, final long timeFrom, final long timeTo) { - final KeyValueIterator bytesIterator = wrapped().backwardFetch(from, to, timeFrom, timeTo); + final KeyValueIterator bytesIterator = wrapped().backwardFetch(keyFrom, keyTo, timeFrom, timeTo); return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java index 6da6a6df2a3e0..a54426216c24a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java @@ -151,19 +151,19 @@ public WindowStoreIterator backwardFetch(final Bytes key, @SuppressWarnings("deprecation") @Override - public KeyValueIterator, byte[]> fetch(final Bytes from, - final Bytes to, + public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, + final Bytes keyTo, final long timeFrom, final long timeTo) { - return wrapped.fetch(from, to, timeFrom, timeTo); + return wrapped.fetch(keyFrom, keyTo, timeFrom, timeTo); } @Override - public KeyValueIterator, byte[]> backwardFetch(final Bytes from, - final Bytes to, + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, + final Bytes keyTo, final long timeFrom, final long timeTo) { - return wrapped.backwardFetch(from, to, timeFrom, timeTo); + return wrapped.backwardFetch(keyFrom, keyTo, timeFrom, timeTo); } @SuppressWarnings("deprecation") diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java index 931f565885a4b..da3259974b9ac 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java @@ -70,40 +70,55 @@ public WindowStoreIterator fetch(final Bytes key, @Override public WindowStoreIterator fetch(final Bytes key, final Instant timeFrom, - final Instant timeTo) { + final Instant timeTo) throws IllegalArgumentException { return new WindowToTimestampedWindowIteratorAdapter(store.fetch(key, timeFrom, timeTo)); } + @Override + public WindowStoreIterator backwardFetch(final Bytes key, + final long timeFrom, + final long timeTo) { + return new WindowToTimestampedWindowIteratorAdapter(store.backwardFetch(key, timeFrom, timeTo)); + } + @Override public WindowStoreIterator backwardFetch(final Bytes key, final Instant timeFrom, - final Instant timeTo) { + final Instant timeTo) throws IllegalArgumentException { return new WindowToTimestampedWindowIteratorAdapter(store.backwardFetch(key, timeFrom, timeTo)); } @Override @SuppressWarnings("deprecation") - public KeyValueIterator, byte[]> fetch(final Bytes from, - final Bytes to, + public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, + final Bytes keyTo, final long timeFrom, final long timeTo) { - return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetch(from, to, timeFrom, timeTo)); + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetch(keyFrom, keyTo, timeFrom, timeTo)); } @Override - public KeyValueIterator, byte[]> backwardFetch(final Bytes from, - final Bytes to, + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, + final Bytes keyTo, final Instant timeFrom, - final Instant timeTo) { - return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardFetch(from, to, timeFrom, timeTo)); + final Instant timeTo) throws IllegalArgumentException { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardFetch(keyFrom, keyTo, timeFrom, timeTo)); } @Override - public KeyValueIterator, byte[]> fetch(final Bytes from, - final Bytes to, + public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, + final Bytes keyTo, final Instant timeFrom, - final Instant timeTo) { - return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetch(from, to, timeFrom, timeTo)); + final Instant timeTo) throws IllegalArgumentException { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetch(keyFrom, keyTo, timeFrom, timeTo)); + } + + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, + final Bytes keyTo, + final long timeFrom, + final long timeTo) { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardFetch(keyFrom, keyTo, timeFrom, timeTo)); } @Override @@ -124,15 +139,20 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, } @Override - public KeyValueIterator, byte[]> backwardFetchAll(final Instant timeFrom, - final Instant timeTo) { + public KeyValueIterator, byte[]> fetchAll(final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetchAll(timeFrom, timeTo)); + } + + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final long timeFrom, final long timeTo) { return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardFetchAll(timeFrom, timeTo)); } @Override - public KeyValueIterator, byte[]> fetchAll(final Instant timeFrom, - final Instant timeTo) { - return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetchAll(timeFrom, timeTo)); + public KeyValueIterator, byte[]> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardFetchAll(timeFrom, timeTo)); } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java index 033e1a927c8d9..39e5d03474a01 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java @@ -91,12 +91,7 @@ public WindowStoreIterator fetch(final Object key, final long timeFrom, final lo } @Override - public WindowStoreIterator fetch(final Object key, final Instant timeFrom, final Instant timeTo) { - return EMPTY_WINDOW_STORE_ITERATOR; - } - - @Override - public WindowStoreIterator backwardFetch(final Object key, final long timeFrom, final long timeTo) { + public WindowStoreIterator fetch(final Object key, final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { return EMPTY_WINDOW_STORE_ITERATOR; } @@ -109,32 +104,24 @@ public WindowStoreIterator backwardFetch(final Object key, @Override @SuppressWarnings("deprecation") - public WindowStoreIterator fetch(final Object from, - final Object to, + public WindowStoreIterator fetch(final Object keyFrom, + final Object keyTo, final long timeFrom, final long timeTo) { return EMPTY_WINDOW_STORE_ITERATOR; } @Override - public KeyValueIterator fetch(final Object from, - final Object to, + public KeyValueIterator fetch(final Object keyFrom, + final Object keyTo, final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { return EMPTY_WINDOW_STORE_ITERATOR; } - @Override - public WindowStoreIterator backwardFetch(final Object from, - final Object to, - final long timeFrom, - final long timeTo) { - return EMPTY_WINDOW_STORE_ITERATOR; - } - @Override public KeyValueIterator backwardFetch(final Object from, - final Object to, + final Object keyTo, final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { return EMPTY_WINDOW_STORE_ITERATOR; @@ -157,12 +144,7 @@ public WindowStoreIterator fetchAll(final long timeFrom, final long ti } @Override - public KeyValueIterator fetchAll(final Instant timeFrom, final Instant timeTo) { - return EMPTY_WINDOW_STORE_ITERATOR; - } - - @Override - public WindowStoreIterator backwardFetchAll(final long timeFrom, final long timeTo) { + public KeyValueIterator fetchAll(final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { return EMPTY_WINDOW_STORE_ITERATOR; } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java index 3261ce8d56334..a2c6b7a81daee 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java @@ -50,7 +50,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Properties; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java index f724047498fe6..55f91f1137544 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java @@ -86,12 +86,14 @@ public WindowStoreIterator fetch(final K key, final Instant timeFrom, final I } @Override - public WindowStoreIterator backwardFetch(final K key, final long timeFrom, final long timeTo) throws IllegalArgumentException { + public WindowStoreIterator backwardFetch(final K key, final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { + final long timeFromTs = ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")); + final long timeToTs = ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo")); if (!open) { throw new InvalidStateStoreException("Store is not open"); } final List> results = new ArrayList<>(); - for (long now = timeTo; now >= timeFrom; now--) { + for (long now = timeToTs; now >= timeFromTs; now--) { final Map kvMap = data.get(now); if (kvMap != null && kvMap.containsKey(key)) { results.add(new KeyValue<>(now, kvMap.get(key))); @@ -100,16 +102,6 @@ public WindowStoreIterator backwardFetch(final K key, final long timeFrom, fi return new TheWindowStoreIterator<>(results.iterator()); } - @Override - public WindowStoreIterator backwardFetch(final K key, - final Instant timeFrom, - final Instant timeTo) throws IllegalArgumentException { - return backwardFetch( - key, - ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), - ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); - } - @Override public KeyValueIterator, V> all() { if (!open) { @@ -239,13 +231,15 @@ public KeyValueIterator, V> fetchAll(final Instant timeFrom, final I } @Override - public KeyValueIterator, V> backwardFetchAll(final long timeFrom, final long timeTo) throws IllegalArgumentException { + public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { + final long timeFromTs = ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")); + final long timeToTs = ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo")); if (!open) { throw new InvalidStateStoreException("Store is not open"); } final List, V>> results = new ArrayList<>(); for (final long now : data.descendingKeySet()) { - if (!(now >= timeFrom && now <= timeTo)) { + if (!(now >= timeFromTs && now <= timeToTs)) { continue; } final NavigableMap kvMap = data.get(now); @@ -280,16 +274,9 @@ public KeyValue, V> next() { }; } - @Override - public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, final Instant timeTo) { - return backwardFetchAll( - ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), - ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); - } - @SuppressWarnings("deprecation") @Override - public KeyValueIterator, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { + public KeyValueIterator, V> fetch(final K keyFrom, final K keyTo, final long timeFrom, final long timeTo) { if (!open) { throw new InvalidStateStoreException("Store is not open"); } @@ -297,7 +284,7 @@ public KeyValueIterator, V> fetch(final K from, final K to, final lo for (long now = timeFrom; now <= timeTo; now++) { final NavigableMap kvMap = data.get(now); if (kvMap != null) { - for (final Entry entry : kvMap.subMap(from, true, to, true).entrySet()) { + for (final Entry entry : kvMap.subMap(keyFrom, true, keyTo, true).entrySet()) { results.add(new KeyValue<>(new Windowed<>(entry.getKey(), new TimeWindow(now, now + windowSize)), entry.getValue())); } } @@ -328,13 +315,13 @@ public KeyValue, V> next() { } @Override - public KeyValueIterator, V> fetch(final K from, - final K to, + public KeyValueIterator, V> fetch(final K keyFrom, + final K keyTo, final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { return fetch( - from, - to, + keyFrom, + keyTo, ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "fromTime")), ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "toTime"))); } @@ -342,13 +329,15 @@ public KeyValueIterator, V> fetch(final K from, @Override public KeyValueIterator, V> backwardFetch(final K from, final K to, - final long timeFrom, - final long timeTo) throws IllegalArgumentException { + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + final long timeFromTs = ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")); + final long timeToTs = ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo")); if (!open) { throw new InvalidStateStoreException("Store is not open"); } final List, V>> results = new ArrayList<>(); - for (long now = timeTo; now >= timeFrom; now--) { + for (long now = timeToTs; now >= timeFromTs; now--) { final NavigableMap kvMap = data.get(now); if (kvMap != null) { for (final Entry entry : kvMap.subMap(from, true, to, true).descendingMap().entrySet()) { @@ -381,18 +370,6 @@ public KeyValue, V> next() { }; } - @Override - public KeyValueIterator, V> backwardFetch(final K from, - final K to, - final Instant timeFrom, - final Instant timeTo) { - return backwardFetch( - from, - to, - ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "fromTime")), - ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "toTime"))); - } - public void put(final K key, final V value, final long timestamp) { if (!data.containsKey(timestamp)) { data.put(timestamp, new TreeMap<>()); diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/internals/WindowStoreFacade.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/internals/WindowStoreFacade.java index e49584f7026a9..d7ebc0f13d126 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/internals/WindowStoreFacade.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/internals/WindowStoreFacade.java @@ -17,13 +17,18 @@ package org.apache.kafka.streams.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.TimestampedWindowStore; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.internals.ReadOnlyWindowStoreFacade; +import java.time.Instant; + public class WindowStoreFacade extends ReadOnlyWindowStoreFacade implements WindowStore { public WindowStoreFacade(final TimestampedWindowStore store) { @@ -50,6 +55,21 @@ public void put(final K key, inner.put(key, ValueAndTimestamp.make(value, ConsumerRecord.NO_TIMESTAMP), windowStartTimestamp); } + @Override + public WindowStoreIterator backwardFetch(K key, long timeFrom, long timeTo) { + return backwardFetch(key, Instant.ofEpochMilli(timeFrom), Instant.ofEpochMilli(timeTo)); + } + + @Override + public KeyValueIterator, V> backwardFetch(K keyFrom, K keyTo, long timeFrom, long timeTo) { + return backwardFetch(keyFrom, keyTo, Instant.ofEpochMilli(timeFrom), Instant.ofEpochMilli(timeTo)); + } + + @Override + public KeyValueIterator, V> backwardFetchAll(long timeFrom, long timeTo) { + return backwardFetchAll(Instant.ofEpochMilli(timeFrom), Instant.ofEpochMilli(timeTo)); + } + @Override public void flush() { inner.flush(); From 58ce1d2d1eea3757548cd8975fe3213804d16b21 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 2 Sep 2020 15:02:39 +0100 Subject: [PATCH 26/29] fix check style --- .../kafka/streams/internals/WindowStoreFacade.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/internals/WindowStoreFacade.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/internals/WindowStoreFacade.java index d7ebc0f13d126..f81e2324fd1d0 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/internals/WindowStoreFacade.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/internals/WindowStoreFacade.java @@ -56,12 +56,17 @@ public void put(final K key, } @Override - public WindowStoreIterator backwardFetch(K key, long timeFrom, long timeTo) { + public WindowStoreIterator backwardFetch(final K key, + final long timeFrom, + final long timeTo) { return backwardFetch(key, Instant.ofEpochMilli(timeFrom), Instant.ofEpochMilli(timeTo)); } @Override - public KeyValueIterator, V> backwardFetch(K keyFrom, K keyTo, long timeFrom, long timeTo) { + public KeyValueIterator, V> backwardFetch(final K keyFrom, + final K keyTo, + final long timeFrom, + final long timeTo) { return backwardFetch(keyFrom, keyTo, Instant.ofEpochMilli(timeFrom), Instant.ofEpochMilli(timeTo)); } From f66f247a7e76907caebb44c33d1e562bd707f51a Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 2 Sep 2020 15:16:17 +0100 Subject: [PATCH 27/29] fix check style --- .../org/apache/kafka/streams/internals/WindowStoreFacade.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/internals/WindowStoreFacade.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/internals/WindowStoreFacade.java index f81e2324fd1d0..69450599c81ad 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/internals/WindowStoreFacade.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/internals/WindowStoreFacade.java @@ -71,7 +71,7 @@ public KeyValueIterator, V> backwardFetch(final K keyFrom, } @Override - public KeyValueIterator, V> backwardFetchAll(long timeFrom, long timeTo) { + public KeyValueIterator, V> backwardFetchAll(final long timeFrom, final long timeTo) { return backwardFetchAll(Instant.ofEpochMilli(timeFrom), Instant.ofEpochMilli(timeTo)); } From 1ccaa57d3e3e7953619c35933c257c416ecca01b Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 2 Sep 2020 15:40:06 +0100 Subject: [PATCH 28/29] range and all already available on kvstore --- .../org/apache/kafka/streams/state/internals/Segment.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java index c86ee9643425c..9fddc163b5a55 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import java.io.IOException; @@ -26,7 +25,4 @@ public interface Segment extends KeyValueStore, BatchWritingStore void destroy() throws IOException; - KeyValueIterator all(); - - KeyValueIterator range(final Bytes from, final Bytes to); } From 273f612bfe98c58c40e77ec8a2d77e5eabdd2f18 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 2 Sep 2020 16:24:38 +0100 Subject: [PATCH 29/29] add default impl --- .../kafka/streams/state/WindowStore.java | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java index 7da9ce0172b79..ef1f799b22317 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java @@ -113,9 +113,11 @@ default WindowStoreIterator fetch(final K key, ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } - WindowStoreIterator backwardFetch(final K key, - final long timeFrom, - final long timeTo); + default WindowStoreIterator backwardFetch(final K key, + final long timeFrom, + final long timeTo) { + throw new UnsupportedOperationException(); + } @Override default WindowStoreIterator backwardFetch(final K key, @@ -156,10 +158,12 @@ default KeyValueIterator, V> fetch(final K keyFrom, ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } - KeyValueIterator, V> backwardFetch(final K keyFrom, - final K keyTo, - final long timeFrom, - final long timeTo); + default KeyValueIterator, V> backwardFetch(final K keyFrom, + final K keyTo, + final long timeFrom, + final long timeTo) { + throw new UnsupportedOperationException(); + } @Override default KeyValueIterator, V> backwardFetch(final K keyFrom, @@ -192,7 +196,9 @@ default KeyValueIterator, V> fetchAll(final Instant timeFrom, final ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } - KeyValueIterator, V> backwardFetchAll(final long timeFrom, final long timeTo); + default KeyValueIterator, V> backwardFetchAll(final long timeFrom, final long timeTo) { + throw new UnsupportedOperationException(); + } @Override default KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException {