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..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 @@ -16,8 +16,6 @@ */ package org.apache.kafka.streams.processor.internals; -import java.util.List; - import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; @@ -32,6 +30,8 @@ import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.internals.WrappedStateStore; +import java.util.List; + abstract class AbstractReadOnlyDecorator extends WrappedStateStore { static final String ERROR_MESSAGE = "Global store is read only"; @@ -180,13 +180,28 @@ 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); + } + @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 keyFrom, + final K keyTo, + final long timeFrom, + final long timeTo) { + return wrapped().backwardFetch(keyFrom, keyTo, timeFrom, timeTo); } @Override @@ -194,12 +209,23 @@ 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 long timeFrom, + final long timeTo) { + return wrapped().backwardFetchAll(timeFrom, timeTo); + } } static class TimestampedWindowStoreReadOnlyDecorator 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..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 @@ -173,13 +173,28 @@ 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, - 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 keyFrom, + final K keyTo, + final long timeFrom, + final long timeTo) { + return wrapped().backwardFetch(keyFrom, keyTo, timeFrom, timeTo); } @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @@ -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 0c43d39e3b7da..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 @@ -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) - * @return an iterator over key-value pairs {@code } + * @param key the key to fetch + * @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 NullPointerException if {@code null} is used for key. * @deprecated Use {@link #fetch(Object, Instant, Instant)} instead */ @Deprecated @@ -104,66 +104,134 @@ 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) - * @return an iterator over key-value pairs {@code } + * @param key the key to fetch + * @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} + * @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 + * 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 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} + */ + 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. *

* 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) - * @return an iterator over windowed key-value pairs {@code , value>} + * @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. * @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 - 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 fromTime time range start (inclusive) - * @param toTime time range end (inclusive) - * @return an iterator over windowed key-value pairs {@code , value>} + * @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. * @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) + KeyValueIterator, V> fetch(K keyFrom, K keyTo, Instant timeFrom, Instant timeTo) 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 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. + * @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> backwardFetch(K keyFrom, K keyTo, Instant timeFrom, Instant timeTo) + throws IllegalArgumentException; + + + /** + * Gets all the key-value pairs in the existing windows. + * + * @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(); - + + /** + * 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 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() { + throw new UnsupportedOperationException(); + } + /** * 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 + * @throws NullPointerException if {@code null} is used for any key * @deprecated Use {@link #fetchAll(Instant, Instant)} instead */ @Deprecated @@ -172,12 +240,25 @@ KeyValueIterator, V> fetch(K from, K to, Instant fromTime, Instant t /** * 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 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 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 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} + * @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> 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 f8c93e2c31dca..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 @@ -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 @@ -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,24 +92,41 @@ 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 default WindowStoreIterator fetch(final K key, - final Instant from, - final Instant to) { + 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, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); + } + + default WindowStoreIterator backwardFetch(final K key, + final long timeFrom, + final long timeTo) { + throw new UnsupportedOperationException(); + } + + @Override + default 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"))); } /** @@ -119,27 +134,47 @@ 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 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>} * @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 - KeyValueIterator, V> fetch(K from, K to, long timeFrom, long timeTo); + // note, this method must be kept if super#fetch(...) is removed + @SuppressWarnings("deprecation") + KeyValueIterator, V> fetch(K keyFrom, K keyTo, long timeFrom, long timeTo); @Override - default KeyValueIterator, V> fetch(final K from, - final K to, - final Instant fromTime, - final Instant toTime) { + default KeyValueIterator, V> fetch(final K keyFrom, + final K keyTo, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { return fetch( - from, - to, - ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")), - ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime"))); + keyFrom, + keyTo, + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "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, + final K keyTo, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return backwardFetch( + keyFrom, + keyTo, + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); } /** @@ -150,13 +185,25 @@ 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 - default KeyValueIterator, V> fetchAll(final Instant from, final Instant to) { + default 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, "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) 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/AbstractMergedSortedCacheStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java index a5e56aa0d9ac0..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 @@ -192,4 +192,3 @@ public void close() { storeIterator.close(); } } - 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..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, @@ -63,7 +66,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, true); + } + + @Override + public KeyValueIterator backwardFetch(final Bytes key, + final long from, + final long to) { + return fetch(key, from, to, false); + } + + KeyValueIterator fetch(final Bytes key, + final long from, + final long to, + 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); @@ -72,7 +89,8 @@ public KeyValueIterator fetch(final Bytes key, searchSpace.iterator(), keySchema.hasNextCondition(key, key, from, to), binaryFrom, - binaryTo); + binaryTo, + forward); } @Override @@ -80,6 +98,22 @@ public KeyValueIterator fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) { + return fetch(keyFrom, keyTo, from, to, true); + } + + @Override + public KeyValueIterator backwardFetch(final Bytes keyFrom, + final Bytes keyTo, + final long from, + final long to) { + return fetch(keyFrom, keyTo, from, to, false); + } + + KeyValueIterator fetch(final Bytes keyFrom, + final Bytes keyTo, + final long from, + final long to, + 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, " + @@ -88,7 +122,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, forward); final Bytes binaryFrom = keySchema.lowerRange(keyFrom, from); final Bytes binaryTo = keySchema.upperRange(keyTo, to); @@ -97,30 +131,58 @@ public KeyValueIterator fetch(final Bytes keyFrom, searchSpace.iterator(), keySchema.hasNextCondition(keyFrom, keyTo, from, to), binaryFrom, - binaryTo); + binaryTo, + forward); } @Override public KeyValueIterator all() { - final List searchSpace = segments.allSegments(); + 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 backwardAll() { + 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 fetchAll(final long timeFrom, final long timeTo) { - final List searchSpace = segments.segments(timeFrom, timeTo); + final List searchSpace = segments.segments(timeFrom, timeTo, true); return new SegmentIterator<>( searchSpace.iterator(), keySchema.hasNextCondition(null, null, timeFrom, timeTo), null, - null); + null, + true); + } + + @Override + public KeyValueIterator backwardFetchAll(final long timeFrom, + final long timeTo) { + final List searchSpace = segments.segments(timeFrom, timeTo, false); + + return new SegmentIterator<>( + searchSpace.iterator(), + keySchema.hasNextCondition(null, null, timeFrom, timeTo), + null, + null, + false); } @Override @@ -211,7 +273,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..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 @@ -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,12 +118,20 @@ 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 forward) { final List result = new ArrayList<>(); - final NavigableMap segmentsInRange = segments.subMap( - segmentId(timeFrom), true, - segmentId(timeTo), true - ); + final NavigableMap segmentsInRange; + if (forward) { + segmentsInRange = segments.subMap( + segmentId(timeFrom), true, + segmentId(timeTo), true + ); + } else { + segmentsInRange = segments.subMap( + segmentId(timeFrom), true, + segmentId(timeTo), true + ).descendingMap(); + } for (final S segment : segmentsInRange.values()) { if (segment.isOpen()) { result.add(segment); @@ -132,9 +141,15 @@ public List segments(final long timeFrom, final long timeTo) { } @Override - public List allSegments() { + public List allSegments(final boolean forward) { final List result = new ArrayList<>(); - for (final S segment : segments.values()) { + final Collection values; + if (forward) { + values = segments.values(); + } else { + values = segments.descendingMap().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..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 @@ -78,7 +78,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()); @@ -153,7 +152,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,27 +200,55 @@ 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, true) : + 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); - final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator( - cacheIterator, hasNextCondition, cacheFunction - ); + final PeekingKeyValueIterator filteredCacheIterator = + new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); + + return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator, true); + } + + @Override + public synchronized WindowStoreIterator backwardFetch(final Bytes key, + 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, timeFrom, timeTo); + if (context.cache() == null) { + return underlyingIterator; + } - return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator); + final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? + new CacheIteratorWrapper(key, timeFrom, timeTo, false) : + 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, false); } @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. " + @@ -234,27 +261,74 @@ 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) : - context.cache().range(cacheName, - cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), - cacheFunction.cacheKey(keySchema.upperRange(to, timeTo)) + new CacheIteratorWrapper(keyFrom, keyTo, timeFrom, timeTo, true) : + context.cache().range( + cacheName, + cacheFunction.cacheKey(keySchema.lowerRange(keyFrom, timeFrom)), + cacheFunction.cacheKey(keySchema.upperRange(keyTo, timeTo)) ); - final HasNextCondition hasNextCondition = keySchema.hasNextCondition(from, to, timeFrom, timeTo); - final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); + final HasNextCondition hasNextCondition = keySchema.hasNextCondition(keyFrom, keyTo, timeFrom, timeTo); + final PeekingKeyValueIterator filteredCacheIterator = + new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); return new MergedSortedCacheWindowStoreKeyValueIterator( filteredCacheIterator, underlyingIterator, bytesSerdes, windowSize, - cacheFunction + cacheFunction, + true + ); + } + + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, + final Bytes keyTo, + final long timeFrom, + final long timeTo) { + 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"); + return KeyValueIterators.emptyIterator(); + } + + // 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(keyFrom, keyTo, timeFrom, timeTo); + if (context.cache() == null) { + return underlyingIterator; + } + + final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? + new CacheIteratorWrapper(keyFrom, keyTo, timeFrom, timeTo, false) : + context.cache().reverseRange( + cacheName, + cacheFunction.cacheKey(keySchema.lowerRange(keyFrom, timeFrom)), + cacheFunction.cacheKey(keySchema.upperRange(keyTo, timeTo)) + ); + + final HasNextCondition hasNextCondition = keySchema.hasNextCondition(keyFrom, keyTo, timeFrom, timeTo); + final PeekingKeyValueIterator filteredCacheIterator = + new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); + + return new MergedSortedCacheWindowStoreKeyValueIterator( + filteredCacheIterator, + underlyingIterator, + bytesSerdes, + windowSize, + cacheFunction, + false ); } @@ -271,11 +345,34 @@ 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, + true + ); + } + + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final long timeFrom, + final long timeTo) { + validateStoreOpen(); + + 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); + final PeekingKeyValueIterator filteredCacheIterator = + new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); + + return new MergedSortedCacheWindowStoreKeyValueIterator( + filteredCacheIterator, + underlyingIterator, + bytesSerdes, + windowSize, + cacheFunction, + false ); } @@ -283,7 +380,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 +388,25 @@ public KeyValueIterator, byte[]> all() { underlyingIterator, bytesSerdes, windowSize, - cacheFunction + cacheFunction, + true + ); + } + + @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, + false ); } @@ -315,21 +430,20 @@ 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; - private final Bytes keyFrom; private final Bytes keyTo; private final long timeTo; - private long lastSegmentId; + private final boolean forward; + private long lastSegmentId; private long currentSegmentId; private Bytes cacheKeyFrom; private Bytes cacheKeyTo; @@ -338,25 +452,36 @@ private class CacheIteratorWrapper implements PeekingKeyValueIterator lastSegmentId) { + current = null; + return; + } - if (currentSegmentId > lastSegmentId) { - current = null; - return; - } + setCacheKeyRange(currentSegmentBeginTime(), currentSegmentLastTime()); - setCacheKeyRange(currentSegmentBeginTime(), currentSegmentLastTime()); + current.close(); - current.close(); - current = context.cache().range(cacheName, cacheKeyFrom, cacheKeyTo); + current = context.cache().range(cacheName, cacheKeyFrom, cacheKeyTo); + } else { + --currentSegmentId; + + // last segment id is stable when iterating backward, therefore no need to update + if (currentSegmentId < lastSegmentId) { + current = null; + return; + } + + setCacheKeyRange(currentSegmentBeginTime(), currentSegmentLastTime()); + + current.close(); + + current = context.cache().reverseRange(cacheName, cacheKeyFrom, cacheKeyTo); + } } private void setCacheKeyRange(final long lowerRangeEndTime, final long upperRangeEndTime) { @@ -435,7 +578,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/ChangeLoggingWindowBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java index 0ba6cad0a16d8..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 @@ -69,13 +69,28 @@ public WindowStoreIterator fetch(final Bytes key, return wrapped().fetch(key, from, to); } + @Override + public WindowStoreIterator backwardFetch(final Bytes 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, 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 long timeFrom, + final long timeTo) { + return wrapped().backwardFetch(keyFrom, keyTo, timeFrom, timeTo); } @Override @@ -83,6 +98,12 @@ 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 +111,12 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, return wrapped().fetchAll(timeFrom, timeTo); } + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final long timeFrom, + final long 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..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 @@ -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,8 @@ 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(); @@ -94,24 +94,47 @@ public WindowStoreIterator fetch(final K key, @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 { + 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 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) { + 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(); } @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<>( @@ -120,15 +143,31 @@ 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 { + public KeyValueIterator, V> fetch(final K keyFrom, + final K keyTo, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { return fetch( - from, - to, - ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")), - ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime"))); + keyFrom, + keyTo, + ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), + ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); + } + + @Override + 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(keyFrom, keyTo, timeFrom, timeTo); + return new DelegatingPeekingKeyValueIterator<>( + storeName, + new CompositeKeyValueIterator<>( + provider.stores(storeName, windowStoreType).iterator(), + nextIteratorFunction)); } @Override @@ -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, @@ -157,10 +207,22 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, @SuppressWarnings("deprecation") // removing fetchAll(long from, long to) will fix this @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 timeFrom, + final Instant timeTo) throws IllegalArgumentException { + final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = + store -> store.backwardFetchAll(timeFrom, timeTo); + return new DelegatingPeekingKeyValueIterator<>( + storeName, + new CompositeKeyValueIterator<>( + provider.stores(storeName, windowStoreType).iterator(), + nextIteratorFunction)); } } 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..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 @@ -54,11 +54,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; @@ -66,6 +61,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, @@ -101,9 +101,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; } @@ -143,7 +142,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(); @@ -163,7 +161,15 @@ 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, true); + } + @Override + public WindowStoreIterator backwardFetch(final Bytes key, final long timeFrom, final long timeTo) { + return fetch(key, timeFrom, timeTo, false); + } + + WindowStoreIterator fetch(final Bytes key, final long timeFrom, final long timeTo, final boolean forward) { Objects.requireNonNull(key, "key cannot be null"); removeExpiredSegments(); @@ -175,16 +181,45 @@ public WindowStoreIterator fetch(final Bytes key, final long timeFrom, f return WrappedInMemoryWindowStoreIterator.emptyIterator(); } - return registerNewWindowStoreIterator( - key, segmentMap.subMap(minTime, true, timeTo, true).entrySet().iterator()); + if (forward) { + return registerNewWindowStoreIterator( + key, + segmentMap.subMap(minTime, true, timeTo, true) + .entrySet().iterator(), + true + ); + } else { + return registerNewWindowStoreIterator( + key, + segmentMap.subMap(minTime, true, timeTo, true) + .descendingMap().entrySet().iterator(), + false + ); + } } @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(keyFrom, keyTo, timeFrom, timeTo, true); + } + + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, + final Bytes keyTo, + final long timeFrom, + final long timeTo) { + return fetch(keyFrom, keyTo, timeFrom, timeTo, false); + } + + KeyValueIterator, byte[]> fetch(final Bytes from, + final Bytes to, + final long timeFrom, + final long timeTo, + final boolean forward) { Objects.requireNonNull(from, "from key cannot be null"); Objects.requireNonNull(to, "to key cannot be null"); @@ -205,13 +240,37 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, return KeyValueIterators.emptyIterator(); } - return registerNewWindowedKeyValueIterator( - from, to, segmentMap.subMap(minTime, true, timeTo, true).entrySet().iterator()); + if (forward) { + return registerNewWindowedKeyValueIterator( + from, + to, + segmentMap.subMap(minTime, true, timeTo, true) + .entrySet().iterator(), + true + ); + } else { + return registerNewWindowedKeyValueIterator( + from, + to, + segmentMap.subMap(minTime, true, timeTo, true) + .descendingMap().entrySet().iterator(), + false + ); + } } @Deprecated @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { + return fetchAll(timeFrom, timeTo, true); + } + + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final long timeFrom, final long timeTo) { + return fetchAll(timeFrom, timeTo, false); + } + + KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo, final boolean forward) { removeExpiredSegments(); // add one b/c records expire exactly retentionPeriod ms after created @@ -221,8 +280,23 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, f return KeyValueIterators.emptyIterator(); } - return registerNewWindowedKeyValueIterator( - null, null, segmentMap.subMap(minTime, true, timeTo, true).entrySet().iterator()); + if (forward) { + return registerNewWindowedKeyValueIterator( + null, + null, + segmentMap.subMap(minTime, true, timeTo, true) + .entrySet().iterator(), + true + ); + } else { + return registerNewWindowedKeyValueIterator( + null, + null, + segmentMap.subMap(minTime, true, timeTo, true) + .descendingMap().entrySet().iterator(), + false + ); + } } @Override @@ -232,7 +306,25 @@ 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 + public KeyValueIterator, byte[]> backwardAll() { + removeExpiredSegments(); + + final long minTime = observedStreamTime - retentionPeriod; + + return registerNewWindowedKeyValueIterator( + null, + null, + segmentMap.tailMap(minTime, false).descendingMap().entrySet().iterator(), + false + ); } @Override @@ -286,18 +378,19 @@ 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); } 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; @@ -305,17 +398,19 @@ 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; final WrappedWindowedKeyValueIterator iterator = new WrappedWindowedKeyValueIterator(from, - to, - segmentIterator, - openIterators::remove, - retainDuplicates, - windowSize); + to, + segmentIterator, + openIterators::remove, + retainDuplicates, + windowSize, + forward); openIterators.add(iterator); return iterator; } @@ -327,7 +422,6 @@ interface ClosingCallback { private static abstract class InMemoryWindowStoreIteratorWrapper { - private Iterator>> segmentIterator; private Iterator> recordIterator; private KeyValue next; private long currentTime; @@ -335,18 +429,22 @@ private static abstract class InMemoryWindowStoreIteratorWrapper { private final boolean allKeys; private final Bytes keyFrom; private final Bytes keyTo; - private final boolean retainDuplicates; + private final Iterator>> segmentIterator; 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; @@ -420,14 +518,15 @@ 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) { - super(keyFrom, keyTo, segmentIterator, callback, retainDuplicates); + final boolean retainDuplicates, + final boolean forward) { + super(keyFrom, keyTo, segmentIterator, callback, retainDuplicates, forward); } @Override @@ -450,11 +549,14 @@ public KeyValue next() { } public static WrappedInMemoryWindowStoreIterator emptyIterator() { - return new WrappedInMemoryWindowStoreIterator(null, null, null, it -> { }, false); + return new WrappedInMemoryWindowStoreIterator(null, null, null, it -> { + }, false, true); } } - private static class WrappedWindowedKeyValueIterator extends InMemoryWindowStoreIteratorWrapper implements KeyValueIterator, byte[]> { + private static class WrappedWindowedKeyValueIterator + extends InMemoryWindowStoreIteratorWrapper + implements KeyValueIterator, byte[]> { private final long windowSize; @@ -463,8 +565,9 @@ private static class WrappedWindowedKeyValueIterator extends InMemoryWindowStore 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/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..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 @@ -169,14 +169,40 @@ public WindowStoreIterator fetch(final K key, ); } + @Override + public WindowStoreIterator backwardFetch(final K key, + final long timeFrom, + final long 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, - 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, + time); + } + + @Override + public KeyValueIterator, V> backwardFetch(final K keyFrom, + final K keyTo, + final long timeFrom, + final long timeTo) { + return new MeteredWindowedKeyValueIterator<>( + wrapped().backwardFetch(keyBytes(keyFrom), keyBytes(keyTo), timeFrom, timeTo), fetchSensor, streamsMetrics, serdes, @@ -195,11 +221,27 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, time); } + @Override + public KeyValueIterator, V> backwardFetchAll(final long timeFrom, + final long 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..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 @@ -51,26 +51,41 @@ 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 @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 keyFrom, + final K keyTo, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return new KeyValueIteratorFacade<>(inner.fetch(keyFrom, keyTo, timeFrom, timeTo)); } @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)); + public KeyValueIterator, V> backwardFetch(final K keyFrom, + final K keyTo, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return new KeyValueIteratorFacade<>(inner.backwardFetch(keyFrom, keyTo, timeFrom, timeTo)); } @Override @@ -81,16 +96,25 @@ 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> all() { - final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.all(); - return new KeyValueIteratorFacade<>(innerIterator); + return new KeyValueIteratorFacade<>(inner.all()); + } + + @Override + public KeyValueIterator, V> backwardAll() { + return new KeyValueIteratorFacade<>(inner.backwardAll()); } private static class WindowStoreIteratorFacade implements WindowStoreIterator { 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() { } } } - } 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..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 @@ -65,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 @@ -76,13 +75,28 @@ public WindowStoreIterator fetch(final Bytes key, final long timeFrom, f return new WindowStoreIteratorWrapper(bytesIterator, windowSize).valuesIterator(); } + @Override + 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(); + } + @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 keyFrom, + final Bytes keyTo, + final long timeFrom, + final long timeTo) { + final KeyValueIterator bytesIterator = wrapped().backwardFetch(keyFrom, keyTo, timeFrom, timeTo); return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); } @@ -92,6 +106,12 @@ public KeyValueIterator, byte[]> 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 +119,12 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, f return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); } + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final long timeFrom, final long timeTo) { + 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/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); } 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..03b66a634f2db 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 forward) { this.segments = segments; this.hasNextCondition = hasNextCondition; this.from = from; this.to = to; + this.forward = forward; } @Override @@ -67,14 +70,22 @@ 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 (forward) { + currentIterator = currentSegment.all(); + } else { + currentIterator = currentSegment.reverseAll(); + } } else { - currentIterator = currentSegment.range(from, to); + if (forward) { + currentIterator = currentSegment.range(from, to); + } else { + currentIterator = currentSegment.reverseRange(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..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 @@ -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 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 bcfb02ebaf37f..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); + List segments(final long timeFrom, final long timeTo, final boolean forward); - List allSegments(); + 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 0c80da4f80880..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 @@ -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 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/TimestampedWindowStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java index 43189395bcc62..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 @@ -29,14 +29,15 @@ 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.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 +61,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,13 +142,28 @@ public WindowStoreIterator fetch(final Bytes key, return wrapped.fetch(key, timeFrom, timeTo); } + @Override + public WindowStoreIterator backwardFetch(final Bytes key, + final long timeFrom, + final long timeTo) { + return wrapped.backwardFetch(key, timeFrom, timeTo); + } + @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 keyFrom, + final Bytes keyTo, + final long timeFrom, + final long timeTo) { + return wrapped.backwardFetch(keyFrom, keyTo, timeFrom, timeTo); } @SuppressWarnings("deprecation") @@ -157,11 +173,22 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, return wrapped.fetchAll(timeFrom, timeTo); } + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final long timeFrom, + final long 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 +198,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..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 @@ -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 forward) { + return segments.segments(from, to, forward); } /** 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..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 @@ -69,26 +69,56 @@ 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) 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) 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 keyFrom, + final Bytes keyTo, + final Instant timeFrom, + 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, - final Instant fromTime, - final Instant toTime) { - return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetch(from, to, fromTime, toTime)); + public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, + final Bytes keyTo, + final Instant timeFrom, + 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 @@ -96,6 +126,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, @@ -104,9 +139,20 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, } @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) 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[]> 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/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/NoOpWindowStore.java b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java index 0e8a7b9b30475..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,21 +91,39 @@ 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) throws IllegalArgumentException { + 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 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, - final Instant fromTime, - final Instant toTime) throws IllegalArgumentException { + public KeyValueIterator fetch(final Object keyFrom, + final Object keyTo, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { + return EMPTY_WINDOW_STORE_ITERATOR; + } + + @Override + public KeyValueIterator backwardFetch(final Object from, + final Object keyTo, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { return EMPTY_WINDOW_STORE_ITERATOR; } @@ -113,7 +131,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 +144,13 @@ 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) throws IllegalArgumentException { + 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 61f317d495416..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 @@ -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,9 +340,9 @@ 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 @@ -349,9 +350,9 @@ public void shouldNotThrowNullPointerExceptionOnPutNullValue() { 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/AbstractWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java index a40621697c33a..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 @@ -59,6 +59,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; @@ -68,6 +69,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 { @@ -270,8 +272,26 @@ public void shouldGetAll() { final KeyValue, String> five = windowedPair(5, "five", startTime + 5); assertEquals( - new HashSet<>(asList(zero, one, two, four, five)), - toSet(windowStore.all()) + asList(zero, one, two, four, five), + toList(windowStore.all()) + ); + } + + @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); + + assertEquals( + asList(five, four, two, one, zero), + toList(windowStore.backwardAll()) ); } @@ -288,16 +308,42 @@ public void shouldFetchAllInTimeRange() { 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))) + asList(one, two, four), + toList(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))) ); assertEquals( - new HashSet<>(asList(zero, one, two)), - toSet(windowStore.fetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))) + asList(zero, one, two), + toList(windowStore.fetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))) ); assertEquals( - new HashSet<>(asList(one, two, four, five)), - toSet(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))) + asList(one, two, four, five), + toList(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))) + ); + } + + @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); + + assertEquals( + asList(four, two, one), + toList(windowStore.backwardFetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))) + ); + assertEquals( + asList(two, one, zero), + toList(windowStore.backwardFetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))) + ); + assertEquals( + asList(five, four, two, one), + toList(windowStore.backwardFetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))) ); } @@ -314,64 +360,142 @@ public void testFetchRange() { final KeyValue, String> five = windowedPair(5, "five", startTime + 5); assertEquals( - new HashSet<>(asList(zero, one)), - toSet(windowStore.fetch( + asList(zero, one), + toList(windowStore.fetch( 0, 1, ofEpochMilli(startTime + 0L - WINDOW_SIZE), ofEpochMilli(startTime + 0L + WINDOW_SIZE))) ); assertEquals( - new HashSet<>(Collections.singletonList(one)), - toSet(windowStore.fetch( + Collections.singletonList(one), + toList(windowStore.fetch( 1, 1, ofEpochMilli(startTime + 0L - WINDOW_SIZE), ofEpochMilli(startTime + 0L + WINDOW_SIZE))) ); assertEquals( - new HashSet<>(asList(one, two)), - toSet(windowStore.fetch( + asList(one, two), + toList(windowStore.fetch( 1, 3, ofEpochMilli(startTime + 0L - WINDOW_SIZE), ofEpochMilli(startTime + 0L + WINDOW_SIZE))) ); assertEquals( - new HashSet<>(asList(zero, one, two)), - toSet(windowStore.fetch( + asList(zero, one, two), + toList(windowStore.fetch( 0, 5, ofEpochMilli(startTime + 0L - WINDOW_SIZE), ofEpochMilli(startTime + 0L + WINDOW_SIZE))) ); assertEquals( - new HashSet<>(asList(zero, one, two, four, five)), - toSet(windowStore.fetch( + asList(zero, one, two, four, five), + toList(windowStore.fetch( 0, 5, ofEpochMilli(startTime + 0L - WINDOW_SIZE), ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))) ); assertEquals( - new HashSet<>(asList(two, four, five)), - toSet(windowStore.fetch( + asList(two, four, five), + toList(windowStore.fetch( 0, 5, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))) ); assertEquals( - new HashSet<>(Collections.emptyList()), - toSet(windowStore.fetch( + Collections.emptyList(), + toList(windowStore.fetch( 4, 5, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + WINDOW_SIZE))) ); assertEquals( - new HashSet<>(Collections.emptyList()), - toSet(windowStore.fetch( + Collections.emptyList(), + toList(windowStore.fetch( + 0, + 3, + ofEpochMilli(startTime + 3L), + ofEpochMilli(startTime + WINDOW_SIZE + 5))) + ); + } + + @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); + + assertEquals( + asList(one, zero), + toList(windowStore.backwardFetch( + 0, + 1, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE))) + ); + assertEquals( + Collections.singletonList(one), + toList(windowStore.backwardFetch( + 1, + 1, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE))) + ); + assertEquals( + asList(two, one), + toList(windowStore.backwardFetch( + 1, + 3, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE))) + ); + assertEquals( + asList(two, one, zero), + toList(windowStore.backwardFetch( + 0, + 5, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE))) + ); + assertEquals( + asList(five, four, two, one, zero), + toList(windowStore.backwardFetch( + 0, + 5, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))) + ); + assertEquals( + asList(five, four, two), + toList(windowStore.backwardFetch( + 0, + 5, + ofEpochMilli(startTime + 2L), + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))) + ); + assertEquals( + Collections.emptyList(), + toList(windowStore.backwardFetch( + 4, + 5, + ofEpochMilli(startTime + 2L), + ofEpochMilli(startTime + WINDOW_SIZE))) + ); + assertEquals( + Collections.emptyList(), + toList(windowStore.backwardFetch( 0, 3, ofEpochMilli(startTime + 3L), @@ -730,25 +854,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 49a3a95a910b9..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()); @@ -294,6 +291,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 +357,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 +518,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 +542,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 +573,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() { @@ -495,23 +601,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 @@ -532,6 +638,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 +692,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,10 +744,28 @@ public void shouldReturnSameResultsForSingleKeyFetchAndEqualKeyRangeFetch() { assertFalse(keyRangeIterator.hasNext()); } - @Test(expected = NullPointerException.class) + @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 @SuppressWarnings("deprecation") public void shouldThrowNullPointerExceptionOnPutNullKey() { - cachingStore.put(null, bytesValue("anyValue")); + assertThrows(NullPointerException.class, () -> cachingStore.put(null, bytesValue("anyValue"))); } @Test @@ -596,19 +774,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 @@ -631,6 +809,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(CachingWindowStore.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..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 @@ -47,7 +47,6 @@ public class ChangeLoggingWindowBytesStoreTest { private ProcessorContextImpl context; private ChangeLoggingWindowBytesStore store; - @Before public void setUp() { store = new ChangeLoggingWindowBytesStore(inner, false); @@ -95,6 +94,18 @@ public void shouldDelegateToUnderlyingStoreWhenFetching() { EasyMock.verify(inner); } + @Test + public void shouldDelegateToUnderlyingStoreWhenBackwardFetching() { + EasyMock + .expect(inner.backwardFetch(bytesKey, 0, 10)) + .andReturn(KeyValueIterators.emptyWindowStoreIterator()); + + init(); + + store.backwardFetch(bytesKey, ofEpochMilli(0), ofEpochMilli(10)); + EasyMock.verify(inner); + } + @Test public void shouldDelegateToUnderlyingStoreWhenFetchingRange() { EasyMock @@ -107,6 +118,18 @@ public void shouldDelegateToUnderlyingStoreWhenFetchingRange() { EasyMock.verify(inner); } + @Test + public void shouldDelegateToUnderlyingStoreWhenBackwardFetchingRange() { + EasyMock + .expect(inner.backwardFetch(bytesKey, bytesKey, 0, 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..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 @@ -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,15 +158,26 @@ 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(expected = InvalidStateStoreException.class) + @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); + } + + @Test 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<>( @@ -132,27 +185,78 @@ public void shouldThrowInvalidStateStoreExceptionOnRebalance() { QueryableStoreTypes.windowStore(), "foo" ); - store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); + + assertThrows(InvalidStateStoreException.class, () -> store.fetch("key", ofEpochMilli(1), ofEpochMilli(10))); + } + + @Test + 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" + ); + assertThrows(InvalidStateStoreException.class, () -> 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 +268,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 +301,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 +317,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 +344,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,46 +376,73 @@ 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) + @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..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 @@ -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,20 @@ 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(); + try (final DelegatingPeekingKeyValueIterator peekingIterator = + new DelegatingPeekingKeyValueIterator<>(name, store.all())) { + assertThrows(NoSuchElementException.class, peekingIterator::next); + } } - @Test(expected = NoSuchElementException.class) + @Test public void shouldThrowNoSuchElementWhenNoMoreItemsLeftAndPeekNextCalled() { - final DelegatingPeekingKeyValueIterator peekingIterator = new DelegatingPeekingKeyValueIterator<>(name, store.all()); - peekingIterator.peekNextKey(); - 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/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/KeyValueSegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java index ff67b78478c9d..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,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, true); 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, false); + 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, true); 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, false); + 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, 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 77d97badc780f..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 @@ -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, true ); 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, false + ); + 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, true ); assertThat(iterator.peekNextKey(), equalTo(0L)); iterator.next(); @@ -103,18 +143,72 @@ 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, false + ); + 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())); 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); + 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)); 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, + false + ); + 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..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,58 +58,108 @@ 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() { 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, 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()); } + @Test + public void shouldReverseIterateBothStoreAndCache() { + 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()); + } + private KeyValue, String> convertKeyValuePair(final KeyValue, byte[]> next) { final String value = deserializer.deserialize("", next.value); return KeyValue.pair(convertWindowedKey(next.key), value); @@ -121,10 +171,9 @@ private Windowed convertWindowedKey(final Windowed bytesWindowed) } - private MergedSortedCacheWindowStoreKeyValueIterator createIterator( - final Iterator, byte[]>> storeKvs, - final Iterator> cacheKvs - ) { + private MergedSortedCacheWindowStoreKeyValueIterator createIterator(final Iterator, byte[]>> storeKvs, + final Iterator> cacheKvs, + final boolean forward) { final DelegatingPeekingKeyValueIterator, byte[]> storeIterator = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(storeKvs)); @@ -135,7 +184,8 @@ private MergedSortedCacheWindowStoreKeyValueIterator createIterator( storeIterator, new StateSerdes<>("name", Serdes.Bytes(), Serdes.ByteArray()), WINDOW_SIZE, - SINGLE_SEGMENT_CACHE_FUNCTION + SINGLE_SEGMENT_CACHE_FUNCTION, + forward ); } } 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..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 @@ -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,8 +44,8 @@ public class ReadOnlyWindowStoreStub implements ReadOnlyWindowStore, StateStore { private final long windowSize; - private final Map> data = new HashMap<>(); - private boolean open = true; + private final NavigableMap> data = new TreeMap<>(); + private boolean open = true; ReadOnlyWindowStoreStub(final long windowSize) { this.windowSize = windowSize; @@ -79,11 +78,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 { + 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"))); + 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) 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 = timeToTs; now >= timeFromTs; 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 @@ -104,7 +120,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.descendingKeySet()) { + 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 +202,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() { @@ -167,15 +224,59 @@ 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 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 >= timeFromTs && now <= timeToTs)) { + 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) { + 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"); } @@ -183,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())); } } @@ -192,7 +293,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,15 +314,60 @@ 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 keyFrom, + final K keyTo, + final Instant timeFrom, + final Instant timeTo) throws IllegalArgumentException { return fetch( - from, - to, - ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")), - ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime"))); + keyFrom, + keyTo, + 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 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 = 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()) { + 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) { @@ -236,13 +383,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() { @@ -258,7 +408,7 @@ void setOpen(final boolean open) { this.open = open; } - private class TheWindowStoreIterator implements WindowStoreIterator { + private static class TheWindowStoreIterator implements WindowStoreIterator { private final Iterator> underlying; @@ -267,7 +417,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..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 @@ -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 { @@ -54,14 +55,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 +87,8 @@ public void shouldIterateOverAllSegments() { Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("a".getBytes()), - Bytes.wrap("z".getBytes())); + Bytes.wrap("z".getBytes()), + true); assertTrue(iterator.hasNext()); assertEquals("a", new String(iterator.peekNextKey().get())); @@ -107,26 +109,76 @@ public void shouldIterateOverAllSegments() { assertFalse(iterator.hasNext()); } + @Test + public void shouldIterateBackwardOverAllSegments() { + iterator = new SegmentIterator<>( + 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("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())); + + 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 shouldNotThrowExceptionOnHasNextWhenStoreClosed() { iterator = new SegmentIterator<>( Collections.singletonList(segmentOne).iterator(), hasNextCondition, Bytes.wrap("a".getBytes()), - Bytes.wrap("z".getBytes())); + Bytes.wrap("z".getBytes()), + true); 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()), + 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())); + + 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()), + true); assertTrue(iterator.hasNext()); assertEquals("a", new String(iterator.peekNextKey().get())); @@ -139,26 +191,28 @@ public void shouldOnlyIterateOverSegmentsInRange() { assertFalse(iterator.hasNext()); } - @Test(expected = NoSuchElementException.class) + @Test public void shouldThrowNoSuchElementOnPeekNextKeyIfNoNext() { iterator = new SegmentIterator<>( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("f".getBytes()), - Bytes.wrap("h".getBytes())); + Bytes.wrap("h".getBytes()), + true); - iterator.peekNextKey(); + assertThrows(NoSuchElementException.class, () -> iterator.peekNextKey()); } - @Test(expected = NoSuchElementException.class) + @Test public void shouldThrowNoSuchElementOnNextIfNoNext() { iterator = new SegmentIterator<>( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("f".getBytes()), - Bytes.wrap("h".getBytes())); + Bytes.wrap("h".getBytes()), + true); - iterator.next(); + assertThrows(NoSuchElementException.class, () -> iterator.next()); } private KeyValue toStringKeyValue(final KeyValue binaryKv) { 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/TimestampedSegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java index 2ae671769909b..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,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, true); 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, false); + 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, true); 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, false); + 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, 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/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 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..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; @@ -63,8 +64,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", @@ -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()); 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..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 @@ -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,26 @@ public void put(final K key, inner.put(key, ValueAndTimestamp.make(value, ConsumerRecord.NO_TIMESTAMP), windowStartTimestamp); } + @Override + 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(final K keyFrom, + final K keyTo, + final long timeFrom, + final long timeTo) { + return backwardFetch(keyFrom, keyTo, Instant.ofEpochMilli(timeFrom), Instant.ofEpochMilli(timeTo)); + } + + @Override + public KeyValueIterator, V> backwardFetchAll(final long timeFrom, final long timeTo) { + return backwardFetchAll(Instant.ofEpochMilli(timeFrom), Instant.ofEpochMilli(timeTo)); + } + @Override public void flush() { inner.flush();