From fa884429aa4b81402ced7ca4a0df7f1b044103d1 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 1 Jul 2020 22:57:44 +0100 Subject: [PATCH 01/24] key value reverse api and implementations --- .../streams/state/ReadOnlyKeyValueStore.java | 8 +++ .../state/internals/CachingKeyValueStore.java | 27 +++++++++- .../ChangeLoggingKeyValueBytesStore.java | 11 ++++ .../CompositeReadOnlyKeyValueStore.java | 34 +++++++++++++ .../internals/InMemoryKeyValueStore.java | 20 +++++++- .../state/internals/MemoryLRUCache.java | 16 ++++++ .../internals/MemoryNavigableLRUCache.java | 22 +++++++- .../state/internals/MeteredKeyValueStore.java | 15 ++++++ .../streams/state/internals/NamedCache.java | 8 +++ .../ReadOnlyKeyValueStoreFacade.java | 10 ++++ .../internals/RocksDBPrefixIterator.java | 2 +- .../state/internals/RocksDBRangeIterator.java | 15 ++++-- .../streams/state/internals/RocksDBStore.java | 43 ++++++++++++---- .../internals/RocksDBTimestampedStore.java | 51 ++++++++++++------- .../state/internals/RocksDbIterator.java | 8 ++- .../streams/state/internals/ThreadCache.java | 16 ++++++ .../TimestampedKeyValueStoreBuilder.java | 11 ++++ 17 files changed, 281 insertions(+), 36 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java index 9b2f8f5fc6667..ddd01bceb66ee 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java @@ -55,6 +55,10 @@ public interface ReadOnlyKeyValueStore { */ KeyValueIterator range(K from, K to); + default KeyValueIterator reverseRange(K from, K to) { + throw new UnsupportedOperationException(); + } + /** * Return an iterator over all keys in this store. This iterator must be closed after use. * The returned iterator must be safe from {@link java.util.ConcurrentModificationException}s @@ -64,6 +68,10 @@ public interface ReadOnlyKeyValueStore { */ KeyValueIterator all(); + default KeyValueIterator reverseAll() { + throw new UnsupportedOperationException(); + } + /** * Return an approximate count of key-value mappings in this store. * diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java index 3502fadfa29fb..15536d7acaa6d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java @@ -252,6 +252,22 @@ public KeyValueIterator range(final Bytes from, return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator); } + @Override + public KeyValueIterator reverseRange(final Bytes from, + final Bytes to) { + if (from.compareTo(to) > 0) { + LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + + "Note that the built-in numerical serdes do not follow this for negative numbers"); + return KeyValueIterators.emptyIterator(); + } + + validateStoreOpen(); + final KeyValueIterator storeIterator = wrapped().reverseRange(from, to); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = context.cache().reverseRange(cacheName, from, to); + return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator); + } + @Override public KeyValueIterator all() { validateStoreOpen(); @@ -261,6 +277,15 @@ public KeyValueIterator all() { return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator); } + @Override + public KeyValueIterator reverseAll() { + validateStoreOpen(); + final KeyValueIterator storeIterator = + new DelegatingPeekingKeyValueIterator<>(this.name(), wrapped().reverseAll()); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = context.cache().reverseAll(cacheName); + return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator); + } + @Override public long approximateNumEntries() { validateStoreOpen(); @@ -297,7 +322,7 @@ public void close() { ); if (!suppressed.isEmpty()) { throwSuppressed("Caught an exception while closing caching key value store for store " + name(), - suppressed); + suppressed); } } finally { lock.writeLock().unlock(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java index 35f6d365891ab..236f21877ef2d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java @@ -100,11 +100,22 @@ public KeyValueIterator range(final Bytes from, return wrapped().range(from, to); } + @Override + public KeyValueIterator reverseRange(final Bytes from, + final Bytes to) { + return wrapped().reverseRange(from, to); + } + @Override public KeyValueIterator all() { return wrapped().all(); } + @Override + public KeyValueIterator reverseAll() { + return wrapped().reverseAll(); + } + void log(final Bytes key, final byte[] value) { context.logChange(name(), key, value, context.timestamp()); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java index c790b89b13f65..4b515e04d8e38 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java @@ -82,6 +82,24 @@ public KeyValueIterator apply(final ReadOnlyKeyValueStore store) { return new DelegatingPeekingKeyValueIterator<>(storeName, new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction)); } + @Override + public KeyValueIterator reverseRange(final K from, final K to) { + Objects.requireNonNull(from); + Objects.requireNonNull(to); + final NextIteratorFunction> nextIteratorFunction = new NextIteratorFunction>() { + @Override + public KeyValueIterator apply(final ReadOnlyKeyValueStore store) { + try { + return store.reverseRange(from, to); + } 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."); + } + } + }; + final List> stores = storeProvider.stores(storeName, storeType); + return new DelegatingPeekingKeyValueIterator<>(storeName, new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction)); + } + @Override public KeyValueIterator all() { final NextIteratorFunction> nextIteratorFunction = new NextIteratorFunction>() { @@ -98,6 +116,22 @@ public KeyValueIterator apply(final ReadOnlyKeyValueStore store) { return new DelegatingPeekingKeyValueIterator<>(storeName, new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction)); } + @Override + public KeyValueIterator reverseAll() { + final NextIteratorFunction> nextIteratorFunction = new NextIteratorFunction>() { + @Override + public KeyValueIterator apply(final ReadOnlyKeyValueStore store) { + try { + return store.reverseAll(); + } 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."); + } + } + }; + final List> stores = storeProvider.stores(storeName, storeType); + return new DelegatingPeekingKeyValueIterator<>(storeName, new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction)); + } + @Override public long approximateNumEntries() { final List> stores = storeProvider.stores(storeName, storeType); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java index 27ec409dc4d31..1a1ad389c9c75 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java @@ -110,7 +110,15 @@ public synchronized byte[] delete(final Bytes key) { @Override public synchronized KeyValueIterator range(final Bytes from, final Bytes to) { + return range(from, to, false); + } + + @Override + public KeyValueIterator reverseRange(Bytes from, Bytes to) { + return range(from, to, true); + } + KeyValueIterator range(final Bytes from, final Bytes to, final boolean reverse) { if (from.compareTo(to) > 0) { LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + @@ -118,7 +126,10 @@ public synchronized KeyValueIterator range(final Bytes from, fina return KeyValueIterators.emptyIterator(); } - return new DelegatingPeekingKeyValueIterator<>( + if (reverse) return new DelegatingPeekingKeyValueIterator<>( + name, + new InMemoryKeyValueIterator(map.subMap(from, true, to, true).descendingKeySet())); + else return new DelegatingPeekingKeyValueIterator<>( name, new InMemoryKeyValueIterator(map.subMap(from, true, to, true).keySet())); } @@ -130,6 +141,13 @@ public synchronized KeyValueIterator all() { new InMemoryKeyValueIterator(map.keySet())); } + @Override + public KeyValueIterator reverseAll() { + return new DelegatingPeekingKeyValueIterator<>( + name, + new InMemoryKeyValueIterator(map.descendingKeySet())); + } + @Override public long approximateNumEntries() { return size; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java index d69df13f4f096..32a91cd671299 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java @@ -142,6 +142,14 @@ public KeyValueIterator range(final Bytes from, final Bytes to) { throw new UnsupportedOperationException("MemoryLRUCache does not support range() function."); } + /** + * @throws UnsupportedOperationException at every invocation + */ + @Override + public KeyValueIterator reverseRange(final Bytes from, final Bytes to) { + throw new UnsupportedOperationException("MemoryLRUCache does not support reverseRange() function."); + } + /** * @throws UnsupportedOperationException at every invocation */ @@ -150,6 +158,14 @@ public KeyValueIterator all() { throw new UnsupportedOperationException("MemoryLRUCache does not support all() function."); } + /** + * @throws UnsupportedOperationException at every invocation + */ + @Override + public KeyValueIterator reverseAll() { + throw new UnsupportedOperationException("MemoryLRUCache does not support reverseAll() function."); + } + @Override public long approximateNumEntries() { return this.map.size(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java index 6e0deaa667eb9..7ce44206d9979 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java @@ -36,7 +36,6 @@ public MemoryNavigableLRUCache(final String name, final int maxCacheSize) { @Override public KeyValueIterator range(final Bytes from, final Bytes to) { - if (from.compareTo(to) > 0) { LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + @@ -50,12 +49,33 @@ public KeyValueIterator range(final Bytes from, final Bytes to) { .subSet(from, true, to, true).iterator(), treeMap)); } + @Override + public KeyValueIterator reverseRange(final Bytes from, final Bytes to) { + if (from.compareTo(to) > 0) { + LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + + "Note that the built-in numerical serdes do not follow this for negative numbers"); + return KeyValueIterators.emptyIterator(); + } + + final TreeMap treeMap = toTreeMap(); + return new DelegatingPeekingKeyValueIterator<>(name(), + new MemoryNavigableLRUCache.CacheIterator(treeMap + .subMap(from, true, to, true).descendingKeySet().iterator(), treeMap)); + } + @Override public KeyValueIterator all() { final TreeMap treeMap = toTreeMap(); return new MemoryNavigableLRUCache.CacheIterator(treeMap.navigableKeySet().iterator(), treeMap); } + @Override + public KeyValueIterator reverseAll() { + final TreeMap treeMap = toTreeMap(); + return new MemoryNavigableLRUCache.CacheIterator(treeMap.descendingKeySet().iterator(), treeMap); + } + private synchronized TreeMap toTreeMap() { return new TreeMap<>(this.map); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java index d77834b66e65c..9fe5be998477e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java @@ -42,6 +42,7 @@ * inner KeyValueStore implementation do not need to provide its own metrics collecting functionality. * The inner {@link KeyValueStore} of this class is of type <Bytes,byte[]>, hence we use {@link Serde}s * to convert from <K,V> to <Bytes,byte[]> + * * @param * @param */ @@ -187,11 +188,25 @@ public KeyValueIterator range(final K from, ); } + @Override + public KeyValueIterator reverseRange(final K from, + final K to) { + return new MeteredKeyValueIterator( + wrapped().reverseRange(Bytes.wrap(serdes.rawKey(from)), Bytes.wrap(serdes.rawKey(to))), + rangeSensor + ); + } + @Override public KeyValueIterator all() { return new MeteredKeyValueIterator(wrapped().all(), allSensor); } + @Override + public KeyValueIterator reverseAll() { + return new MeteredKeyValueIterator(wrapped().reverseAll(), allSensor); + } + @Override public void flush() { maybeMeasureLatency(super::flush, time, flushSensor); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java index 4693fbc7b6540..438a438503936 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java @@ -284,6 +284,10 @@ synchronized Iterator keyRange(final Bytes from, final Bytes to) { return keySetIterator(cache.navigableKeySet().subSet(from, true, to, true)); } + synchronized Iterator reverseKeyRange(final Bytes from, final Bytes to) { + return keySetIterator(cache.subMap(from, true, to, true).descendingKeySet()); + } + private Iterator keySetIterator(final Set keySet) { return new TreeSet<>(keySet).iterator(); } @@ -292,6 +296,10 @@ synchronized Iterator allKeys() { return keySetIterator(cache.navigableKeySet()); } + synchronized Iterator reverseAllKeys() { + return keySetIterator(cache.descendingKeySet()); + } + synchronized LRUCacheEntry first() { if (head == null) { return null; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyKeyValueStoreFacade.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyKeyValueStoreFacade.java index 862e6fcfbe89e..04acc58138393 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyKeyValueStoreFacade.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyKeyValueStoreFacade.java @@ -40,11 +40,21 @@ public KeyValueIterator range(final K from, return new KeyValueIteratorFacade<>(inner.range(from, to)); } + @Override + public KeyValueIterator reverseRange(K from, K to) { + return new KeyValueIteratorFacade<>(inner.reverseRange(from, to)); + } + @Override public KeyValueIterator all() { return new KeyValueIteratorFacade<>(inner.all()); } + @Override + public KeyValueIterator reverseAll() { + return new KeyValueIteratorFacade<>(inner.reverseAll()); + } + @Override public long approximateNumEntries() { return inner.approximateNumEntries(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBPrefixIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBPrefixIterator.java index b84175e1ee0c6..303ceead0ffc0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBPrefixIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBPrefixIterator.java @@ -29,7 +29,7 @@ class RocksDBPrefixIterator extends RocksDbIterator { final RocksIterator newIterator, final Set> openIterators, final Bytes prefix) { - super(name, newIterator, openIterators); + super(name, newIterator, openIterators, false); rawPrefix = prefix.get(); newIterator.seek(rawPrefix); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java index b1cf24dcdd1b5..3f27b556487da 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java @@ -35,10 +35,17 @@ class RocksDBRangeIterator extends RocksDbIterator { final RocksIterator iter, final Set> openIterators, final Bytes from, - final Bytes to) { - super(storeName, iter, openIterators); - iter.seek(from.get()); - rawToKey = to.get(); + final Bytes to, + final boolean reverse) { + super(storeName, iter, openIterators, reverse); + if (reverse) { + iter.seek(to.get()); + rawToKey = from.get(); + } + else { + iter.seek(from.get()); + rawToKey = to.get(); + } if (rawToKey == null) { throw new NullPointerException("RocksDBRangeIterator: RawToKey is null for key " + to); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index fc2bb24724f43..8cb73ecd5f39f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -307,6 +307,18 @@ public synchronized byte[] delete(final Bytes key) { @Override public synchronized KeyValueIterator range(final Bytes from, final Bytes to) { + return range(from, to, false); + } + + @Override + public synchronized KeyValueIterator reverseRange(final Bytes from, + final Bytes to) { + return range(from, to, true); + } + + KeyValueIterator range(final Bytes from, + final Bytes to, + final boolean reverse) { Objects.requireNonNull(from, "from cannot be null"); Objects.requireNonNull(to, "to cannot be null"); @@ -319,7 +331,7 @@ public synchronized KeyValueIterator range(final Bytes from, validateStoreOpen(); - final KeyValueIterator rocksDBRangeIterator = dbAccessor.range(from, to); + final KeyValueIterator rocksDBRangeIterator = dbAccessor.range(from, to, reverse); openIterators.add(rocksDBRangeIterator); return rocksDBRangeIterator; @@ -327,8 +339,17 @@ public synchronized KeyValueIterator range(final Bytes from, @Override public synchronized KeyValueIterator all() { + return all(false); + } + + @Override + public KeyValueIterator reverseAll() { + return all(true); + } + + KeyValueIterator all(final boolean reverse) { validateStoreOpen(); - final KeyValueIterator rocksDbIterator = dbAccessor.all(); + final KeyValueIterator rocksDbIterator = dbAccessor.all(reverse); openIterators.add(rocksDbIterator); return rocksDbIterator; } @@ -461,9 +482,10 @@ void prepareBatch(final List> entries, byte[] getOnly(final byte[] key) throws RocksDBException; KeyValueIterator range(final Bytes from, - final Bytes to); + final Bytes to, + final boolean reverse); - KeyValueIterator all(); + KeyValueIterator all(final boolean reverse); long approximateNumEntries() throws RocksDBException; @@ -527,20 +549,23 @@ public byte[] getOnly(final byte[] key) throws RocksDBException { @Override public KeyValueIterator range(final Bytes from, - final Bytes to) { + final Bytes to, + final boolean reverse) { return new RocksDBRangeIterator( name, db.newIterator(columnFamily), openIterators, from, - to); + to, + reverse); } @Override - public KeyValueIterator all() { + public KeyValueIterator all(final boolean reverse) { final RocksIterator innerIterWithTimestamp = db.newIterator(columnFamily); - innerIterWithTimestamp.seekToFirst(); - return new RocksDbIterator(name, innerIterWithTimestamp, openIterators); + if (reverse) innerIterWithTimestamp.seekToLast(); + else innerIterWithTimestamp.seekToFirst(); + return new RocksDbIterator(name, innerIterWithTimestamp, openIterators, reverse); } @Override 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 6c31e9b43d208..311b62ff377a1 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 @@ -193,22 +193,26 @@ public byte[] getOnly(final byte[] key) throws RocksDBException { @Override public KeyValueIterator range(final Bytes from, - final Bytes to) { + final Bytes to, + final boolean reverse) { return new RocksDBDualCFRangeIterator( name, db.newIterator(newColumnFamily), db.newIterator(oldColumnFamily), from, - to); + to, + reverse); } @Override - public KeyValueIterator all() { + public KeyValueIterator all(final boolean reverse) { final RocksIterator innerIterWithTimestamp = db.newIterator(newColumnFamily); - innerIterWithTimestamp.seekToFirst(); + if (reverse) innerIterWithTimestamp.seekToLast(); + else innerIterWithTimestamp.seekToFirst(); final RocksIterator innerIterNoTimestamp = db.newIterator(oldColumnFamily); - innerIterNoTimestamp.seekToFirst(); - return new RocksDBDualCFIterator(name, innerIterWithTimestamp, innerIterNoTimestamp); + if (reverse) innerIterNoTimestamp.seekToLast(); + else innerIterNoTimestamp.seekToFirst(); + return new RocksDBDualCFIterator(name, innerIterWithTimestamp, innerIterNoTimestamp, reverse); } @Override @@ -262,6 +266,7 @@ private class RocksDBDualCFIterator extends AbstractIterator makeNext() { } else { next = KeyValue.pair(new Bytes(nextWithTimestamp), iterWithTimestamp.value()); nextWithTimestamp = null; - iterWithTimestamp.next(); + if (reverse) iterWithTimestamp.prev(); + else iterWithTimestamp.next(); } } else { if (nextWithTimestamp == null) { next = KeyValue.pair(new Bytes(nextNoTimestamp), convertToTimestampedFormat(iterNoTimestamp.value())); nextNoTimestamp = null; - iterNoTimestamp.next(); + if (reverse) iterNoTimestamp.prev(); + else iterNoTimestamp.next(); } else { if (comparator.compare(nextNoTimestamp, nextWithTimestamp) <= 0) { next = KeyValue.pair(new Bytes(nextNoTimestamp), convertToTimestampedFormat(iterNoTimestamp.value())); nextNoTimestamp = null; - iterNoTimestamp.next(); + if (reverse) iterNoTimestamp.prev(); + else iterNoTimestamp.next(); } else { next = KeyValue.pair(new Bytes(nextWithTimestamp), iterWithTimestamp.value()); nextWithTimestamp = null; - iterWithTimestamp.next(); + if (reverse) iterWithTimestamp.prev(); + else iterWithTimestamp.next(); } } } - return next; } @@ -357,11 +367,18 @@ private class RocksDBDualCFRangeIterator extends RocksDBDualCFIterator { final RocksIterator iterWithTimestamp, final RocksIterator iterNoTimestamp, final Bytes from, - final Bytes to) { - super(storeName, iterWithTimestamp, iterNoTimestamp); - iterWithTimestamp.seek(from.get()); - iterNoTimestamp.seek(from.get()); - upperBoundKey = to.get(); + final Bytes to, + final boolean reverse) { + super(storeName, iterWithTimestamp, iterNoTimestamp, reverse); + if (reverse) { + iterWithTimestamp.seek(to.get()); + iterNoTimestamp.seek(to.get()); + upperBoundKey = from.get(); + } else { + iterWithTimestamp.seek(from.get()); + iterNoTimestamp.seek(from.get()); + upperBoundKey = to.get(); + } if (upperBoundKey == null) { throw new NullPointerException("RocksDBDualCFRangeIterator: upperBoundKey is null for key " + to); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbIterator.java index 9fa747a45b9c0..23087baaa5024 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbIterator.java @@ -31,6 +31,7 @@ class RocksDbIterator extends AbstractIterator> implemen private final String storeName; private final RocksIterator iter; private final Set> openIterators; + private final boolean reverse; private volatile boolean open = true; @@ -38,10 +39,12 @@ class RocksDbIterator extends AbstractIterator> implemen RocksDbIterator(final String storeName, final RocksIterator iter, - final Set> openIterators) { + final Set> openIterators, + final boolean reverse) { this.storeName = storeName; this.iter = iter; this.openIterators = openIterators; + this.reverse = reverse; } @Override @@ -58,7 +61,8 @@ public KeyValue makeNext() { return allDone(); } else { next = getKeyValue(); - iter.next(); + if (reverse) iter.prev(); + else iter.next(); return next; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java index 0179536b63a28..0a745c01549ed 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java @@ -185,6 +185,14 @@ public MemoryLRUCacheBytesIterator range(final String namespace, final Bytes fro return new MemoryLRUCacheBytesIterator(cache.keyRange(from, to), cache); } + public MemoryLRUCacheBytesIterator reverseRange(final String namespace, final Bytes from, final Bytes to) { + final NamedCache cache = getCache(namespace); + if (cache == null) { + return new MemoryLRUCacheBytesIterator(Collections.emptyIterator(), new NamedCache(namespace, this.metrics)); + } + return new MemoryLRUCacheBytesIterator(cache.reverseKeyRange(from, to), cache); + } + public MemoryLRUCacheBytesIterator all(final String namespace) { final NamedCache cache = getCache(namespace); if (cache == null) { @@ -193,6 +201,14 @@ public MemoryLRUCacheBytesIterator all(final String namespace) { return new MemoryLRUCacheBytesIterator(cache.allKeys(), cache); } + public MemoryLRUCacheBytesIterator reverseAll(final String namespace) { + final NamedCache cache = getCache(namespace); + if (cache == null) { + return new MemoryLRUCacheBytesIterator(Collections.emptyIterator(), new NamedCache(namespace, this.metrics)); + } + return new MemoryLRUCacheBytesIterator(cache.reverseAllKeys(), cache); + } + public long size() { long size = 0; for (final NamedCache cache : caches.values()) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedKeyValueStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedKeyValueStoreBuilder.java index 863b44ba4f68e..be8f259366841 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedKeyValueStoreBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedKeyValueStoreBuilder.java @@ -133,11 +133,22 @@ public KeyValueIterator range(final Bytes from, return wrapped.range(from, to); } + @Override + public KeyValueIterator reverseRange(final Bytes from, + final Bytes to) { + return wrapped.reverseRange(from, to); + } + @Override public KeyValueIterator all() { return wrapped.all(); } + @Override + public KeyValueIterator reverseAll() { + return wrapped.reverseAll(); + } + @Override public long approximateNumEntries() { return wrapped.approximateNumEntries(); From c0364e3360e9e5d2d7ec882955a1f0c84bcb8ed4 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 2 Jul 2020 00:16:23 +0100 Subject: [PATCH 02/24] backward window api and implementation --- .../streams/state/ReadOnlyWindowStore.java | 19 ++- .../AbstractRocksDBSegmentedBytesStore.java | 58 ++++++- .../state/internals/AbstractSegments.java | 17 ++- .../state/internals/CachingWindowStore.java | 141 ++++++++++++++++-- .../ChangeLoggingWindowBytesStore.java | 29 ++++ .../CompositeReadOnlyWindowStore.java | 70 ++++++++- .../state/internals/InMemoryWindowStore.java | 83 +++++++++-- .../state/internals/MeteredWindowStore.java | 44 ++++++ .../internals/ReadOnlyWindowStoreFacade.java | 28 ++++ .../state/internals/RocksDBWindowStore.java | 35 +++++ .../state/internals/SegmentedBytesStore.java | 14 +- .../streams/state/internals/Segments.java | 4 +- .../state/internals/SessionKeySchema.java | 5 +- .../TimestampedWindowStoreBuilder.java | 29 ++++ .../state/internals/WindowKeySchema.java | 5 +- ...owToTimestampedWindowByteStoreAdapter.java | 26 ++++ .../state/internals/KeyValueSegmentsTest.java | 6 +- .../internals/TimestampedSegmentsTest.java | 6 +- .../kafka/test/SegmentedBytesStoreStub.java | 27 +++- 19 files changed, 590 insertions(+), 56 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java index 0c43d39e3b7da..4f89009eefd74 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 @@ -114,6 +114,10 @@ public interface ReadOnlyWindowStore { */ WindowStoreIterator fetch(K key, Instant from, Instant to) throws IllegalArgumentException; + default WindowStoreIterator backwardFetch(K key, Instant from, Instant to) throws IllegalArgumentException { + throw new UnsupportedOperationException(); + } + /** * Get all the key-value pairs in the given key range and time range from all the existing windows. *

@@ -148,6 +152,11 @@ public interface ReadOnlyWindowStore { KeyValueIterator, V> fetch(K from, K to, Instant fromTime, Instant toTime) throws IllegalArgumentException; + default KeyValueIterator, V> backwardFetch(K from, K to, Instant fromTime, Instant toTime) + throws IllegalArgumentException { + throw new UnsupportedOperationException(); + } + /** * Gets all the key-value pairs in the existing windows. * @@ -155,7 +164,11 @@ KeyValueIterator, V> fetch(K from, K to, Instant fromTime, Instant t * @throws InvalidStateStoreException if the store is not initialized */ KeyValueIterator, V> all(); - + + default KeyValueIterator, V> backwardAll() { + throw new UnsupportedOperationException(); + } + /** * Gets all the key-value pairs that belong to the windows within in the given time range. * @@ -180,4 +193,8 @@ KeyValueIterator, V> fetch(K from, K to, Instant fromTime, Instant t * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ KeyValueIterator, V> fetchAll(Instant from, Instant to) throws IllegalArgumentException; + + default KeyValueIterator, V> backwardFetchAll(Instant from, Instant to) throws IllegalArgumentException { + throw new UnsupportedOperationException(); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java index dc1ae86ab08c8..af136587ea712 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java @@ -63,7 +63,19 @@ public class AbstractRocksDBSegmentedBytesStore implements Se public KeyValueIterator fetch(final Bytes key, final long from, final long to) { - final List searchSpace = keySchema.segmentsToSearch(segments, from, to); + return fetch(key, from, to, false); + } + + @Override + public KeyValueIterator backwardFetch(Bytes key, long from, long to) { + return fetch(key, from, to, true); + } + + KeyValueIterator fetch(final Bytes key, + final long from, + final long to, + final boolean backward) { + final List searchSpace = keySchema.segmentsToSearch(segments, from, to, backward); final Bytes binaryFrom = keySchema.lowerRangeFixedSize(key, from); final Bytes binaryTo = keySchema.upperRangeFixedSize(key, to); @@ -80,6 +92,19 @@ public KeyValueIterator fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) { + return fetch(keyFrom, keyTo, from, to, false); + } + + @Override + public KeyValueIterator backwardFetch(Bytes keyFrom, Bytes keyTo, long from, long to) { + return fetch(keyFrom, keyTo, from, to, true); + } + + KeyValueIterator fetch(final Bytes keyFrom, + final Bytes keyTo, + final long from, + final long to, + final boolean backward) { if (keyFrom.compareTo(keyTo) > 0) { LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + @@ -87,7 +112,7 @@ public KeyValueIterator fetch(final Bytes keyFrom, return KeyValueIterators.emptyIterator(); } - final List searchSpace = keySchema.segmentsToSearch(segments, from, to); + final List searchSpace = keySchema.segmentsToSearch(segments, from, to, backward); final Bytes binaryFrom = keySchema.lowerRange(keyFrom, from); final Bytes binaryTo = keySchema.upperRange(keyTo, to); @@ -101,7 +126,18 @@ public KeyValueIterator fetch(final Bytes keyFrom, @Override public KeyValueIterator all() { - final List searchSpace = segments.allSegments(); + final List searchSpace = segments.allSegments(false); + + return new SegmentIterator<>( + searchSpace.iterator(), + keySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE), + null, + null); + } + + @Override + public KeyValueIterator backwardAll() { + final List searchSpace = segments.allSegments(true); return new SegmentIterator<>( searchSpace.iterator(), @@ -113,7 +149,19 @@ public KeyValueIterator all() { @Override public KeyValueIterator fetchAll(final long timeFrom, final long timeTo) { - final List searchSpace = segments.segments(timeFrom, timeTo); + final List searchSpace = segments.segments(timeFrom, timeTo, false); + + return new SegmentIterator<>( + searchSpace.iterator(), + keySchema.hasNextCondition(null, null, timeFrom, timeTo), + null, + null); + } + + @Override + public KeyValueIterator backwardFetchAll(final long timeFrom, + final long timeTo) { + final List searchSpace = segments.segments(timeFrom, timeTo, true); return new SegmentIterator<>( searchSpace.iterator(), @@ -210,7 +258,7 @@ public boolean isOpen() { // Visible for testing List getSegments() { - return segments.allSegments(); + return segments.allSegments(false); } // Visible for testing diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java index 044a57448502b..666a773374f0d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractSegments.java @@ -27,6 +27,7 @@ import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -117,9 +118,14 @@ public void openExisting(final ProcessorContext context, final long streamTime) } @Override - public List segments(final long timeFrom, final long timeTo) { + public List segments(final long timeFrom, final long timeTo, final boolean backward) { final List result = new ArrayList<>(); - final NavigableMap segmentsInRange = segments.subMap( + final NavigableMap segmentsInRange; + if (backward) segmentsInRange = segments.subMap( + segmentId(timeFrom), true, + segmentId(timeTo), true + ).descendingMap(); + else segmentsInRange = segments.subMap( segmentId(timeFrom), true, segmentId(timeTo), true ); @@ -132,9 +138,12 @@ public List segments(final long timeFrom, final long timeTo) { } @Override - public List allSegments() { + public List allSegments(final boolean backward) { final List result = new ArrayList<>(); - for (final S segment : segments.values()) { + final Collection values; + if (backward) values = segments.descendingMap().values(); + else values = segments.values(); + for (final S segment : values) { if (segment.isOpen()) { result.add(segment); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index ee31cf68f07b4..e0c837dff0a5c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; @@ -33,10 +34,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Instant; import java.util.LinkedList; import java.util.NoSuchElementException; import java.util.concurrent.atomic.AtomicLong; +import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; import static org.apache.kafka.streams.state.internals.ExceptionUtils.executeAll; import static org.apache.kafka.streams.state.internals.ExceptionUtils.throwSuppressed; @@ -154,7 +157,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( @@ -204,8 +207,38 @@ public synchronized WindowStoreIterator fetch(final Bytes key, final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? new CacheIteratorWrapper(key, timeFrom, timeTo) : context.cache().range(name, - cacheFunction.cacheKey(keySchema.lowerRangeFixedSize(key, timeFrom)), - cacheFunction.cacheKey(keySchema.upperRangeFixedSize(key, timeTo)) + 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); + } + + @Override + public synchronized WindowStoreIterator backwardFetch(final Bytes key, + final Instant from, + final Instant to) { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + // since this function may not access the underlying inner store, we need to validate + // if store is open outside as well. + validateStoreOpen(); + + final WindowStoreIterator underlyingIterator = wrapped().backwardFetch(key, from, to); + if (context.cache() == null) { + return underlyingIterator; + } + + final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? + new CacheIteratorWrapper(key, timeFrom, timeTo) : + context.cache().reverseRange(name, + cacheFunction.cacheKey(keySchema.lowerRangeFixedSize(key, timeFrom)), + cacheFunction.cacheKey(keySchema.upperRangeFixedSize(key, timeTo)) ); final HasNextCondition hasNextCondition = keySchema.hasNextCondition(key, key, timeFrom, timeTo); @@ -242,8 +275,52 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? new CacheIteratorWrapper(from, to, timeFrom, timeTo) : context.cache().range(name, - cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), - cacheFunction.cacheKey(keySchema.upperRange(to, timeTo)) + cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), + cacheFunction.cacheKey(keySchema.upperRange(to, timeTo)) + ); + + final HasNextCondition hasNextCondition = keySchema.hasNextCondition(from, to, timeFrom, timeTo); + final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); + + return new MergedSortedCacheWindowStoreKeyValueIterator( + filteredCacheIterator, + underlyingIterator, + bytesSerdes, + windowSize, + cacheFunction + ); + } + + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, + final Bytes to, + final Instant fromTime, + final Instant toTime) { + if (from.compareTo(to) > 0) { + LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + + "Note that the built-in numerical serdes do not follow this for negative numbers"); + return KeyValueIterators.emptyIterator(); + } + + final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")); + final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime")); + + // since this function may not access the underlying inner store, we need to validate + // if store is open outside as well. + validateStoreOpen(); + + final KeyValueIterator, byte[]> underlyingIterator = + wrapped().backwardFetch(from, to, fromTime, toTime); + if (context.cache() == null) { + return underlyingIterator; + } + + final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? + new CacheIteratorWrapper(from, to, timeFrom, timeTo) : + context.cache().reverseRange(name, + cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), + cacheFunction.cacheKey(keySchema.upperRange(to, timeTo)) ); final HasNextCondition hasNextCondition = keySchema.hasNextCondition(from, to, timeFrom, timeTo); @@ -271,11 +348,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 + ); + } + + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, + final Instant to) { + validateStoreOpen(); + + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + + final KeyValueIterator, byte[]> underlyingIterator = wrapped().backwardFetchAll(from, to); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = context.cache().reverseAll(name); + + 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 ); } @@ -283,7 +383,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(name); return new MergedSortedCacheWindowStoreKeyValueIterator( @@ -295,6 +395,22 @@ public KeyValueIterator, byte[]> all() { ); } + @Override + public KeyValueIterator, byte[]> backwardAll() { + validateStoreOpen(); + + final KeyValueIterator, byte[]> underlyingIterator = wrapped().backwardAll(); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = context.cache().reverseAll(name); + + return new MergedSortedCacheWindowStoreKeyValueIterator( + cacheIterator, + underlyingIterator, + bytesSerdes, + windowSize, + cacheFunction + ); + } + @Override public synchronized void flush() { context.cache().flush(name); @@ -310,12 +426,11 @@ public synchronized void close() { ); if (!suppressed.isEmpty()) { throwSuppressed("Caught an exception while closing caching window store for store " + name(), - suppressed); + suppressed); } } - private class CacheIteratorWrapper implements PeekingKeyValueIterator { private final long segmentInterval; 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..374deca9f8841 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java @@ -25,6 +25,8 @@ import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; +import java.time.Instant; + /** * Simple wrapper around a {@link WindowStore} to support writing * updates to a changelog @@ -69,6 +71,13 @@ public WindowStoreIterator fetch(final Bytes key, return wrapped().fetch(key, from, to); } + @Override + public WindowStoreIterator backwardFetch(final Bytes key, + final Instant from, + final Instant to) { + return wrapped().backwardFetch(key, from, to); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, @@ -78,11 +87,25 @@ public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, return wrapped().fetch(keyFrom, keyTo, from, to); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, + final Bytes keyTo, + final Instant from, + final Instant to) { + return wrapped().backwardFetch(keyFrom, keyTo, from, to); + } + @Override public KeyValueIterator, byte[]> all() { return wrapped().all(); } + + @Override + public KeyValueIterator, byte[]> backwardAll() { + return wrapped().backwardAll(); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, @@ -90,6 +113,12 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, return wrapped().fetchAll(timeFrom, timeTo); } + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) { + return wrapped().backwardFetchAll(timeFrom, timeTo); + } + @Deprecated @Override public void put(final Bytes key, final byte[] value) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java index fbfc7a02e9c3f..81368fe9bb8d2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java @@ -60,8 +60,8 @@ public V fetch(final K key, final long time) { } } catch (final InvalidStateStoreException e) { throw new InvalidStateStoreException( - "State store is not available anymore and may have been migrated to another instance; " + - "please re-discover its location from the state metadata."); + "State store is not available anymore and may have been migrated to another instance; " + + "please re-discover its location from the state metadata."); } } return null; @@ -84,8 +84,31 @@ public WindowStoreIterator fetch(final K key, } } catch (final InvalidStateStoreException e) { throw new InvalidStateStoreException( - "State store is not available anymore and may have been migrated to another instance; " + - "please re-discover its location from the state metadata."); + "State store is not available anymore and may have been migrated to another instance; " + + "please re-discover its location from the state metadata."); + } + } + return KeyValueIterators.emptyWindowStoreIterator(); + } + + @Override + public WindowStoreIterator backwardFetch(final K key, + final Instant timeFrom, + final Instant timeTo) { + Objects.requireNonNull(key, "key can't be null"); + final List> stores = provider.stores(storeName, windowStoreType); + for (final ReadOnlyWindowStore windowStore : stores) { + try { + final WindowStoreIterator result = windowStore.backwardFetch(key, timeFrom, timeTo); + if (!result.hasNext()) { + result.close(); + } else { + return result; + } + } catch (final InvalidStateStoreException e) { + throw new InvalidStateStoreException( + "State store is not available anymore and may have been migrated to another instance; " + + "please re-discover its location from the state metadata."); } } return KeyValueIterators.emptyWindowStoreIterator(); @@ -119,6 +142,22 @@ public KeyValueIterator, V> fetch(final K from, nextIteratorFunction)); } + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final Instant timeFrom, + final Instant timeTo) { + Objects.requireNonNull(from, "from can't be null"); + Objects.requireNonNull(to, "to can't be null"); + final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = + store -> store.backwardFetch(from, to, timeFrom, timeTo); + return new DelegatingPeekingKeyValueIterator<>( + storeName, + new CompositeKeyValueIterator<>( + provider.stores(storeName, windowStoreType).iterator(), + nextIteratorFunction)); + } + @Override public KeyValueIterator, V> fetch(final K from, final K to, @@ -142,6 +181,17 @@ public KeyValueIterator, V> all() { nextIteratorFunction)); } + @Override + public KeyValueIterator, V> backwardAll() { + final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = + ReadOnlyWindowStore::backwardAll; + return new DelegatingPeekingKeyValueIterator<>( + storeName, + new CompositeKeyValueIterator<>( + provider.stores(storeName, windowStoreType).iterator(), + nextIteratorFunction)); + } + @Override @Deprecated public KeyValueIterator, V> fetchAll(final long timeFrom, @@ -155,6 +205,18 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, nextIteratorFunction)); } + @Override + public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) { + final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = + store -> store.backwardFetchAll(timeFrom, timeTo); + return new DelegatingPeekingKeyValueIterator<>( + storeName, + new CompositeKeyValueIterator<>( + provider.stores(storeName, windowStoreType).iterator(), + nextIteratorFunction)); + } + @SuppressWarnings("deprecation") // removing fetchAll(long from, long to) will fix this @Override public KeyValueIterator, V> fetchAll(final Instant from, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java index e7220e80178d4..952f8ed5133fe 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.TimeWindow; import org.apache.kafka.streams.processor.ProcessorContext; @@ -34,6 +35,7 @@ import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; +import java.time.Instant; import java.util.Iterator; import java.util.Map; import java.util.NoSuchElementException; @@ -43,6 +45,7 @@ import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; +import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; import static org.apache.kafka.streams.state.internals.WindowKeySchema.extractStoreKeyBytes; import static org.apache.kafka.streams.state.internals.WindowKeySchema.extractStoreTimestamp; @@ -163,7 +166,17 @@ public byte[] fetch(final Bytes key, final long windowStartTimestamp) { @Deprecated @Override public WindowStoreIterator fetch(final Bytes key, final long timeFrom, final long timeTo) { + return fetch(key, timeFrom, timeTo, false); + } + + @Override + public WindowStoreIterator backwardFetch(final Bytes key, final Instant from, final Instant to) { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + return fetch(key, timeFrom, timeTo, true); + } + WindowStoreIterator fetch(final Bytes key, final long timeFrom, final long timeTo, final boolean backward) { Objects.requireNonNull(key, "key cannot be null"); removeExpiredSegments(); @@ -175,7 +188,9 @@ public WindowStoreIterator fetch(final Bytes key, final long timeFrom, f return WrappedInMemoryWindowStoreIterator.emptyIterator(); } - return registerNewWindowStoreIterator( + if (backward) return registerNewWindowStoreIterator( + key, segmentMap.subMap(minTime, true, timeTo, true).descendingMap().entrySet().iterator()); + else return registerNewWindowStoreIterator( key, segmentMap.subMap(minTime, true, timeTo, true).entrySet().iterator()); } @@ -185,6 +200,24 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to, final long timeFrom, final long timeTo) { + return fetch(from, to, timeFrom, timeTo, false); + } + + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, + final Bytes to, + final Instant fromTime, + final Instant toTime) { + final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")); + final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime")); + return fetch(from, to, timeFrom, timeTo, false); + } + + KeyValueIterator, byte[]> fetch(final Bytes from, + final Bytes to, + final long timeFrom, + final long timeTo, + final boolean backward) { Objects.requireNonNull(from, "from key cannot be null"); Objects.requireNonNull(to, "to key cannot be null"); @@ -204,13 +237,26 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, return KeyValueIterators.emptyIterator(); } - return registerNewWindowedKeyValueIterator( + if (backward) return registerNewWindowedKeyValueIterator( + from, to, segmentMap.subMap(minTime, true, timeTo, true).descendingMap().entrySet().iterator()); + else return registerNewWindowedKeyValueIterator( from, to, segmentMap.subMap(minTime, true, timeTo, true).entrySet().iterator()); } @Deprecated @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { + return fetchAll(timeFrom, timeTo, false); + } + + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, final Instant to) { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + return fetchAll(timeFrom, timeTo, false); + } + + KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo, final boolean backward) { removeExpiredSegments(); // add one b/c records expire exactly retentionPeriod ms after created @@ -220,7 +266,9 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, f return KeyValueIterators.emptyIterator(); } - return registerNewWindowedKeyValueIterator( + if (backward) return registerNewWindowedKeyValueIterator( + null, null, segmentMap.subMap(minTime, true, timeTo, true).descendingMap().entrySet().iterator()); + else return registerNewWindowedKeyValueIterator( null, null, segmentMap.subMap(minTime, true, timeTo, true).entrySet().iterator()); } @@ -234,6 +282,16 @@ public KeyValueIterator, byte[]> all() { null, null, segmentMap.tailMap(minTime, false).entrySet().iterator()); } + @Override + public KeyValueIterator, byte[]> backwardAll() { + removeExpiredSegments(); + + final long minTime = observedStreamTime - retentionPeriod; + + return registerNewWindowedKeyValueIterator( + null, null, segmentMap.tailMap(minTime, false).descendingMap().entrySet().iterator()); + } + @Override public boolean persistent() { return false; @@ -285,7 +343,7 @@ private static Bytes wrapForDups(final Bytes key, final int seqnum) { } private static Bytes getKey(final Bytes keyBytes) { - final byte[] bytes = new byte[keyBytes.get().length - SEQNUM_SIZE]; + final byte[] bytes = new byte[keyBytes.get().length - SEQNUM_SIZE]; System.arraycopy(keyBytes.get(), 0, bytes, 0, bytes.length); return Bytes.wrap(bytes); } @@ -310,11 +368,11 @@ private WrappedWindowedKeyValueIterator registerNewWindowedKeyValueIterator(fina final WrappedWindowedKeyValueIterator iterator = new WrappedWindowedKeyValueIterator(from, - to, - segmentIterator, - openIterators::remove, - retainDuplicates, - windowSize); + to, + segmentIterator, + openIterators::remove, + retainDuplicates, + windowSize); openIterators.add(iterator); return iterator; } @@ -419,13 +477,13 @@ Long minTime() { } } - private static class WrappedInMemoryWindowStoreIterator extends InMemoryWindowStoreIteratorWrapper implements WindowStoreIterator { + private static class WrappedInMemoryWindowStoreIterator extends InMemoryWindowStoreIteratorWrapper implements WindowStoreIterator { WrappedInMemoryWindowStoreIterator(final Bytes keyFrom, final Bytes keyTo, final Iterator>> segmentIterator, final ClosingCallback callback, - final boolean retainDuplicates) { + final boolean retainDuplicates) { super(keyFrom, keyTo, segmentIterator, callback, retainDuplicates); } @@ -449,7 +507,8 @@ public KeyValue next() { } public static WrappedInMemoryWindowStoreIterator emptyIterator() { - return new WrappedInMemoryWindowStoreIterator(null, null, null, it -> { }, false); + return new WrappedInMemoryWindowStoreIterator(null, null, null, it -> { + }, false); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index b253f39a4f632..fcd761016d2ee 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -33,6 +33,8 @@ import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics; +import java.time.Instant; + import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; public class MeteredWindowStore @@ -169,6 +171,19 @@ public WindowStoreIterator fetch(final K key, ); } + @Override + public WindowStoreIterator backwardFetch(final K key, + final Instant timeFrom, + final Instant timeTo) { + return new MeteredWindowStoreIterator<>( + wrapped().backwardFetch(keyBytes(key), timeFrom, timeTo), + fetchSensor, + streamsMetrics, + serdes, + time + ); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed @Override public KeyValueIterator, V> fetch(final K from, @@ -183,6 +198,19 @@ public KeyValueIterator, V> fetch(final K from, time); } + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final Instant timeFrom, + final Instant timeTo) { + return new MeteredWindowedKeyValueIterator<>( + wrapped().backwardFetch(keyBytes(from), keyBytes(to), timeFrom, timeTo), + fetchSensor, + streamsMetrics, + serdes, + time); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, V> fetchAll(final long timeFrom, @@ -195,11 +223,27 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, time); } + @Override + public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) { + return new MeteredWindowedKeyValueIterator<>( + wrapped().backwardFetchAll(timeFrom, timeTo), + fetchSensor, + streamsMetrics, + serdes, + time); + } + @Override public KeyValueIterator, V> all() { return new MeteredWindowedKeyValueIterator<>(wrapped().all(), fetchSensor, streamsMetrics, serdes, time); } + @Override + public KeyValueIterator, V> backwardAll() { + return new MeteredWindowedKeyValueIterator<>(wrapped().backwardAll(), fetchSensor, streamsMetrics, serdes, time); + } + @Override public void flush() { maybeMeasureLatency(super::flush, time, flushSensor); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java index 713959db46d37..ff84d4e2da041 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java @@ -56,6 +56,13 @@ public WindowStoreIterator fetch(final K key, return new WindowStoreIteratorFacade<>(inner.fetch(key, from, to)); } + @Override + public WindowStoreIterator backwardFetch(final K key, + final Instant from, + final Instant to) throws IllegalArgumentException { + return new WindowStoreIteratorFacade<>(inner.backwardFetch(key, from, to)); + } + @Override @SuppressWarnings("deprecation") public KeyValueIterator, V> fetch(final K from, @@ -73,6 +80,14 @@ public KeyValueIterator, V> fetch(final K from, return new KeyValueIteratorFacade<>(inner.fetch(from, to, fromTime, toTime)); } + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final Instant fromTime, + final Instant toTime) throws IllegalArgumentException { + return new KeyValueIteratorFacade<>(inner.backwardFetch(from, to, fromTime, toTime)); + } + @Override @SuppressWarnings("deprecation") public KeyValueIterator, V> fetchAll(final long timeFrom, @@ -87,12 +102,25 @@ public KeyValueIterator, V> fetchAll(final Instant from, return new KeyValueIteratorFacade<>(innerIterator); } + @Override + public KeyValueIterator, V> backwardFetchAll(final Instant from, + final Instant to) throws IllegalArgumentException { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.backwardFetchAll(from, to); + return new KeyValueIteratorFacade<>(innerIterator); + } + @Override public KeyValueIterator, V> all() { final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.all(); return new KeyValueIteratorFacade<>(innerIterator); } + @Override + public KeyValueIterator, V> backwardAll() { + final KeyValueIterator, ValueAndTimestamp> innerIterator = inner.backwardAll(); + return new KeyValueIteratorFacade<>(innerIterator); + } + private static class WindowStoreIteratorFacade implements WindowStoreIterator { final KeyValueIterator> innerIterator; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index 010c09215843a..d9eb6b955d5e5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; @@ -24,6 +25,10 @@ import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; +import java.time.Instant; + +import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; + public class RocksDBWindowStore extends WrappedStateStore implements WindowStore { @@ -76,6 +81,14 @@ public WindowStoreIterator fetch(final Bytes key, final long timeFrom, f return new WindowStoreIteratorWrapper(bytesIterator, windowSize).valuesIterator(); } + @Override + public WindowStoreIterator backwardFetch(Bytes key, Instant from, Instant to) throws IllegalArgumentException { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + final KeyValueIterator bytesIterator = wrapped().backwardFetch(key, timeFrom, timeTo); + return new WindowStoreIteratorWrapper(bytesIterator, windowSize).valuesIterator(); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, byte[]> fetch(final Bytes from, @@ -86,12 +99,26 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); } + @Override + public KeyValueIterator, byte[]> backwardFetch(Bytes from, Bytes to, Instant fromTime, Instant toTime) { + final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "to")); + final KeyValueIterator bytesIterator = wrapped().backwardFetch(from, to, timeFrom, timeTo); + return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + } + @Override public KeyValueIterator, byte[]> all() { final KeyValueIterator bytesIterator = wrapped().all(); return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); } + @Override + public KeyValueIterator, byte[]> backwardAll() { + final KeyValueIterator bytesIterator = wrapped().backwardAll(); + return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + } + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { @@ -99,6 +126,14 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, f return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); } + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, final Instant to) { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + final KeyValueIterator bytesIterator = wrapped().backwardFetchAll(timeFrom, timeTo); + return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + } + private void maybeUpdateSeqnumForDups() { if (retainDuplicates) { seqnum = (seqnum + 1) & 0x7FFFFFFF; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java index 068dc5f0dc1bb..f91e8416b4e52 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,8 @@ public interface SegmentedBytesStore extends StateStore { */ KeyValueIterator fetch(final Bytes key, final long from, final long to); + 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 +52,9 @@ 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); - + + 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 +62,9 @@ public interface SegmentedBytesStore extends StateStore { * @throws InvalidStateStoreException if the store is not initialized */ KeyValueIterator all(); - + + KeyValueIterator backwardAll(); + /** * Gets all the key-value pairs that belong to the windows within in the given time range. * @@ -70,6 +76,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 +179,6 @@ interface KeySchema { * @param to * @return List of segments to search */ - List segmentsToSearch(Segments segments, long from, long to); + List segmentsToSearch(Segments segments, long from, long to, boolean backward); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java index bcfb02ebaf37f..38a5d318bcf66 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java @@ -34,9 +34,9 @@ interface Segments { void openExisting(final ProcessorContext context, final long streamTime); - List segments(final long timeFrom, final long timeTo); + List segments(final long timeFrom, final long timeTo, final boolean backward); - List allSegments(); + List allSegments(final boolean backward); void flush(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java index 0c80da4f80880..568297e0d7825 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java @@ -87,8 +87,9 @@ public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom, final Bytes @Override public List segmentsToSearch(final Segments segments, final long from, - final long to) { - return segments.segments(from, Long.MAX_VALUE); + final long to, + final boolean backward) { + return segments.segments(from, Long.MAX_VALUE, backward); } private static K extractKey(final byte[] binaryKey, 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..a27fb4923020b 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 @@ -30,7 +30,9 @@ import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; +import java.time.Instant; import java.util.Objects; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -141,6 +143,13 @@ public WindowStoreIterator fetch(final Bytes key, return wrapped.fetch(key, timeFrom, timeTo); } + @Override + public WindowStoreIterator backwardFetch(final Bytes key, + final Instant timeFrom, + final Instant timeTo) { + return wrapped.backwardFetch(key, timeFrom, timeTo); + } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, byte[]> fetch(final Bytes from, @@ -150,6 +159,14 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, return wrapped.fetch(from, to, timeFrom, timeTo); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, + final Bytes to, + final Instant timeFrom, + final Instant timeTo) { + return wrapped.backwardFetch(from, to, timeFrom, timeTo); + } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, @@ -157,11 +174,22 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, return wrapped.fetchAll(timeFrom, timeTo); } + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) { + return wrapped.backwardFetchAll(timeFrom, timeTo); + } + @Override public KeyValueIterator, byte[]> all() { return wrapped.all(); } + @Override + public KeyValueIterator, byte[]> backwardAll() { + return wrapped.backwardAll(); + } + @Override public void flush() { wrapped.flush(); @@ -171,6 +199,7 @@ public void flush() { public void close() { wrapped.close(); } + @Override public boolean isOpen() { return wrapped.isOpen(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java index 9218ccf0a7752..0437c7a904294 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 @@ -93,8 +93,9 @@ public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom, @Override public List segmentsToSearch(final Segments segments, final long from, - final long to) { - return segments.segments(from, to); + final long to, + final boolean backward) { + return segments.segments(from, to, backward); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java index 7bf8a0c5872d3..c36c4146f8dc2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java @@ -74,6 +74,13 @@ public WindowStoreIterator fetch(final Bytes key, return new WindowToTimestampedWindowIteratorAdapter(store.fetch(key, from, to)); } + @Override + public WindowStoreIterator backwardFetch(final Bytes key, + final Instant from, + final Instant to) { + return new WindowToTimestampedWindowIteratorAdapter(store.backwardFetch(key, from, to)); + } + @Override @SuppressWarnings("deprecation") public KeyValueIterator, byte[]> fetch(final Bytes from, @@ -83,6 +90,14 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetch(from, to, timeFrom, timeTo)); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, + final Bytes to, + final Instant fromTime, + final Instant toTime) { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardFetch(from, to, fromTime, toTime)); + } + @Override public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to, @@ -96,6 +111,11 @@ public KeyValueIterator, byte[]> all() { return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.all()); } + @Override + public KeyValueIterator, byte[]> backwardAll() { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardAll()); + } + @Override @SuppressWarnings("deprecation") public KeyValueIterator, byte[]> fetchAll(final long timeFrom, @@ -103,6 +123,12 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.fetchAll(timeFrom, timeTo)); } + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, + final Instant to) { + return new KeyValueToTimestampedKeyValueIteratorAdapter<>(store.backwardFetchAll(from, to)); + } + @Override public KeyValueIterator, byte[]> fetchAll(final Instant from, final Instant to) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java index ff67b78478c9d..b3d612b345fa0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java @@ -187,7 +187,7 @@ public void shouldGetSegmentsWithinTimeRange() { segments.getOrCreateSegmentIfLive(3, context, streamTime); segments.getOrCreateSegmentIfLive(4, context, streamTime); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); @@ -202,7 +202,7 @@ public void shouldGetSegmentsWithinTimeRangeOutOfOrder() { updateStreamTimeAndCreateSegment(1); updateStreamTimeAndCreateSegment(3); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); @@ -309,7 +309,7 @@ public void shouldClearSegmentsOnClose() { } private void verifyCorrectSegments(final long first, final int numSegments) { - final List result = this.segments.segments(0, Long.MAX_VALUE); + final List result = this.segments.segments(0, Long.MAX_VALUE, false); assertEquals(numSegments, result.size()); for (int i = 0; i < numSegments; i++) { assertEquals(i + first, result.get(i).id); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java index 2ae671769909b..e7f32ab9cc35d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java @@ -188,7 +188,7 @@ public void shouldGetSegmentsWithinTimeRange() { segments.getOrCreateSegmentIfLive(3, context, streamTime); segments.getOrCreateSegmentIfLive(4, context, streamTime); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); @@ -203,7 +203,7 @@ public void shouldGetSegmentsWithinTimeRangeOutOfOrder() { updateStreamTimeAndCreateSegment(1); updateStreamTimeAndCreateSegment(3); - final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL); + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, false); assertEquals(3, segments.size()); assertEquals(0, segments.get(0).id); assertEquals(1, segments.get(1).id); @@ -310,7 +310,7 @@ public void shouldClearSegmentsOnClose() { } private void verifyCorrectSegments(final long first, final int numSegments) { - final List result = this.segments.segments(0, Long.MAX_VALUE); + final List result = this.segments.segments(0, Long.MAX_VALUE, false); assertEquals(numSegments, result.size()); for (int i = 0; i < numSegments; i++) { assertEquals(i + first, result.get(i).id); diff --git a/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java b/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java index 17c5e6814d018..71c090e6eafd4 100644 --- a/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java @@ -53,24 +53,47 @@ public KeyValueIterator fetch(final Bytes key, final long from, f return fetch(key, key, from, to); } + @Override + public KeyValueIterator backwardFetch(Bytes key, long from, long to) { + return backwardFetch(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 backwardFetch(Bytes keyFrom, Bytes keyTo, long from, long to) { + fetchCalled = true; + return new KeyValueIteratorStub<>(Collections.>emptyIterator()); + } + @Override public KeyValueIterator all() { fetchCalled = true; return new KeyValueIteratorStub<>(Collections.>emptyIterator()); } - + + @Override + public KeyValueIterator backwardAll() { + 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 KeyValueIterator backwardFetchAll(long from, long to) { + fetchCalled = true; + return new KeyValueIteratorStub<>(Collections.>emptyIterator()); + } + @Override public void remove(final Bytes key) { store.put(key, null); From 52398a966133cea0f2dab60c94bc7ab391efaecb Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 2 Jul 2020 17:44:17 +0100 Subject: [PATCH 03/24] backward session windows api and impls --- .../internals/AbstractReadOnlyDecorator.java | 27 ++++++ .../streams/state/ReadOnlySessionStore.java | 8 ++ .../kafka/streams/state/SessionStore.java | 8 ++ .../state/internals/CachingSessionStore.java | 91 ++++++++++++++++--- .../ChangeLoggingSessionBytesStore.java | 20 ++++ .../state/internals/InMemorySessionStore.java | 74 +++++++++++++-- .../state/internals/MeteredSessionStore.java | 62 +++++++++++++ .../state/internals/RocksDBSessionStore.java | 36 ++++++++ 8 files changed, 307 insertions(+), 19 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java index a63cd992cbe63..fb6ae593c078b 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 @@ -17,6 +17,7 @@ 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; @@ -214,6 +215,13 @@ public KeyValueIterator, AGG> findSessions(final K key, return wrapped().findSessions(key, earliestSessionEndTime, latestSessionStartTime); } + @Override + public KeyValueIterator, AGG> backwardFindSessions(final K key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + return wrapped().backwardFindSessions(key, earliestSessionEndTime, latestSessionStartTime); + } + @Override public KeyValueIterator, AGG> findSessions(final K keyFrom, final K keyTo, @@ -222,6 +230,14 @@ public KeyValueIterator, AGG> findSessions(final K keyFrom, return wrapped().findSessions(keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime); } + @Override + public KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, + final K keyTo, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + return wrapped().backwardFindSessions(keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime); + } + @Override public void remove(final Windowed sessionKey) { throw new UnsupportedOperationException(ERROR_MESSAGE); @@ -243,10 +259,21 @@ public KeyValueIterator, AGG> fetch(final K key) { return wrapped().fetch(key); } + @Override + public KeyValueIterator, AGG> backwardFetch(final K key) { + return wrapped().backwardFetch(key); + } + @Override public KeyValueIterator, AGG> fetch(final K from, final K to) { return wrapped().fetch(from, to); } + + @Override + public KeyValueIterator, AGG> backwardFetch(final K from, + final K to) { + return wrapped().backwardFetch(from, to); + } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java index 230d2576178da..c93cd21484765 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java @@ -42,6 +42,10 @@ public interface ReadOnlySessionStore { */ KeyValueIterator, AGG> fetch(final K key); + default KeyValueIterator, AGG> backwardFetch(final K key) { + throw new UnsupportedOperationException(); + } + /** * Retrieve all aggregated sessions for the given range of keys. * This iterator must be closed after use. @@ -55,4 +59,8 @@ public interface ReadOnlySessionStore { * @throws NullPointerException If null is used for any of the keys. */ KeyValueIterator, AGG> fetch(final K from, final K to); + + default KeyValueIterator, AGG> backwardFetch(final K from, final K to) { + throw new UnsupportedOperationException(); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java index faaa751489af4..c29f390498b98 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java @@ -48,6 +48,10 @@ public interface SessionStore extends StateStore, ReadOnlySessionStore, AGG> findSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime); + default KeyValueIterator, AGG> backwardFindSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime) { + throw new UnsupportedOperationException(); + } + /** * Fetch any sessions in the given range of keys and the sessions end is ≥ earliestSessionEndTime and the sessions * start is ≤ latestSessionStartTime @@ -63,6 +67,10 @@ public interface SessionStore extends StateStore, ReadOnlySessionStore, AGG> findSessions(final K keyFrom, final K keyTo, final long earliestSessionEndTime, final long latestSessionStartTime); + default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, final K keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { + throw new UnsupportedOperationException(); + } + /** * Get the value of key from a single session. * diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java index 25068c054f5e2..3bf624ee3ebdd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java @@ -151,17 +151,42 @@ public KeyValueIterator, byte[]> findSessions(final Bytes key, final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? new CacheIteratorWrapper(key, earliestSessionEndTime, latestSessionStartTime) : context.cache().range(cacheName, - cacheFunction.cacheKey(keySchema.lowerRangeFixedSize(key, earliestSessionEndTime)), - cacheFunction.cacheKey(keySchema.upperRangeFixedSize(key, latestSessionStartTime)) + cacheFunction.cacheKey(keySchema.lowerRangeFixedSize(key, earliestSessionEndTime)), + cacheFunction.cacheKey(keySchema.upperRangeFixedSize(key, latestSessionStartTime)) ); final KeyValueIterator, byte[]> storeIterator = wrapped().findSessions(key, - earliestSessionEndTime, - latestSessionStartTime); + earliestSessionEndTime, + latestSessionStartTime); final HasNextCondition hasNextCondition = keySchema.hasNextCondition(key, - key, - earliestSessionEndTime, - latestSessionStartTime); + key, + earliestSessionEndTime, + latestSessionStartTime); + final PeekingKeyValueIterator filteredCacheIterator = + new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); + return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction); + } + + @Override + public KeyValueIterator, byte[]> backwardFindSessions(final Bytes key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + validateStoreOpen(); + + final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? + new CacheIteratorWrapper(key, earliestSessionEndTime, latestSessionStartTime) : + context.cache().reverseRange(cacheName, + cacheFunction.cacheKey(keySchema.lowerRangeFixedSize(key, earliestSessionEndTime)), + cacheFunction.cacheKey(keySchema.upperRangeFixedSize(key, latestSessionStartTime)) + ); + + final KeyValueIterator, byte[]> storeIterator = wrapped().backwardFindSessions(key, + earliestSessionEndTime, + latestSessionStartTime); + final HasNextCondition hasNextCondition = keySchema.hasNextCondition(key, + key, + earliestSessionEndTime, + latestSessionStartTime); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction); @@ -189,9 +214,39 @@ public KeyValueIterator, byte[]> findSessions(final Bytes keyFro keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime ); final HasNextCondition hasNextCondition = keySchema.hasNextCondition(keyFrom, - keyTo, - earliestSessionEndTime, - latestSessionStartTime); + keyTo, + earliestSessionEndTime, + latestSessionStartTime); + final PeekingKeyValueIterator filteredCacheIterator = + new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); + return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction); + } + + @Override + public KeyValueIterator, byte[]> backwardFindSessions(final Bytes keyFrom, + final Bytes keyTo, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + 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(); + } + + validateStoreOpen(); + + final Bytes cacheKeyFrom = cacheFunction.cacheKey(keySchema.lowerRange(keyFrom, earliestSessionEndTime)); + final Bytes cacheKeyTo = cacheFunction.cacheKey(keySchema.upperRange(keyTo, latestSessionStartTime)); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = context.cache().reverseRange(cacheName, cacheKeyFrom, cacheKeyTo); + + final KeyValueIterator, byte[]> storeIterator = wrapped().backwardFindSessions( + keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime + ); + final HasNextCondition hasNextCondition = keySchema.hasNextCondition(keyFrom, + keyTo, + earliestSessionEndTime, + latestSessionStartTime); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction); @@ -221,6 +276,12 @@ public KeyValueIterator, byte[]> fetch(final Bytes key) { return findSessions(key, 0, Long.MAX_VALUE); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes key) { + Objects.requireNonNull(key, "key cannot be null"); + return backwardFindSessions(key, 0, Long.MAX_VALUE); + } + @Override public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to) { @@ -229,6 +290,14 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, return findSessions(from, to, 0, Long.MAX_VALUE); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, + final Bytes to) { + Objects.requireNonNull(from, "from cannot be null"); + Objects.requireNonNull(to, "to cannot be null"); + return backwardFindSessions(from, to, 0, Long.MAX_VALUE); + } + public void flush() { context.cache().flush(cacheName); wrapped().flush(); @@ -242,7 +311,7 @@ public void close() { ); if (!suppressed.isEmpty()) { throwSuppressed("Caught an exception while closing caching session store for store " + name(), - suppressed); + suppressed); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java index cc586d3ba1a90..a41bec43d4e5d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java @@ -49,11 +49,21 @@ public KeyValueIterator, byte[]> findSessions(final Bytes key, f return wrapped().findSessions(key, earliestSessionEndTime, latestSessionStartTime); } + @Override + public KeyValueIterator, byte[]> backwardFindSessions(final Bytes key, final long earliestSessionEndTime, final long latestSessionStartTime) { + return wrapped().backwardFindSessions(key, earliestSessionEndTime, latestSessionStartTime); + } + @Override public KeyValueIterator, byte[]> findSessions(final Bytes keyFrom, final Bytes keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { return wrapped().findSessions(keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime); } + @Override + public KeyValueIterator, byte[]> backwardFindSessions(final Bytes keyFrom, final Bytes keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { + return wrapped().backwardFindSessions(keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime); + } + @Override public void remove(final Windowed sessionKey) { wrapped().remove(sessionKey); @@ -71,11 +81,21 @@ public byte[] fetchSession(final Bytes key, final long startTime, final long end return wrapped().fetchSession(key, startTime, endTime); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes key) { + return wrapped().backwardFetch(key); + } + @Override public KeyValueIterator, byte[]> fetch(final Bytes key) { return wrapped().fetch(key); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Bytes to) { + return wrapped().backwardFetch(from, to); + } + @Override public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to) { return wrapped().fetch(from, to); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index b10f4c4b52406..4d46d33d39f7b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -55,7 +55,7 @@ public class InMemorySessionStore implements SessionStore { private final long retentionPeriod; private final ConcurrentNavigableMap>> endTimeMap = new ConcurrentSkipListMap<>(); - private final Set openIterators = ConcurrentHashMap.newKeySet(); + private final Set openIterators = ConcurrentHashMap.newKeySet(); private volatile boolean open = false; @@ -165,9 +165,24 @@ public KeyValueIterator, byte[]> findSessions(final Bytes key, removeExpiredSegments(); return registerNewIterator(key, - key, - latestSessionStartTime, - endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator()); + key, + latestSessionStartTime, + endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator()); + } + + @Deprecated + @Override + public KeyValueIterator, byte[]> backwardFindSessions(final Bytes key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + Objects.requireNonNull(key, "key cannot be null"); + + removeExpiredSegments(); + + return registerNewIterator(key, + key, + earliestSessionEndTime, + endTimeMap.headMap(latestSessionStartTime, true).descendingMap().entrySet().iterator()); } @Override @@ -188,9 +203,33 @@ public KeyValueIterator, byte[]> findSessions(final Bytes keyFro } return registerNewIterator(keyFrom, - keyTo, - latestSessionStartTime, - endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator()); + keyTo, + latestSessionStartTime, + endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator()); + } + + @Deprecated + @Override + public KeyValueIterator, byte[]> backwardFindSessions(final Bytes keyFrom, + final Bytes keyTo, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + Objects.requireNonNull(keyFrom, "from key cannot be null"); + Objects.requireNonNull(keyTo, "to key cannot be null"); + + removeExpiredSegments(); + + 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(); + } + + return registerNewIterator(keyFrom, + keyTo, + earliestSessionEndTime, + endTimeMap.headMap(latestSessionStartTime, true).entrySet().iterator()); } @Override @@ -203,6 +242,16 @@ public KeyValueIterator, byte[]> fetch(final Bytes key) { return registerNewIterator(key, key, Long.MAX_VALUE, endTimeMap.entrySet().iterator()); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes key) { + + Objects.requireNonNull(key, "key cannot be null"); + + removeExpiredSegments(); + + return registerNewIterator(key, key, Long.MAX_VALUE, endTimeMap.descendingMap().entrySet().iterator()); + } + @Override public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to) { @@ -211,10 +260,19 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, final B removeExpiredSegments(); - return registerNewIterator(from, to, Long.MAX_VALUE, endTimeMap.entrySet().iterator()); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Bytes to) { + Objects.requireNonNull(from, "from key cannot be null"); + Objects.requireNonNull(to, "to key cannot be null"); + + removeExpiredSegments(); + + return registerNewIterator(from, to, Long.MAX_VALUE, endTimeMap.descendingMap().entrySet().iterator()); + } + @Override public boolean persistent() { return false; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java index 976ca5ecc0fbe..dffc87f5609f7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java @@ -180,6 +180,17 @@ public KeyValueIterator, V> fetch(final K key) { time); } + @Override + public KeyValueIterator, V> backwardFetch(final K key) { + Objects.requireNonNull(key, "key cannot be null"); + return new MeteredWindowedKeyValueIterator<>( + wrapped().backwardFetch(keyBytes(key)), + fetchSensor, + streamsMetrics, + serdes, + time); + } + @Override public KeyValueIterator, V> fetch(final K from, final K to) { @@ -193,6 +204,19 @@ public KeyValueIterator, V> fetch(final K from, time); } + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to) { + Objects.requireNonNull(from, "from cannot be null"); + Objects.requireNonNull(to, "to cannot be null"); + return new MeteredWindowedKeyValueIterator<>( + wrapped().backwardFetch(keyBytes(from), keyBytes(to)), + fetchSensor, + streamsMetrics, + serdes, + time); + } + @Override public KeyValueIterator, V> findSessions(final K key, final long earliestSessionEndTime, @@ -210,6 +234,23 @@ public KeyValueIterator, V> findSessions(final K key, time); } + @Override + public KeyValueIterator, V> backwardFindSessions(final K key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + Objects.requireNonNull(key, "key cannot be null"); + final Bytes bytesKey = keyBytes(key); + return new MeteredWindowedKeyValueIterator<>( + wrapped().backwardFindSessions( + bytesKey, + earliestSessionEndTime, + latestSessionStartTime), + fetchSensor, + streamsMetrics, + serdes, + time); + } + @Override public KeyValueIterator, V> findSessions(final K keyFrom, final K keyTo, @@ -231,6 +272,27 @@ public KeyValueIterator, V> findSessions(final K keyFrom, time); } + @Override + public KeyValueIterator, V> backwardFindSessions(final K keyFrom, + final K keyTo, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + Objects.requireNonNull(keyFrom, "keyFrom cannot be null"); + Objects.requireNonNull(keyTo, "keyTo cannot be null"); + final Bytes bytesKeyFrom = keyBytes(keyFrom); + final Bytes bytesKeyTo = keyBytes(keyTo); + return new MeteredWindowedKeyValueIterator<>( + wrapped().backwardFindSessions( + bytesKeyFrom, + bytesKeyTo, + earliestSessionEndTime, + latestSessionStartTime), + 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/RocksDBSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java index 2f7a211d5266a..338769abea4a5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java @@ -42,6 +42,18 @@ public KeyValueIterator, byte[]> findSessions(final Bytes key, return new WrappedSessionStoreIterator(bytesIterator); } + @Override + public KeyValueIterator, byte[]> backwardFindSessions(final Bytes key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + final KeyValueIterator bytesIterator = wrapped().backwardFetch( + key, + earliestSessionEndTime, + latestSessionStartTime + ); + return new WrappedSessionStoreIterator(bytesIterator); + } + @Override public KeyValueIterator, byte[]> findSessions(final Bytes keyFrom, final Bytes keyTo, @@ -56,6 +68,20 @@ public KeyValueIterator, byte[]> findSessions(final Bytes keyFro return new WrappedSessionStoreIterator(bytesIterator); } + @Override + public KeyValueIterator, byte[]> backwardFindSessions(final Bytes keyFrom, + final Bytes keyTo, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + final KeyValueIterator bytesIterator = wrapped().backwardFetch( + keyFrom, + keyTo, + earliestSessionEndTime, + latestSessionStartTime + ); + return new WrappedSessionStoreIterator(bytesIterator); + } + @Override public byte[] fetchSession(final Bytes key, final long startTime, final long endTime) { return wrapped().get(SessionKeySchema.toBinary(key, startTime, endTime)); @@ -66,11 +92,21 @@ public KeyValueIterator, byte[]> fetch(final Bytes key) { return findSessions(key, 0, Long.MAX_VALUE); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes key) { + return backwardFindSessions(key, 0, Long.MAX_VALUE); + } + @Override public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to) { return findSessions(from, to, 0, Long.MAX_VALUE); } + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Bytes to) { + return backwardFindSessions(from, to, 0, Long.MAX_VALUE); + } + @Override public void remove(final Windowed key) { wrapped().remove(SessionKeySchema.toBinary(key)); From 07ef39ff4474409f2333877182cc38165c1d81ac Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 3 Jul 2020 08:50:42 +0100 Subject: [PATCH 04/24] move read ops from session store to read-only --- .../streams/state/ReadOnlySessionStore.java | 49 ++++++ .../kafka/streams/state/SessionStore.java | 48 ------ .../CompositeReadOnlySessionStore.java | 152 +++++++++++++++++- .../state/internals/InMemorySessionStore.java | 1 - .../kafka/test/ReadOnlySessionStoreStub.java | 42 ++++- 5 files changed, 236 insertions(+), 56 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java index c93cd21484765..a0af5bc1a2325 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java @@ -28,6 +28,55 @@ * @param the aggregated value type */ public interface ReadOnlySessionStore { + + /** + * Fetch any sessions with the matching key and the sessions end is ≥ earliestSessionEndTime and the sessions + * start is ≤ latestSessionStartTime + * + * This iterator must be closed after use. + * + * @param key the key to return sessions for + * @param earliestSessionEndTime the end timestamp of the earliest session to search for + * @param latestSessionStartTime the end timestamp of the latest session to search for + * @return iterator of sessions with the matching key and aggregated values + * @throws NullPointerException If null is used for key. + */ + KeyValueIterator, AGG> findSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime); + + default KeyValueIterator, AGG> backwardFindSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime) { + throw new UnsupportedOperationException(); + } + + /** + * Fetch any sessions in the given range of keys and the sessions end is ≥ earliestSessionEndTime and the sessions + * start is ≤ latestSessionStartTime + * + * This iterator must be closed after use. + * + * @param keyFrom The first key that could be in the range + * @param keyTo The last key that could be in the range + * @param earliestSessionEndTime the end timestamp of the earliest session to search for + * @param latestSessionStartTime the end timestamp of the latest session to search for + * @return iterator of sessions with the matching keys and aggregated values + * @throws NullPointerException If null is used for any key. + */ + KeyValueIterator, AGG> findSessions(final K keyFrom, final K keyTo, final long earliestSessionEndTime, final long latestSessionStartTime); + + default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, final K keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { + throw new UnsupportedOperationException(); + } + + /** + * Get the value of key from a single session. + * + * @param key the key to fetch + * @param startTime start timestamp of the session + * @param endTime end timestamp of the session + * @return The value or {@code null} if no session associated with the key can be found + * @throws NullPointerException If {@code null} is used for any key. + */ + AGG fetchSession(final K key, final long startTime, final long endTime); + /** * Retrieve all aggregated sessions for the provided key. * This iterator must be closed after use. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java index c29f390498b98..47f48d5e8143c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java @@ -34,54 +34,6 @@ */ public interface SessionStore extends StateStore, ReadOnlySessionStore { - /** - * Fetch any sessions with the matching key and the sessions end is ≥ earliestSessionEndTime and the sessions - * start is ≤ latestSessionStartTime - * - * This iterator must be closed after use. - * - * @param key the key to return sessions for - * @param earliestSessionEndTime the end timestamp of the earliest session to search for - * @param latestSessionStartTime the end timestamp of the latest session to search for - * @return iterator of sessions with the matching key and aggregated values - * @throws NullPointerException If null is used for key. - */ - KeyValueIterator, AGG> findSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime); - - default KeyValueIterator, AGG> backwardFindSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime) { - throw new UnsupportedOperationException(); - } - - /** - * Fetch any sessions in the given range of keys and the sessions end is ≥ earliestSessionEndTime and the sessions - * start is ≤ latestSessionStartTime - * - * This iterator must be closed after use. - * - * @param keyFrom The first key that could be in the range - * @param keyTo The last key that could be in the range - * @param earliestSessionEndTime the end timestamp of the earliest session to search for - * @param latestSessionStartTime the end timestamp of the latest session to search for - * @return iterator of sessions with the matching keys and aggregated values - * @throws NullPointerException If null is used for any key. - */ - KeyValueIterator, AGG> findSessions(final K keyFrom, final K keyTo, final long earliestSessionEndTime, final long latestSessionStartTime); - - default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, final K keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { - throw new UnsupportedOperationException(); - } - - /** - * Get the value of key from a single session. - * - * @param key the key to fetch - * @param startTime start timestamp of the session - * @param endTime end timestamp of the session - * @return The value or {@code null} if no session associated with the key can be found - * @throws NullPointerException If {@code null} is used for any key. - */ - AGG fetchSession(final K key, final long startTime, final long endTime); - /** * Remove the session aggregated with provided {@link Windowed} key from the store * @param sessionKey key of the session to remove diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java index 63d551c1d089e..bef544291d8d9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java @@ -42,6 +42,113 @@ public CompositeReadOnlySessionStore(final StateStoreProvider storeProvider, this.storeName = storeName; } + @Override + public KeyValueIterator, V> findSessions(K key, long earliestSessionEndTime, long latestSessionStartTime) { + Objects.requireNonNull(key, "key can't be null"); + final List> stores = storeProvider.stores(storeName, queryableStoreType); + for (final ReadOnlySessionStore store : stores) { + try { + final KeyValueIterator, V> result = store.findSessions(key, earliestSessionEndTime, latestSessionStartTime); + if (!result.hasNext()) { + result.close(); + } else { + return result; + } + } catch (final InvalidStateStoreException ise) { + throw new InvalidStateStoreException("State store [" + storeName + "] is not available anymore" + + " and may have been migrated to another instance; " + + "please re-discover its location from the state metadata. " + + "Original error message: " + ise.toString()); + } + } + return KeyValueIterators.emptyIterator(); + } + + @Override + public KeyValueIterator, V> backwardFindSessions(K key, long earliestSessionEndTime, long latestSessionStartTime) { + Objects.requireNonNull(key, "key can't be null"); + final List> stores = storeProvider.stores(storeName, queryableStoreType); + for (final ReadOnlySessionStore store : stores) { + try { + final KeyValueIterator, V> result = store.backwardFindSessions(key, earliestSessionEndTime, latestSessionStartTime); + if (!result.hasNext()) { + result.close(); + } else { + return result; + } + } catch (final InvalidStateStoreException ise) { + throw new InvalidStateStoreException("State store [" + storeName + "] is not available anymore" + + " and may have been migrated to another instance; " + + "please re-discover its location from the state metadata. " + + "Original error message: " + ise.toString()); + } + } + return KeyValueIterators.emptyIterator(); + } + + @Override + public KeyValueIterator, V> findSessions(K keyFrom, K keyTo, long earliestSessionEndTime, long latestSessionStartTime) { + Objects.requireNonNull(keyFrom, "from can't be null"); + Objects.requireNonNull(keyTo, "to can't be null"); + final List> stores = storeProvider.stores(storeName, queryableStoreType); + for (final ReadOnlySessionStore store : stores) { + try { + final KeyValueIterator, V> result = store.findSessions(keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime); + if (!result.hasNext()) { + result.close(); + } else { + return result; + } + } catch (final InvalidStateStoreException ise) { + throw new InvalidStateStoreException("State store [" + storeName + "] is not available anymore" + + " and may have been migrated to another instance; " + + "please re-discover its location from the state metadata. " + + "Original error message: " + ise.toString()); + } + } + return KeyValueIterators.emptyIterator(); + } + + @Override + public KeyValueIterator, V> backwardFindSessions(K keyFrom, K keyTo, long earliestSessionEndTime, long latestSessionStartTime) { + Objects.requireNonNull(keyFrom, "from can't be null"); + Objects.requireNonNull(keyTo, "to can't be null"); + final List> stores = storeProvider.stores(storeName, queryableStoreType); + for (final ReadOnlySessionStore store : stores) { + try { + final KeyValueIterator, V> result = store.backwardFindSessions(keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime); + if (!result.hasNext()) { + result.close(); + } else { + return result; + } + } catch (final InvalidStateStoreException ise) { + throw new InvalidStateStoreException("State store [" + storeName + "] is not available anymore" + + " and may have been migrated to another instance; " + + "please re-discover its location from the state metadata. " + + "Original error message: " + ise.toString()); + } + } + return KeyValueIterators.emptyIterator(); + } + + @Override + public V fetchSession(K key, long startTime, long endTime) { + Objects.requireNonNull(key, "key can't be null"); + final List> stores = storeProvider.stores(storeName, queryableStoreType); + for (final ReadOnlySessionStore store : stores) { + try { + return store.fetchSession(key, startTime, endTime); + } catch (final InvalidStateStoreException ise) { + throw new InvalidStateStoreException("State store [" + storeName + "] is not available anymore" + + " and may have been migrated to another instance; " + + "please re-discover its location from the state metadata. " + + "Original error message: " + ise.toString()); + } + } + return null; + } + @Override public KeyValueIterator, V> fetch(final K key) { Objects.requireNonNull(key, "key can't be null"); @@ -56,9 +163,31 @@ public KeyValueIterator, V> fetch(final K key) { } } catch (final InvalidStateStoreException ise) { throw new InvalidStateStoreException("State store [" + storeName + "] is not available anymore" + - " and may have been migrated to another instance; " + - "please re-discover its location from the state metadata. " + - "Original error message: " + ise.toString()); + " and may have been migrated to another instance; " + + "please re-discover its location from the state metadata. " + + "Original error message: " + ise.toString()); + } + } + return KeyValueIterators.emptyIterator(); + } + + @Override + public KeyValueIterator, V> backwardFetch(K key) { + Objects.requireNonNull(key, "key can't be null"); + final List> stores = storeProvider.stores(storeName, queryableStoreType); + for (final ReadOnlySessionStore store : stores) { + try { + final KeyValueIterator, V> result = store.backwardFetch(key); + if (!result.hasNext()) { + result.close(); + } else { + return result; + } + } catch (final InvalidStateStoreException ise) { + throw new InvalidStateStoreException("State store [" + storeName + "] is not available anymore" + + " and may have been migrated to another instance; " + + "please re-discover its location from the state metadata. " + + "Original error message: " + ise.toString()); } } return KeyValueIterators.emptyIterator(); @@ -70,8 +199,19 @@ public KeyValueIterator, V> fetch(final K from, final K to) { Objects.requireNonNull(to, "to can't be null"); final NextIteratorFunction, V, ReadOnlySessionStore> nextIteratorFunction = store -> store.fetch(from, to); return new DelegatingPeekingKeyValueIterator<>(storeName, - new CompositeKeyValueIterator<>( - storeProvider.stores(storeName, queryableStoreType).iterator(), - nextIteratorFunction)); + new CompositeKeyValueIterator<>( + storeProvider.stores(storeName, queryableStoreType).iterator(), + nextIteratorFunction)); + } + + @Override + public KeyValueIterator, V> backwardFetch(K from, K to) { + Objects.requireNonNull(from, "from can't be null"); + Objects.requireNonNull(to, "to can't be null"); + final NextIteratorFunction, V, ReadOnlySessionStore> nextIteratorFunction = store -> store.backwardFetch(from, to); + return new DelegatingPeekingKeyValueIterator<>(storeName, + new CompositeKeyValueIterator<>( + storeProvider.stores(storeName, queryableStoreType).iterator(), + nextIteratorFunction)); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index 4d46d33d39f7b..c499c67bbfbf1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -170,7 +170,6 @@ public KeyValueIterator, byte[]> findSessions(final Bytes key, endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator()); } - @Deprecated @Override public KeyValueIterator, byte[]> backwardFindSessions(final Bytes key, final long earliestSessionEndTime, diff --git a/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java b/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java index 4f6d5debb0ada..5298894dd34eb 100644 --- a/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java @@ -42,6 +42,31 @@ public void put(final Windowed sessionKey, final V value) { sessions.get(sessionKey.key()).add(KeyValue.pair(sessionKey, value)); } + @Override + public KeyValueIterator, V> findSessions(K key, long earliestSessionEndTime, long latestSessionStartTime) { + return null; + } + + @Override + public KeyValueIterator, V> backwardFindSessions(K key, long earliestSessionEndTime, long latestSessionStartTime) { + return null; + } + + @Override + public KeyValueIterator, V> findSessions(K keyFrom, K keyTo, long earliestSessionEndTime, long latestSessionStartTime) { + return null; + } + + @Override + public KeyValueIterator, V> backwardFindSessions(K keyFrom, K keyTo, long earliestSessionEndTime, long latestSessionStartTime) { + return null; + } + + @Override + public V fetchSession(K key, long startTime, long endTime) { + return null; + } + @Override public KeyValueIterator, V> fetch(final K key) { if (!open) { @@ -53,6 +78,17 @@ public KeyValueIterator, V> fetch(final K key) { return new KeyValueIteratorStub<>(sessions.get(key).iterator()); } + @Override + public KeyValueIterator, V> backwardFetch(K key) { + if (!open) { + throw new InvalidStateStoreException("not open"); + } + if (!sessions.containsKey(key)) { + return new KeyValueIteratorStub<>(Collections., V>>emptyIterator()); + } + return new KeyValueIteratorStub<>(sessions.descendingMap().get(key).iterator()); + } + @Override public KeyValueIterator, V> fetch(final K from, final K to) { if (!open) { @@ -82,11 +118,15 @@ public boolean hasNext() { public KeyValue, V> next() { return it.next(); } - } ); } + @Override + public KeyValueIterator, V> backwardFetch(K from, K to) { + return null; + } + @Override public String name() { return ""; From b18f73775a41b493f33f1b0a9cc632a5cced0123 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 15 Jul 2020 09:49:03 +0100 Subject: [PATCH 05/24] fix comparator for reverse --- .../state/internals/RocksDBRangeIterator.java | 29 ++++++++++++------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java index 3f27b556487da..8e058cc9b5ca9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java @@ -29,7 +29,8 @@ class RocksDBRangeIterator extends RocksDbIterator { // comparator to be pluggable, and the default is lexicographic, so it's // safe to just force lexicographic comparator here for now. private final Comparator comparator = Bytes.BYTES_LEXICO_COMPARATOR; - private final byte[] rawToKey; + private final byte[] rawLastKey; + private final boolean reverse; RocksDBRangeIterator(final String storeName, final RocksIterator iter, @@ -38,15 +39,15 @@ class RocksDBRangeIterator extends RocksDbIterator { final Bytes to, final boolean reverse) { super(storeName, iter, openIterators, reverse); + this.reverse = reverse; if (reverse) { - iter.seek(to.get()); - rawToKey = from.get(); - } - else { + iter.seekForPrev(to.get()); + rawLastKey = from.get(); + } else { iter.seek(from.get()); - rawToKey = to.get(); + rawLastKey = to.get(); } - if (rawToKey == null) { + if (rawLastKey == null) { throw new NullPointerException("RocksDBRangeIterator: RawToKey is null for key " + to); } } @@ -58,10 +59,18 @@ public KeyValue makeNext() { if (next == null) { return allDone(); } else { - if (comparator.compare(next.key.get(), rawToKey) <= 0) { - return next; + if (!reverse) { + if (comparator.compare(next.key.get(), rawLastKey) <= 0) { + return next; + } else { + return allDone(); + } } else { - return allDone(); + if (comparator.compare(next.key.get(), rawLastKey) >= 0) { + return next; + } else { + return allDone(); + } } } } From c044e8786e864595eb25b2684bbf200c5cc04efb Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 15 Jul 2020 10:09:45 +0100 Subject: [PATCH 06/24] fix checkstyle --- .../AbstractRocksDBSegmentedBytesStore.java | 9 +++- .../CompositeReadOnlySessionStore.java | 24 ++++++--- .../internals/InMemoryKeyValueStore.java | 2 +- .../ReadOnlyKeyValueStoreFacade.java | 3 +- .../state/internals/RocksDBWindowStore.java | 7 ++- .../internals/AbstractKeyValueStoreTest.java | 50 ++++++++++++++++++- 6 files changed, 81 insertions(+), 14 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java index af136587ea712..06e5b955f22c6 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 @@ -67,7 +67,9 @@ public KeyValueIterator fetch(final Bytes key, } @Override - public KeyValueIterator backwardFetch(Bytes key, long from, long to) { + public KeyValueIterator backwardFetch(final Bytes key, + final long from, + final long to) { return fetch(key, from, to, true); } @@ -96,7 +98,10 @@ public KeyValueIterator fetch(final Bytes keyFrom, } @Override - public KeyValueIterator backwardFetch(Bytes keyFrom, Bytes keyTo, long from, long to) { + public KeyValueIterator backwardFetch(final Bytes keyFrom, + final Bytes keyTo, + final long from, + final long to) { return fetch(keyFrom, keyTo, from, to, true); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java index bef544291d8d9..46f3e115885de 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java @@ -43,7 +43,9 @@ public CompositeReadOnlySessionStore(final StateStoreProvider storeProvider, } @Override - public KeyValueIterator, V> findSessions(K key, long earliestSessionEndTime, long latestSessionStartTime) { + public KeyValueIterator, V> findSessions(final K key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { Objects.requireNonNull(key, "key can't be null"); final List> stores = storeProvider.stores(storeName, queryableStoreType); for (final ReadOnlySessionStore store : stores) { @@ -65,7 +67,9 @@ public KeyValueIterator, V> findSessions(K key, long earliestSession } @Override - public KeyValueIterator, V> backwardFindSessions(K key, long earliestSessionEndTime, long latestSessionStartTime) { + public KeyValueIterator, V> backwardFindSessions(final K key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { Objects.requireNonNull(key, "key can't be null"); final List> stores = storeProvider.stores(storeName, queryableStoreType); for (final ReadOnlySessionStore store : stores) { @@ -87,7 +91,10 @@ public KeyValueIterator, V> backwardFindSessions(K key, long earlies } @Override - public KeyValueIterator, V> findSessions(K keyFrom, K keyTo, long earliestSessionEndTime, long latestSessionStartTime) { + public KeyValueIterator, V> findSessions(final K keyFrom, + final K keyTo, + final long earliestSessionEndTime, + final long latestSessionStartTime) { Objects.requireNonNull(keyFrom, "from can't be null"); Objects.requireNonNull(keyTo, "to can't be null"); final List> stores = storeProvider.stores(storeName, queryableStoreType); @@ -110,7 +117,10 @@ public KeyValueIterator, V> findSessions(K keyFrom, K keyTo, long ea } @Override - public KeyValueIterator, V> backwardFindSessions(K keyFrom, K keyTo, long earliestSessionEndTime, long latestSessionStartTime) { + public KeyValueIterator, V> backwardFindSessions(final K keyFrom, + final K keyTo, + final long earliestSessionEndTime, + final long latestSessionStartTime) { Objects.requireNonNull(keyFrom, "from can't be null"); Objects.requireNonNull(keyTo, "to can't be null"); final List> stores = storeProvider.stores(storeName, queryableStoreType); @@ -133,7 +143,7 @@ public KeyValueIterator, V> backwardFindSessions(K keyFrom, K keyTo, } @Override - public V fetchSession(K key, long startTime, long endTime) { + public V fetchSession(final K key, final long startTime, final long endTime) { Objects.requireNonNull(key, "key can't be null"); final List> stores = storeProvider.stores(storeName, queryableStoreType); for (final ReadOnlySessionStore store : stores) { @@ -172,7 +182,7 @@ public KeyValueIterator, V> fetch(final K key) { } @Override - public KeyValueIterator, V> backwardFetch(K key) { + public KeyValueIterator, V> backwardFetch(final K key) { Objects.requireNonNull(key, "key can't be null"); final List> stores = storeProvider.stores(storeName, queryableStoreType); for (final ReadOnlySessionStore store : stores) { @@ -205,7 +215,7 @@ public KeyValueIterator, V> fetch(final K from, final K to) { } @Override - public KeyValueIterator, V> backwardFetch(K from, K to) { + public KeyValueIterator, V> backwardFetch(final K from, final K to) { Objects.requireNonNull(from, "from can't be null"); Objects.requireNonNull(to, "to can't be null"); final NextIteratorFunction, V, ReadOnlySessionStore> nextIteratorFunction = store -> store.backwardFetch(from, to); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java index 1a1ad389c9c75..9e9795f2dd569 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java @@ -114,7 +114,7 @@ public synchronized KeyValueIterator range(final Bytes from, fina } @Override - public KeyValueIterator reverseRange(Bytes from, Bytes to) { + public KeyValueIterator reverseRange(final Bytes from, final Bytes to) { return range(from, to, true); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyKeyValueStoreFacade.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyKeyValueStoreFacade.java index 04acc58138393..2ffa3f3529863 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyKeyValueStoreFacade.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyKeyValueStoreFacade.java @@ -41,7 +41,8 @@ public KeyValueIterator range(final K from, } @Override - public KeyValueIterator reverseRange(K from, K to) { + public KeyValueIterator reverseRange(final K from, + final K to) { return new KeyValueIteratorFacade<>(inner.reverseRange(from, to)); } 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 d9eb6b955d5e5..fbe03fef29fa0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -82,7 +82,7 @@ public WindowStoreIterator fetch(final Bytes key, final long timeFrom, f } @Override - public WindowStoreIterator backwardFetch(Bytes key, Instant from, Instant to) throws IllegalArgumentException { + public WindowStoreIterator backwardFetch(final Bytes key, final Instant from, final Instant to) { final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); final KeyValueIterator bytesIterator = wrapped().backwardFetch(key, timeFrom, timeTo); @@ -100,7 +100,10 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, } @Override - public KeyValueIterator, byte[]> backwardFetch(Bytes from, Bytes to, Instant fromTime, Instant toTime) { + public KeyValueIterator, byte[]> backwardFetch(final Bytes from, + final Bytes to, + final Instant fromTime, + final Instant toTime) { final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "from")); final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "to")); final KeyValueIterator bytesIterator = wrapped().backwardFetch(from, to, timeFrom, timeTo); 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 ca5fd2fc7a840..c47bca8887e72 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 @@ -188,7 +188,55 @@ public void testPutGetRange() { } @Test - public void testPutGetRangeWithDefaultSerdes() { + public void testPutGetReverseRange() { + // Verify that the store reads and writes correctly ... + store.put(0, "zero"); + store.put(1, "one"); + store.put(2, "two"); + store.put(4, "four"); + store.put(5, "five"); + assertEquals(5, driver.sizeOf(store)); + assertEquals("zero", store.get(0)); + assertEquals("one", store.get(1)); + assertEquals("two", store.get(2)); + assertNull(store.get(3)); + assertEquals("four", store.get(4)); + assertEquals("five", store.get(5)); + // Flush now so that for caching store, we will not skip the deletion following an put + store.flush(); + store.delete(5); + assertEquals(4, driver.sizeOf(store)); + + // Flush the store and verify all current entries were properly flushed ... + store.flush(); + assertEquals("zero", driver.flushedEntryStored(0)); + assertEquals("one", driver.flushedEntryStored(1)); + assertEquals("two", driver.flushedEntryStored(2)); + assertEquals("four", driver.flushedEntryStored(4)); + assertNull(driver.flushedEntryStored(5)); + + assertFalse(driver.flushedEntryRemoved(0)); + assertFalse(driver.flushedEntryRemoved(1)); + assertFalse(driver.flushedEntryRemoved(2)); + assertFalse(driver.flushedEntryRemoved(4)); + assertTrue(driver.flushedEntryRemoved(5)); + + final HashMap expectedContents = new HashMap<>(); + expectedContents.put(2, "two"); + expectedContents.put(4, "four"); + + // Check range iteration ... + assertEquals(expectedContents, getContents(store.reverseRange(2, 4))); + assertEquals(expectedContents, getContents(store.reverseRange(2, 6))); + + // Check all iteration ... + expectedContents.put(0, "zero"); + expectedContents.put(1, "one"); + assertEquals(expectedContents, getContents(store.reverseAll())); + } + + @Test + public void testPutGetWithDefaultSerdes() { // Verify that the store reads and writes correctly ... store.put(0, "zero"); store.put(1, "one"); From 5c6016653668e4f11f0414e96512d81c60b6e83e Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 21 Jul 2020 09:53:32 +0100 Subject: [PATCH 07/24] complete read only decorator --- .../internals/AbstractReadOnlyDecorator.java | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) 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 fb6ae593c078b..466877b9adebc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.time.Instant; import java.util.List; import org.apache.kafka.streams.KeyValue; @@ -91,11 +92,22 @@ public KeyValueIterator range(final K from, return wrapped().range(from, to); } + @Override + public KeyValueIterator reverseRange(final K from, + final K to) { + return wrapped().reverseRange(from, to); + } + @Override public KeyValueIterator all() { return wrapped().all(); } + @Override + public KeyValueIterator reverseAll() { + return wrapped().reverseAll(); + } + @Override public long approximateNumEntries() { return wrapped().approximateNumEntries(); @@ -169,6 +181,13 @@ public WindowStoreIterator fetch(final K key, return wrapped().fetch(key, timeFrom, timeTo); } + @Override + public WindowStoreIterator backwardFetch(final K key, + final Instant timeFrom, + final Instant timeTo) { + return wrapped().backwardFetch(key, timeFrom, timeTo); + } + @Override @Deprecated public KeyValueIterator, V> fetch(final K from, @@ -178,17 +197,36 @@ public KeyValueIterator, V> fetch(final K from, return wrapped().fetch(from, to, timeFrom, timeTo); } + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final Instant timeFrom, + final Instant timeTo) { + return wrapped().backwardFetch(from, to, timeFrom, timeTo); + } + @Override public KeyValueIterator, V> all() { return wrapped().all(); } + @Override + public KeyValueIterator, V> backwardAll() { + return wrapped().backwardAll(); + } + @Override @Deprecated public KeyValueIterator, V> fetchAll(final long timeFrom, final long timeTo) { return wrapped().fetchAll(timeFrom, timeTo); } + + @Override + public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, + final Instant timeTo) { + return wrapped().backwardFetchAll(timeFrom, timeTo); + } } static class TimestampedWindowStoreReadOnlyDecorator From 3e3bc2c5051fd8af290840f475362fca671d7e9f Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 21 Jul 2020 10:28:13 +0100 Subject: [PATCH 08/24] fix reverse iteration on in-mem structures --- .../streams/state/ReadOnlyKeyValueStore.java | 17 +++++++ .../internals/InMemoryKeyValueStore.java | 16 +++---- .../streams/state/internals/NamedCache.java | 13 ++--- .../internals/AbstractKeyValueStoreTest.java | 48 +++++++++++++++++++ 4 files changed, 79 insertions(+), 15 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java index ddd01bceb66ee..c02b78b8c907d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java @@ -55,6 +55,16 @@ public interface ReadOnlyKeyValueStore { */ KeyValueIterator range(K from, K to); + /** + * Get an reverse iterator over a given range of keys. This iterator must be closed after use. + * The returned iterator must be safe from {@link java.util.ConcurrentModificationException}s + * and must not return null values. No ordering guarantees are provided. + * @param from The last key that could be in the range + * @param to The first key that could be in the range + * @return The reverse iterator for this range. + * @throws NullPointerException If null is used for from or to. + * @throws InvalidStateStoreException if the store is not initialized + */ default KeyValueIterator reverseRange(K from, K to) { throw new UnsupportedOperationException(); } @@ -68,6 +78,13 @@ default KeyValueIterator reverseRange(K from, K to) { */ KeyValueIterator all(); + /** + * Return an reverse iterator over all keys in this store. This iterator must be closed after use. + * The returned iterator must be safe from {@link java.util.ConcurrentModificationException}s + * and must not return null values. No ordering guarantees are provided. + * @return An reverse iterator of all key/value pairs in the store. + * @throws InvalidStateStoreException if the store is not initialized + */ default KeyValueIterator reverseAll() { throw new UnsupportedOperationException(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java index 9e9795f2dd569..137edfec36a69 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java @@ -126,26 +126,23 @@ KeyValueIterator range(final Bytes from, final Bytes to, final bo return KeyValueIterators.emptyIterator(); } - if (reverse) return new DelegatingPeekingKeyValueIterator<>( - name, - new InMemoryKeyValueIterator(map.subMap(from, true, to, true).descendingKeySet())); - else return new DelegatingPeekingKeyValueIterator<>( + return new DelegatingPeekingKeyValueIterator<>( name, - new InMemoryKeyValueIterator(map.subMap(from, true, to, true).keySet())); + new InMemoryKeyValueIterator(map.subMap(from, true, to, true).keySet(), reverse)); } @Override public synchronized KeyValueIterator all() { return new DelegatingPeekingKeyValueIterator<>( name, - new InMemoryKeyValueIterator(map.keySet())); + new InMemoryKeyValueIterator(map.keySet(), false)); } @Override public KeyValueIterator reverseAll() { return new DelegatingPeekingKeyValueIterator<>( name, - new InMemoryKeyValueIterator(map.descendingKeySet())); + new InMemoryKeyValueIterator(map.keySet(), true)); } @Override @@ -168,8 +165,9 @@ public void close() { private class InMemoryKeyValueIterator implements KeyValueIterator { private final Iterator iter; - private InMemoryKeyValueIterator(final Set keySet) { - this.iter = new TreeSet<>(keySet).iterator(); + private InMemoryKeyValueIterator(final Set keySet, final boolean reverse) { + if (reverse) this.iter = new TreeSet<>(keySet).descendingIterator(); + else this.iter = new TreeSet<>(keySet).iterator(); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java index 438a438503936..be8e9009bd59f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java @@ -281,23 +281,24 @@ public boolean isEmpty() { } synchronized Iterator keyRange(final Bytes from, final Bytes to) { - return keySetIterator(cache.navigableKeySet().subSet(from, true, to, true)); + return keySetIterator(cache.navigableKeySet().subSet(from, true, to, true), false); } synchronized Iterator reverseKeyRange(final Bytes from, final Bytes to) { - return keySetIterator(cache.subMap(from, true, to, true).descendingKeySet()); + return keySetIterator(cache.navigableKeySet().subSet(from, true, to, true), true); } - private Iterator keySetIterator(final Set keySet) { - return new TreeSet<>(keySet).iterator(); + private Iterator keySetIterator(final Set keySet, final boolean reverse) { + if (reverse) return new TreeSet<>(keySet).descendingIterator(); + else return new TreeSet<>(keySet).iterator(); } synchronized Iterator allKeys() { - return keySetIterator(cache.navigableKeySet()); + return keySetIterator(cache.navigableKeySet(), false); } synchronized Iterator reverseAllKeys() { - return keySetIterator(cache.descendingKeySet()); + return keySetIterator(cache.navigableKeySet(), true); } synchronized LRUCacheEntry first() { 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 c47bca8887e72..55c43c2756881 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 @@ -419,7 +419,25 @@ public void shouldPutAll() { allReturned.add(iterator.next()); } assertThat(allReturned, equalTo(expectedReturned)); + } + + @Test + public void shouldPutReverseAll() { + final List> entries = new ArrayList<>(); + entries.add(new KeyValue<>(1, "one")); + entries.add(new KeyValue<>(2, "two")); + + store.putAll(entries); + + final List> allReturned = new ArrayList<>(); + final List> expectedReturned = + Arrays.asList(KeyValue.pair(2, "two"), KeyValue.pair(1, "one")); + final Iterator> iterator = store.reverseAll(); + while (iterator.hasNext()) { + allReturned.add(iterator.next()); + } + assertThat(allReturned, equalTo(expectedReturned)); } @Test @@ -445,6 +463,21 @@ public void shouldReturnSameResultsForGetAndRangeWithEqualKeys() { assertFalse(iterator.hasNext()); } + @Test + public void shouldReturnSameResultsForGetAndReverseRangeWithEqualKeys() { + final List> entries = new ArrayList<>(); + entries.add(new KeyValue<>(1, "one")); + entries.add(new KeyValue<>(2, "two")); + entries.add(new KeyValue<>(3, "three")); + + store.putAll(entries); + + final Iterator> iterator = store.reverseRange(2, 2); + + assertEquals(iterator.next().value, store.get(2)); + assertFalse(iterator.hasNext()); + } + @Test public void shouldNotThrowConcurrentModificationException() { store.put(0, "zero"); @@ -470,6 +503,21 @@ public void shouldNotThrowInvalidRangeExceptionWithNegativeFromKey() { " Note that the built-in numerical serdes do not follow this for negative numbers") ); } + } + + @Test + public void shouldNotThrowInvalidReverseRangeExceptionWithNegativeFromKey() { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister()) { + final KeyValueIterator iterator = store.reverseRange(-1, 1); + assertFalse(iterator.hasNext()); + final List messages = appender.getMessages(); + assertThat( + messages, + hasItem("Returning empty iterator for fetch with invalid key range: from > to." + + " This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes." + + " Note that the built-in numerical serdes do not follow this for negative numbers") + ); + } } } From 59cda8ee0033aa1e9fbc218ad5a021c33ace3425 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 21 Jul 2020 10:39:20 +0100 Subject: [PATCH 09/24] test composite readonly kv store --- .../CompositeReadOnlyKeyValueStoreTest.java | 87 +++++++++++++++++++ 1 file changed, 87 insertions(+) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java index 67b876ab48ed0..9564dca12748f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java @@ -107,6 +107,16 @@ public void shouldThrowNullPointerExceptionOnRangeNullToKey() { theStore.range("from", null); } + @Test(expected = NullPointerException.class) + public void shouldThrowNullPointerExceptionOnReverseRangeNullFromKey() { + theStore.reverseRange(null, "to"); + } + + @Test(expected = NullPointerException.class) + public void shouldThrowNullPointerExceptionOnReverseRangeNullToKey() { + theStore.reverseRange("from", null); + } + @Test public void shouldReturnValueIfExists() { stubOneUnderlying.put("key", "value"); @@ -150,6 +160,17 @@ public void shouldThrowUnsupportedOperationExceptionWhileRemove() { } catch (final UnsupportedOperationException e) { } } + @Test + public void shouldThrowUnsupportedOperationExceptionWhileReverseRange() { + stubOneUnderlying.put("a", "1"); + stubOneUnderlying.put("b", "1"); + final KeyValueIterator keyValueIterator = theStore.reverseRange("a", "b"); + try { + keyValueIterator.remove(); + fail("Should have thrown UnsupportedOperationException"); + } catch (final UnsupportedOperationException e) { } + } + @Test public void shouldThrowUnsupportedOperationExceptionWhileRange() { stubOneUnderlying.put("a", "1"); @@ -185,6 +206,18 @@ public void shouldSupportRange() { assertEquals(2, results.size()); } + @Test + public void shouldSupportReverseRange() { + stubOneUnderlying.put("a", "a"); + stubOneUnderlying.put("b", "b"); + stubOneUnderlying.put("c", "c"); + + final List> results = toList(theStore.reverseRange("a", "b")); + assertTrue(results.contains(new KeyValue<>("a", "a"))); + assertTrue(results.contains(new KeyValue<>("b", "b"))); + assertEquals(2, results.size()); + } + @Test public void shouldSupportRangeAcrossMultipleKVStores() { final KeyValueStore cache = newStoreInstance(); @@ -206,6 +239,27 @@ public void shouldSupportRangeAcrossMultipleKVStores() { assertEquals(4, results.size()); } + @Test + public void shouldSupportReverseRangeAcrossMultipleKVStores() { + final KeyValueStore cache = newStoreInstance(); + stubProviderTwo.addStore(storeName, cache); + + stubOneUnderlying.put("a", "a"); + stubOneUnderlying.put("b", "b"); + stubOneUnderlying.put("z", "z"); + + cache.put("c", "c"); + cache.put("d", "d"); + cache.put("x", "x"); + + final List> results = toList(theStore.reverseRange("a", "e")); + assertTrue(results.contains(new KeyValue<>("a", "a"))); + assertTrue(results.contains(new KeyValue<>("b", "b"))); + assertTrue(results.contains(new KeyValue<>("c", "c"))); + assertTrue(results.contains(new KeyValue<>("d", "d"))); + assertEquals(4, results.size()); + } + @Test public void shouldSupportAllAcrossMultipleStores() { final KeyValueStore cache = newStoreInstance(); @@ -229,6 +283,29 @@ public void shouldSupportAllAcrossMultipleStores() { assertEquals(6, results.size()); } + @Test + public void shouldSupportReverseAllAcrossMultipleStores() { + final KeyValueStore cache = newStoreInstance(); + stubProviderTwo.addStore(storeName, cache); + + stubOneUnderlying.put("a", "a"); + stubOneUnderlying.put("b", "b"); + stubOneUnderlying.put("z", "z"); + + cache.put("c", "c"); + cache.put("d", "d"); + cache.put("x", "x"); + + final List> results = toList(theStore.reverseAll()); + assertTrue(results.contains(new KeyValue<>("a", "a"))); + assertTrue(results.contains(new KeyValue<>("b", "b"))); + assertTrue(results.contains(new KeyValue<>("c", "c"))); + assertTrue(results.contains(new KeyValue<>("d", "d"))); + assertTrue(results.contains(new KeyValue<>("x", "x"))); + assertTrue(results.contains(new KeyValue<>("z", "z"))); + assertEquals(6, results.size()); + } + @Test(expected = InvalidStateStoreException.class) public void shouldThrowInvalidStoreExceptionDuringRebalance() { rebalancing().get("anything"); @@ -244,11 +321,21 @@ public void shouldThrowInvalidStoreExceptionOnRangeDuringRebalance() { rebalancing().range("anything", "something"); } + @Test(expected = InvalidStateStoreException.class) + public void shouldThrowInvalidStoreExceptionOnReverseRangeDuringRebalance() { + rebalancing().reverseRange("anything", "something"); + } + @Test(expected = InvalidStateStoreException.class) public void shouldThrowInvalidStoreExceptionOnAllDuringRebalance() { rebalancing().all(); } + @Test(expected = InvalidStateStoreException.class) + public void shouldThrowInvalidStoreExceptionOnReverseAllDuringRebalance() { + rebalancing().reverseAll(); + } + @Test public void shouldGetApproximateEntriesAcrossAllStores() { final KeyValueStore cache = newStoreInstance(); From 17c360c5f9bd1fe727dfab9364dee3cda64fcd6e Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 21 Jul 2020 11:05:33 +0100 Subject: [PATCH 10/24] test backward segments --- .../state/internals/SegmentedBytesStore.java | 23 +++ .../state/internals/KeyValueSegmentsTest.java | 35 +++++ .../internals/TimestampedSegmentsTest.java | 35 +++++ .../kafka/test/SegmentedBytesStoreStub.java | 135 ------------------ 4 files changed, 93 insertions(+), 135 deletions(-) delete mode 100644 streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java index f91e8416b4e52..07082e8ea3631 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java @@ -40,6 +40,14 @@ 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); /** @@ -53,6 +61,15 @@ public interface SegmentedBytesStore extends StateStore { */ 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); /** @@ -63,6 +80,12 @@ public interface SegmentedBytesStore extends StateStore { */ 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(); /** 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 b3d612b345fa0..442723d0c7459 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentsTest.java @@ -194,6 +194,26 @@ public void shouldGetSegmentsWithinTimeRange() { assertEquals(2, segments.get(2).id); } + @Test + public void shouldGetSegmentsWithinBackwardTimeRange() { + updateStreamTimeAndCreateSegment(0); + updateStreamTimeAndCreateSegment(1); + updateStreamTimeAndCreateSegment(2); + updateStreamTimeAndCreateSegment(3); + final long streamTime = updateStreamTimeAndCreateSegment(4); + segments.getOrCreateSegmentIfLive(0, context, streamTime); + segments.getOrCreateSegmentIfLive(1, context, streamTime); + segments.getOrCreateSegmentIfLive(2, context, streamTime); + segments.getOrCreateSegmentIfLive(3, context, streamTime); + segments.getOrCreateSegmentIfLive(4, context, streamTime); + + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); + assertEquals(3, segments.size()); + assertEquals(0, segments.get(2).id); + assertEquals(1, segments.get(1).id); + assertEquals(2, segments.get(0).id); + } + @Test public void shouldGetSegmentsWithinTimeRangeOutOfOrder() { updateStreamTimeAndCreateSegment(4); @@ -209,6 +229,21 @@ public void shouldGetSegmentsWithinTimeRangeOutOfOrder() { assertEquals(2, segments.get(2).id); } + @Test + public void shouldGetSegmentsWithinTimeBackwardRangeOutOfOrder() { + updateStreamTimeAndCreateSegment(4); + updateStreamTimeAndCreateSegment(2); + updateStreamTimeAndCreateSegment(0); + updateStreamTimeAndCreateSegment(1); + updateStreamTimeAndCreateSegment(3); + + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); + assertEquals(3, segments.size()); + assertEquals(2, segments.get(0).id); + assertEquals(1, segments.get(1).id); + assertEquals(0, segments.get(2).id); + } + @Test public void shouldRollSegments() { updateStreamTimeAndCreateSegment(0); 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 e7f32ab9cc35d..a275df74e77e9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentsTest.java @@ -195,6 +195,26 @@ public void shouldGetSegmentsWithinTimeRange() { assertEquals(2, segments.get(2).id); } + @Test + public void shouldGetSegmentsWithinBackwardTimeRange() { + updateStreamTimeAndCreateSegment(0); + updateStreamTimeAndCreateSegment(1); + updateStreamTimeAndCreateSegment(2); + updateStreamTimeAndCreateSegment(3); + final long streamTime = updateStreamTimeAndCreateSegment(4); + segments.getOrCreateSegmentIfLive(0, context, streamTime); + segments.getOrCreateSegmentIfLive(1, context, streamTime); + segments.getOrCreateSegmentIfLive(2, context, streamTime); + segments.getOrCreateSegmentIfLive(3, context, streamTime); + segments.getOrCreateSegmentIfLive(4, context, streamTime); + + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); + assertEquals(3, segments.size()); + assertEquals(0, segments.get(2).id); + assertEquals(1, segments.get(1).id); + assertEquals(2, segments.get(0).id); + } + @Test public void shouldGetSegmentsWithinTimeRangeOutOfOrder() { updateStreamTimeAndCreateSegment(4); @@ -210,6 +230,21 @@ public void shouldGetSegmentsWithinTimeRangeOutOfOrder() { assertEquals(2, segments.get(2).id); } + @Test + public void shouldGetSegmentsWithinBackwardTimeRangeOutOfOrder() { + updateStreamTimeAndCreateSegment(4); + updateStreamTimeAndCreateSegment(2); + updateStreamTimeAndCreateSegment(0); + updateStreamTimeAndCreateSegment(1); + updateStreamTimeAndCreateSegment(3); + + final List segments = this.segments.segments(0, 2 * SEGMENT_INTERVAL, true); + assertEquals(3, segments.size()); + assertEquals(0, segments.get(2).id); + assertEquals(1, segments.get(1).id); + assertEquals(2, segments.get(0).id); + } + @Test public void shouldRollSegments() { updateStreamTimeAndCreateSegment(0); 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 71c090e6eafd4..0000000000000 --- a/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java +++ /dev/null @@ -1,135 +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 backwardFetch(Bytes key, long from, long to) { - return backwardFetch(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 backwardFetch(Bytes keyFrom, Bytes keyTo, long from, long to) { - fetchCalled = true; - return new KeyValueIteratorStub<>(Collections.>emptyIterator()); - } - - @Override - public KeyValueIterator all() { - fetchCalled = true; - return new KeyValueIteratorStub<>(Collections.>emptyIterator()); - } - - @Override - public KeyValueIterator backwardAll() { - 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 KeyValueIterator backwardFetchAll(long from, long to) { - 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; - } -} From 44d8fbaf88b1416b23549bef6fa42f740346726a Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 21 Jul 2020 15:26:05 +0100 Subject: [PATCH 11/24] test window and keyvalue stores --- .../streams/state/ReadOnlyWindowStore.java | 69 ++++++ ...bstractMergedSortedCacheStoreIterator.java | 57 +++-- .../AbstractRocksDBSegmentedBytesStore.java | 18 +- .../state/internals/CachingKeyValueStore.java | 8 +- .../state/internals/CachingSessionStore.java | 8 +- .../state/internals/CachingWindowStore.java | 46 ++-- .../state/internals/InMemoryWindowStore.java | 4 +- ...SortedCacheKeyValueBytesStoreIterator.java | 5 +- ...MergedSortedCacheSessionStoreIterator.java | 5 +- .../MergedSortedCacheWindowStoreIterator.java | 5 +- ...ortedCacheWindowStoreKeyValueIterator.java | 5 +- .../state/internals/SegmentIterator.java | 11 +- .../AbstractWindowBytesStoreTest.java | 196 +++++++++++++---- .../internals/CachingKeyValueStoreTest.java | 39 ++++ .../internals/CachingWindowStoreTest.java | 201 ++++++++++++++++++ ...edCacheKeyValueBytesStoreIteratorTest.java | 13 +- ...dCacheWrappedSessionStoreIteratorTest.java | 2 +- ...edCacheWrappedWindowStoreIteratorTest.java | 6 +- ...rappedWindowStoreKeyValueIteratorTest.java | 3 +- .../state/internals/SegmentIteratorTest.java | 62 +++++- .../apache/kafka/test/StreamsTestUtils.java | 3 +- 21 files changed, 651 insertions(+), 115 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java index 4f89009eefd74..b06a4e7b2eb7a 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 @@ -114,6 +114,42 @@ public interface ReadOnlyWindowStore { */ WindowStoreIterator fetch(K key, Instant from, Instant to) throws IllegalArgumentException; + /** + * Get all the key-value pairs with the given key and the time range from all the existing windows + * in backward order with respect to time (from end to beginning of time). + *

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

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

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

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

+ * This iterator must be closed after use. + * + * @param from the first key in the range + * @param to the last key in the range + * @param fromTime time range start (inclusive) + * @param toTime time range end (inclusive) + * @return an iterator over windowed key-value pairs {@code , value>} + * @throws InvalidStateStoreException if the store is not initialized + * @throws NullPointerException If {@code null} is used for any key. + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + */ default KeyValueIterator, V> backwardFetch(K from, K to, Instant fromTime, Instant toTime) throws IllegalArgumentException { throw new UnsupportedOperationException(); @@ -165,6 +216,13 @@ default KeyValueIterator, V> backwardFetch(K from, K to, Instant fro */ KeyValueIterator, V> all(); + /** + * Gets all the key-value pairs in the existing windows in backward order + * with respect to time (from end to beginning of time). + * + * @return an iterator over windowed key-value pairs {@code , value>} + * @throws InvalidStateStoreException if the store is not initialized + */ default KeyValueIterator, V> backwardAll() { throw new UnsupportedOperationException(); } @@ -194,6 +252,17 @@ default KeyValueIterator, V> backwardAll() { */ KeyValueIterator, V> fetchAll(Instant from, Instant to) throws IllegalArgumentException; + /** + * Gets all the key-value pairs that belong to the windows within in the given time range in backward order + * with respect to time (from end to beginning of time). + * + * @param from the beginning of the time slot from which to search (inclusive) + * @param to the end of the time slot from which to search (inclusive) + * @return an iterator over windowed key-value pairs {@code , value>} + * @throws InvalidStateStoreException if the store is not initialized + * @throws NullPointerException if {@code null} is used for any key + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + */ default KeyValueIterator, V> backwardFetchAll(Instant from, Instant to) throws IllegalArgumentException { throw new UnsupportedOperationException(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java index 16bdbeb8449f4..412b5af4eb3dd 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 @@ -31,11 +31,14 @@ abstract class AbstractMergedSortedCacheStoreIterator implements KeyValueIterator { private final PeekingKeyValueIterator cacheIterator; private final KeyValueIterator storeIterator; + private final boolean reverse; AbstractMergedSortedCacheStoreIterator(final PeekingKeyValueIterator cacheIterator, - final KeyValueIterator storeIterator) { + final KeyValueIterator storeIterator, + final boolean reverse) { this.cacheIterator = cacheIterator; this.storeIterator = storeIterator; + this.reverse = reverse; } abstract int compare(final Bytes cacheKey, final KS storeKey); @@ -87,14 +90,26 @@ public KeyValue next() { } final int comparison = compare(nextCacheKey, nextStoreKey); - if (comparison > 0) { - return nextStoreValue(nextStoreKey); - } else if (comparison < 0) { - return nextCacheValue(nextCacheKey); + if (!reverse) { + if (comparison > 0) { + return nextStoreValue(nextStoreKey); + } else if (comparison < 0) { + return nextCacheValue(nextCacheKey); + } else { + // skip the same keyed element + storeIterator.next(); + return nextCacheValue(nextCacheKey); + } } else { - // skip the same keyed element - storeIterator.next(); - return nextCacheValue(nextCacheKey); + if (comparison < 0) { + return nextStoreValue(nextStoreKey); + } else if (comparison > 0) { + return nextCacheValue(nextCacheKey); + } else { + // skip the same keyed element + storeIterator.next(); + return nextCacheValue(nextCacheKey); + } } } @@ -136,14 +151,26 @@ public K peekNextKey() { } final int comparison = compare(nextCacheKey, nextStoreKey); - if (comparison > 0) { - return deserializeStoreKey(nextStoreKey); - } else if (comparison < 0) { - return deserializeCacheKey(nextCacheKey); + if (!reverse) { + if (comparison > 0) { + return deserializeStoreKey(nextStoreKey); + } else if (comparison < 0) { + return deserializeCacheKey(nextCacheKey); + } else { + // skip the same keyed element + storeIterator.next(); + return deserializeCacheKey(nextCacheKey); + } } else { - // skip the same keyed element - storeIterator.next(); - return deserializeCacheKey(nextCacheKey); + if (comparison < 0) { + return deserializeStoreKey(nextStoreKey); + } else if (comparison > 0) { + return deserializeCacheKey(nextCacheKey); + } else { + // skip the same keyed element + storeIterator.next(); + return deserializeCacheKey(nextCacheKey); + } } } 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 06e5b955f22c6..3bfdb93e70170 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 @@ -86,7 +86,8 @@ KeyValueIterator fetch(final Bytes key, searchSpace.iterator(), keySchema.hasNextCondition(key, key, from, to), binaryFrom, - binaryTo); + binaryTo, + backward); } @Override @@ -126,7 +127,8 @@ KeyValueIterator fetch(final Bytes keyFrom, searchSpace.iterator(), keySchema.hasNextCondition(keyFrom, keyTo, from, to), binaryFrom, - binaryTo); + binaryTo, + backward); } @Override @@ -137,7 +139,8 @@ public KeyValueIterator all() { searchSpace.iterator(), keySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE), null, - null); + null, + false); } @Override @@ -148,7 +151,8 @@ public KeyValueIterator backwardAll() { searchSpace.iterator(), keySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE), null, - null); + null, + true); } @Override @@ -160,7 +164,8 @@ public KeyValueIterator fetchAll(final long timeFrom, searchSpace.iterator(), keySchema.hasNextCondition(null, null, timeFrom, timeTo), null, - null); + null, + false); } @Override @@ -172,7 +177,8 @@ public KeyValueIterator backwardFetchAll(final long timeFrom, searchSpace.iterator(), keySchema.hasNextCondition(null, null, timeFrom, timeTo), null, - null); + null, + true); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java index 15536d7acaa6d..7c581c91e6ca4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java @@ -249,7 +249,7 @@ public KeyValueIterator range(final Bytes from, validateStoreOpen(); final KeyValueIterator storeIterator = wrapped().range(from, to); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = context.cache().range(cacheName, from, to); - return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator); + return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator, false); } @Override @@ -265,7 +265,7 @@ public KeyValueIterator reverseRange(final Bytes from, validateStoreOpen(); final KeyValueIterator storeIterator = wrapped().reverseRange(from, to); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = context.cache().reverseRange(cacheName, from, to); - return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator); + return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator, true); } @Override @@ -274,7 +274,7 @@ public KeyValueIterator all() { final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>(this.name(), wrapped().all()); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = context.cache().all(cacheName); - return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator); + return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator, false); } @Override @@ -283,7 +283,7 @@ public KeyValueIterator reverseAll() { final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>(this.name(), wrapped().reverseAll()); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = context.cache().reverseAll(cacheName); - return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator); + return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator, true); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java index 3bf624ee3ebdd..097fe72f69b05 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java @@ -164,7 +164,7 @@ public KeyValueIterator, byte[]> findSessions(final Bytes key, latestSessionStartTime); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); - return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction); + return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction, false); } @Override @@ -189,7 +189,7 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte latestSessionStartTime); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); - return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction); + return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction, true); } @Override @@ -219,7 +219,7 @@ public KeyValueIterator, byte[]> findSessions(final Bytes keyFro latestSessionStartTime); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); - return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction); + return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction, false); } @Override @@ -249,7 +249,7 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte latestSessionStartTime); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); - return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction); + return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction, true); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index e0c837dff0a5c..314fcbe3257f8 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 @@ -205,7 +205,7 @@ public synchronized WindowStoreIterator fetch(final Bytes key, } final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? - new CacheIteratorWrapper(key, timeFrom, timeTo) : + new CacheIteratorWrapper(key, timeFrom, timeTo, false) : context.cache().range(name, cacheFunction.cacheKey(keySchema.lowerRangeFixedSize(key, timeFrom)), cacheFunction.cacheKey(keySchema.upperRangeFixedSize(key, timeTo)) @@ -216,7 +216,7 @@ public synchronized WindowStoreIterator fetch(final Bytes key, cacheIterator, hasNextCondition, cacheFunction ); - return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator); + return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator, false); } @Override @@ -235,18 +235,17 @@ public synchronized WindowStoreIterator backwardFetch(final Bytes key, } final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? - new CacheIteratorWrapper(key, timeFrom, timeTo) : + new CacheIteratorWrapper(key, timeFrom, timeTo, true) : context.cache().reverseRange(name, 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); + return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator, true); } @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @@ -273,7 +272,7 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, } final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? - new CacheIteratorWrapper(from, to, timeFrom, timeTo) : + new CacheIteratorWrapper(from, to, timeFrom, timeTo, false) : context.cache().range(name, cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), cacheFunction.cacheKey(keySchema.upperRange(to, timeTo)) @@ -287,7 +286,8 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, underlyingIterator, bytesSerdes, windowSize, - cacheFunction + cacheFunction, + false ); } @@ -317,7 +317,7 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes from, } final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? - new CacheIteratorWrapper(from, to, timeFrom, timeTo) : + new CacheIteratorWrapper(from, to, timeFrom, timeTo, true) : context.cache().reverseRange(name, cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), cacheFunction.cacheKey(keySchema.upperRange(to, timeTo)) @@ -331,7 +331,8 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes from, underlyingIterator, bytesSerdes, windowSize, - cacheFunction + cacheFunction, + true ); } @@ -352,7 +353,8 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, underlyingIterator, bytesSerdes, windowSize, - cacheFunction + cacheFunction, + false ); } @@ -375,7 +377,8 @@ public KeyValueIterator, byte[]> backwardFetchAll(final Instant underlyingIterator, bytesSerdes, windowSize, - cacheFunction + cacheFunction, + true ); } @@ -391,7 +394,8 @@ public KeyValueIterator, byte[]> all() { underlyingIterator, bytesSerdes, windowSize, - cacheFunction + cacheFunction, + false ); } @@ -407,7 +411,8 @@ public KeyValueIterator, byte[]> backwardAll() { underlyingIterator, bytesSerdes, windowSize, - cacheFunction + cacheFunction, + true ); } @@ -448,14 +453,16 @@ private class CacheIteratorWrapper implements PeekingKeyValueIterator, byte[]> backwardFetch(final Bytes from, final Instant toTime) { final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")); final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime")); - return fetch(from, to, timeFrom, timeTo, false); + return fetch(from, to, timeFrom, timeTo, true); } KeyValueIterator, byte[]> fetch(final Bytes from, @@ -253,7 +253,7 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, f public KeyValueIterator, byte[]> backwardFetchAll(final Instant from, final Instant to) { final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); - return fetchAll(timeFrom, timeTo, false); + return fetchAll(timeFrom, timeTo, true); } KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo, final boolean backward) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIterator.java index 7a545e4da3f9b..54174aab33adc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIterator.java @@ -28,8 +28,9 @@ class MergedSortedCacheKeyValueBytesStoreIterator extends AbstractMergedSortedCa MergedSortedCacheKeyValueBytesStoreIterator(final PeekingKeyValueIterator cacheIterator, - final KeyValueIterator storeIterator) { - super(cacheIterator, storeIterator); + final KeyValueIterator storeIterator, + final boolean reverse) { + super(cacheIterator, storeIterator, reverse); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java index 6994222531067..e1abcba77747a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java @@ -32,8 +32,9 @@ class MergedSortedCacheSessionStoreIterator extends AbstractMergedSortedCacheSto MergedSortedCacheSessionStoreIterator(final PeekingKeyValueIterator cacheIterator, final KeyValueIterator, byte[]> storeIterator, - final SegmentedCacheFunction cacheFunction) { - super(cacheIterator, storeIterator); + final SegmentedCacheFunction cacheFunction, + final boolean reverse) { + super(cacheIterator, storeIterator, reverse); this.cacheFunction = cacheFunction; } 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 98b3f064db178..255e6552c3966 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); + final KeyValueIterator storeIterator, + final boolean reverse) { + super(cacheIterator, storeIterator, reverse); } @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 1cba018c285d6..4ca59601da2f0 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 reverse ) { - super(filteredCacheIterator, underlyingIterator); + super(filteredCacheIterator, underlyingIterator, reverse); this.serdes = serdes; this.windowSize = windowSize; this.cacheFunction = cacheFunction; 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..5d0cb3bb7437e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java @@ -31,6 +31,7 @@ class SegmentIterator implements KeyValueIterator segments; protected final HasNextCondition hasNextCondition; @@ -40,11 +41,13 @@ class SegmentIterator implements KeyValueIterator segments, final HasNextCondition hasNextCondition, final Bytes from, - final Bytes to) { + final Bytes to, + final boolean reverse) { this.segments = segments; this.hasNextCondition = hasNextCondition; this.from = from; this.to = to; + this.reverse = reverse; } @Override @@ -72,9 +75,11 @@ public boolean hasNext() { currentSegment = segments.next(); try { if (from == null || to == null) { - currentIterator = currentSegment.all(); + if (reverse) currentIterator = currentSegment.reverseAll(); + else currentIterator = currentSegment.all(); } else { - currentIterator = currentSegment.range(from, to); + if (reverse) currentIterator = currentSegment.reverseRange(from, to); + else currentIterator = currentSegment.range(from, to); } } catch (final InvalidStateStoreException e) { // segment may have been closed so we ignore it. diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java index 2211bd3dc200d..aefd1ed380e99 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java @@ -50,6 +50,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Properties; @@ -64,6 +65,7 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; @@ -269,9 +271,27 @@ public void shouldGetAll() { final KeyValue, String> four = windowedPair(4, "four", startTime + 4); final KeyValue, String> five = windowedPair(5, "five", startTime + 5); - assertEquals( - new HashSet<>(asList(zero, one, two, four, five)), - toSet(windowStore.all()) + assertArrayEquals( + new LinkedHashSet<>(asList(zero, one, two, four, five)).toArray(), + toSet(windowStore.all()).toArray() + ); + } + + @Test + public void shouldGetBackwardAll() { + final long startTime = SEGMENT_INTERVAL - 4L; + + putFirstBatch(windowStore, startTime, context); + + final KeyValue, String> zero = windowedPair(0, "zero", startTime + 0); + final KeyValue, String> one = windowedPair(1, "one", startTime + 1); + final KeyValue, String> two = windowedPair(2, "two", startTime + 2); + final KeyValue, String> four = windowedPair(4, "four", startTime + 4); + final KeyValue, String> five = windowedPair(5, "five", startTime + 5); + + assertArrayEquals( + new LinkedHashSet<>(asList(five, four, two, one, zero)).toArray(), + toSet(windowStore.backwardAll()).toArray() ); } @@ -287,17 +307,43 @@ public void shouldFetchAllInTimeRange() { final KeyValue, String> four = windowedPair(4, "four", startTime + 4); final KeyValue, String> five = windowedPair(5, "five", startTime + 5); - assertEquals( - new HashSet<>(asList(one, two, four)), - toSet(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))) + assertArrayEquals( + new LinkedHashSet<>(asList(one, two, four)).toArray(), + toSet(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))).toArray() ); - assertEquals( - new HashSet<>(asList(zero, one, two)), - toSet(windowStore.fetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))) + assertArrayEquals( + new LinkedHashSet<>(asList(zero, one, two)).toArray(), + toSet(windowStore.fetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))).toArray() ); - assertEquals( - new HashSet<>(asList(one, two, four, five)), - toSet(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))) + assertArrayEquals( + new LinkedHashSet<>(asList(one, two, four, five)).toArray(), + toSet(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))).toArray() + ); + } + + @Test + public void shouldBackwardFetchAllInTimeRange() { + final long startTime = SEGMENT_INTERVAL - 4L; + + putFirstBatch(windowStore, startTime, context); + + final KeyValue, String> zero = windowedPair(0, "zero", startTime + 0); + final KeyValue, String> one = windowedPair(1, "one", startTime + 1); + final KeyValue, String> two = windowedPair(2, "two", startTime + 2); + final KeyValue, String> four = windowedPair(4, "four", startTime + 4); + final KeyValue, String> five = windowedPair(5, "five", startTime + 5); + + assertArrayEquals( + new LinkedHashSet<>(asList(four, two, one)).toArray(), + toSet(windowStore.backwardFetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(asList(two, one, zero)).toArray(), + toSet(windowStore.backwardFetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(asList(five, four, two, one)).toArray(), + toSet(windowStore.backwardFetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))).toArray() ); } @@ -313,69 +359,147 @@ public void testFetchRange() { final KeyValue, String> four = windowedPair(4, "four", startTime + 4); final KeyValue, String> five = windowedPair(5, "five", startTime + 5); - assertEquals( - new HashSet<>(asList(zero, one)), + assertArrayEquals( + new LinkedHashSet<>(asList(zero, one)).toArray(), toSet(windowStore.fetch( 0, 1, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))) + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() ); - assertEquals( - new HashSet<>(Collections.singletonList(one)), + assertArrayEquals( + new LinkedHashSet<>(Collections.singletonList(one)).toArray(), toSet(windowStore.fetch( 1, 1, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))) + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() ); - assertEquals( - new HashSet<>(asList(one, two)), + assertArrayEquals( + new LinkedHashSet<>(asList(one, two)).toArray(), toSet(windowStore.fetch( 1, 3, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))) + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() ); - assertEquals( - new HashSet<>(asList(zero, one, two)), + assertArrayEquals( + new LinkedHashSet<>(asList(zero, one, two)).toArray(), toSet(windowStore.fetch( 0, 5, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE))) + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() ); - assertEquals( - new HashSet<>(asList(zero, one, two, four, five)), + assertArrayEquals( + new LinkedHashSet<>(asList(zero, one, two, four, five)).toArray(), toSet(windowStore.fetch( 0, 5, ofEpochMilli(startTime + 0L - WINDOW_SIZE), - ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))) + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))).toArray() ); - assertEquals( - new HashSet<>(asList(two, four, five)), + assertArrayEquals( + new LinkedHashSet<>(asList(two, four, five)).toArray(), toSet(windowStore.fetch( 0, 5, ofEpochMilli(startTime + 2L), - ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))) + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))).toArray() ); - assertEquals( - new HashSet<>(Collections.emptyList()), + assertArrayEquals( + new LinkedHashSet<>(Collections.emptyList()).toArray(), toSet(windowStore.fetch( 4, 5, ofEpochMilli(startTime + 2L), - ofEpochMilli(startTime + WINDOW_SIZE))) + ofEpochMilli(startTime + WINDOW_SIZE))).toArray() ); - assertEquals( - new HashSet<>(Collections.emptyList()), + assertArrayEquals( + new LinkedHashSet<>(Collections.emptyList()).toArray(), toSet(windowStore.fetch( 0, 3, ofEpochMilli(startTime + 3L), - ofEpochMilli(startTime + WINDOW_SIZE + 5))) + ofEpochMilli(startTime + WINDOW_SIZE + 5))).toArray() + ); + } + + @Test + public void testBackwardFetchRange() { + final long startTime = SEGMENT_INTERVAL - 4L; + + putFirstBatch(windowStore, startTime, context); + + final KeyValue, String> zero = windowedPair(0, "zero", startTime + 0); + final KeyValue, String> one = windowedPair(1, "one", startTime + 1); + final KeyValue, String> two = windowedPair(2, "two", startTime + 2); + final KeyValue, String> four = windowedPair(4, "four", startTime + 4); + final KeyValue, String> five = windowedPair(5, "five", startTime + 5); + + assertArrayEquals( + new LinkedHashSet<>(asList(one, zero)).toArray(), + toSet(windowStore.backwardFetch( + 0, + 1, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(Collections.singletonList(one)).toArray(), + toSet(windowStore.backwardFetch( + 1, + 1, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(asList(two, one)).toArray(), + toSet(windowStore.backwardFetch( + 1, + 3, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(asList(two, one, zero)).toArray(), + toSet(windowStore.backwardFetch( + 0, + 5, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(asList(five, four, two, one, zero)).toArray(), + toSet(windowStore.backwardFetch( + 0, + 5, + ofEpochMilli(startTime + 0L - WINDOW_SIZE), + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(asList(five, four, two)).toArray(), + toSet(windowStore.backwardFetch( + 0, + 5, + ofEpochMilli(startTime + 2L), + ofEpochMilli(startTime + 0L + WINDOW_SIZE + 5L))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(Collections.emptyList()).toArray(), + toSet(windowStore.backwardFetch( + 4, + 5, + ofEpochMilli(startTime + 2L), + ofEpochMilli(startTime + WINDOW_SIZE))).toArray() + ); + assertArrayEquals( + new LinkedHashSet<>(Collections.emptyList()).toArray(), + toSet(windowStore.backwardFetch( + 0, + 3, + ofEpochMilli(startTime + 3L), + ofEpochMilli(startTime + WINDOW_SIZE + 5))).toArray() ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java index ffc8134a43a6c..9ce0d20a5d018 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java @@ -282,6 +282,17 @@ public void shouldIterateAllStoredItems() { assertEquals(items, results.size()); } + @Test + public void shouldReverseIterateAllStoredItems() { + final int items = addItemsToCache(); + final KeyValueIterator all = store.reverseAll(); + final List results = new ArrayList<>(); + while (all.hasNext()) { + results.add(all.next().key); + } + assertEquals(items, results.size()); + } + @Test public void shouldIterateOverRange() { final int items = addItemsToCache(); @@ -294,13 +305,27 @@ public void shouldIterateOverRange() { assertEquals(items, results.size()); } + @Test + public void shouldReverseIterateOverRange() { + final int items = addItemsToCache(); + final KeyValueIterator range = + store.reverseRange(bytesKey(String.valueOf(0)), bytesKey(String.valueOf(items))); + final List results = new ArrayList<>(); + while (range.hasNext()) { + results.add(range.next().key); + } + assertEquals(items, results.size()); + } + @Test public void shouldDeleteItemsFromCache() { store.put(bytesKey("a"), bytesValue("a")); store.delete(bytesKey("a")); assertNull(store.get(bytesKey("a"))); assertFalse(store.range(bytesKey("a"), bytesKey("b")).hasNext()); + assertFalse(store.reverseRange(bytesKey("a"), bytesKey("b")).hasNext()); assertFalse(store.all().hasNext()); + assertFalse(store.reverseAll().hasNext()); } @Test @@ -310,7 +335,9 @@ public void shouldNotShowItemsDeletedFromCacheButFlushedToStoreBeforeDelete() { store.delete(bytesKey("a")); assertNull(store.get(bytesKey("a"))); assertFalse(store.range(bytesKey("a"), bytesKey("b")).hasNext()); + assertFalse(store.reverseRange(bytesKey("a"), bytesKey("b")).hasNext()); assertFalse(store.all().hasNext()); + assertFalse(store.reverseAll().hasNext()); } @Test @@ -339,12 +366,24 @@ public void shouldThrowIfTryingToDoRangeQueryOnClosedCachingStore() { store.range(bytesKey("a"), bytesKey("b")); } + @Test(expected = InvalidStateStoreException.class) + public void shouldThrowIfTryingToDoReverseRangeQueryOnClosedCachingStore() { + store.close(); + store.reverseRange(bytesKey("a"), bytesKey("b")); + } + @Test(expected = InvalidStateStoreException.class) public void shouldThrowIfTryingToDoAllQueryOnClosedCachingStore() { store.close(); store.all(); } + @Test(expected = InvalidStateStoreException.class) + public void shouldThrowIfTryingToDoReverseAllQueryOnClosedCachingStore() { + store.close(); + store.reverseAll(); + } + @Test(expected = InvalidStateStoreException.class) public void shouldThrowIfTryingToDoGetApproxSizeOnClosedCachingStore() { store.close(); 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 097ebe9a58953..61878b5716784 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 @@ -293,6 +293,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() { @@ -336,6 +359,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(); @@ -454,6 +520,18 @@ public void shouldIterateAcrossWindows() { assertFalse(fetch.hasNext()); } + @Test + public void shouldIterateAcrossWindowsBackward() { + 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()); @@ -466,6 +544,18 @@ public void shouldIterateCacheAndStore() { assertFalse(fetch.hasNext()); } + @Test + public void shouldIterateCacheBackwardAndStore() { + 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()); @@ -485,6 +575,25 @@ public void shouldIterateCacheAndStoreKeyRange() { assertFalse(fetchRange.hasNext()); } + @Test + public void shouldIterateCacheAndStoreKeyRangeBackward() { + 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() { @@ -531,6 +640,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); @@ -567,6 +694,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); @@ -583,6 +746,24 @@ public void shouldReturnSameResultsForSingleKeyFetchAndEqualKeyRangeFetch() { assertFalse(keyRangeIterator.hasNext()); } + @Test + public void shouldReturnSameResultsForSingleKeyFetchAndEqualKeyRangeBackwardFetch() { + cachingStore.put(bytesKey("a"), bytesValue("0001"), 0); + cachingStore.put(bytesKey("aa"), bytesValue("0002"), 1); + cachingStore.put(bytesKey("aa"), bytesValue("0003"), 2); + cachingStore.put(bytesKey("aaa"), bytesValue("0004"), 3); + + final WindowStoreIterator singleKeyIterator = + cachingStore.backwardFetch(bytesKey("aa"), Instant.ofEpochMilli(0L), Instant.ofEpochMilli(5L)); + final KeyValueIterator, byte[]> keyRangeIterator = + cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), Instant.ofEpochMilli(0L), Instant.ofEpochMilli(5L)); + + assertEquals(stringFrom(singleKeyIterator.next().value), stringFrom(keyRangeIterator.next().value)); + assertEquals(stringFrom(singleKeyIterator.next().value), stringFrom(keyRangeIterator.next().value)); + assertFalse(singleKeyIterator.hasNext()); + assertFalse(keyRangeIterator.hasNext()); + } + @Test(expected = NullPointerException.class) @SuppressWarnings("deprecation") public void shouldThrowNullPointerExceptionOnPutNullKey() { @@ -629,6 +810,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/MergedSortedCacheKeyValueBytesStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIteratorTest.java index 4028b0cd901f3..1226148538132 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIteratorTest.java @@ -51,12 +51,13 @@ public void shouldIterateOverRange() throws Exception { cache.put(namespace, Bytes.wrap(bytes[i + 1]), new LRUCacheEntry(bytes[i + 1])); } - final Bytes from = Bytes.wrap(new byte[]{2}); - final Bytes to = Bytes.wrap(new byte[]{9}); + final Bytes from = Bytes.wrap(new byte[] {2}); + final Bytes to = Bytes.wrap(new byte[] {9}); final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("store", store.range(from, to)); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(namespace, from, to); - final MergedSortedCacheKeyValueBytesStoreIterator iterator = new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator); + final MergedSortedCacheKeyValueBytesStoreIterator iterator = + new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator, false); final byte[][] values = new byte[8][]; int index = 0; int bytesIndex = 2; @@ -160,9 +161,7 @@ public void shouldPeekNextKey() throws Exception { final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(namespace, from, to); final MergedSortedCacheKeyValueBytesStoreIterator iterator = - new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, - storeIterator - ); + new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator, false); final byte[][] values = new byte[8][]; int index = 0; int bytesIndex = 2; @@ -178,6 +177,6 @@ public void shouldPeekNextKey() throws Exception { private MergedSortedCacheKeyValueBytesStoreIterator createIterator() { final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.all(namespace); final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("store", store.all()); - return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator); + return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator, false); } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java index 617ff362b694a..1909da770d83d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java @@ -104,7 +104,7 @@ private MergedSortedCacheSessionStoreIterator createIterator(final Iterator cacheIterator = new DelegatingPeekingKeyValueIterator<>("cache", new KeyValueIteratorStub<>(cacheKvs)); - return new MergedSortedCacheSessionStoreIterator(cacheIterator, storeIterator, SINGLE_SEGMENT_CACHE_FUNCTION); + return new MergedSortedCacheSessionStoreIterator(cacheIterator, storeIterator, SINGLE_SEGMENT_CACHE_FUNCTION, false); } } 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..6f12ea40d2ecc 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 @@ -72,7 +72,7 @@ public void shouldIterateOverValueFromBothIterators() { ); final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( - cacheIterator, storeIterator + cacheIterator, storeIterator, false ); int index = 0; while (iterator.hasNext()) { @@ -95,7 +95,7 @@ public void shouldPeekNextStoreKey() { namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes) ); final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( - cacheIterator, storeIterator + cacheIterator, storeIterator, false ); assertThat(iterator.peekNextKey(), equalTo(0L)); iterator.next(); @@ -111,7 +111,7 @@ public void shouldPeekNextCacheKey() { final Bytes toBytes = WindowKeySchema.toStoreKeyBinary("a", 100, 0, stateSerdes); final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes)); - final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator(cacheIterator, storeIterator); + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator(cacheIterator, storeIterator, false); assertThat(iterator.peekNextKey(), equalTo(0L)); iterator.next(); assertThat(iterator.peekNextKey(), equalTo(10L)); 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..f5e34c3c95954 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 @@ -135,7 +135,8 @@ private MergedSortedCacheWindowStoreKeyValueIterator createIterator( storeIterator, new StateSerdes<>("name", Serdes.Bytes(), Serdes.ByteArray()), WINDOW_SIZE, - SINGLE_SEGMENT_CACHE_FUNCTION + SINGLE_SEGMENT_CACHE_FUNCTION, + false ); } } 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..ab16b4eb76e00 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 @@ -86,7 +86,7 @@ public void shouldIterateOverAllSegments() { Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("a".getBytes()), - Bytes.wrap("z".getBytes())); + Bytes.wrap("z".getBytes()), false); assertTrue(iterator.hasNext()); assertEquals("a", new String(iterator.peekNextKey().get())); @@ -107,26 +107,76 @@ public void shouldIterateOverAllSegments() { assertFalse(iterator.hasNext()); } + @Test + public void shouldIterateBackwardOverAllSegments() { + iterator = new SegmentIterator<>( + Arrays.asList(segmentOne, segmentTwo).iterator(), + hasNextCondition, + Bytes.wrap("a".getBytes()), + Bytes.wrap("z".getBytes()), + true); + + assertTrue(iterator.hasNext()); + assertEquals("b", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("b", "2"), toStringKeyValue(iterator.next())); + + assertTrue(iterator.hasNext()); + assertEquals("a", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("a", "1"), toStringKeyValue(iterator.next())); + + assertTrue(iterator.hasNext()); + assertEquals("d", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("d", "4"), toStringKeyValue(iterator.next())); + + assertTrue(iterator.hasNext()); + assertEquals("c", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("c", "3"), toStringKeyValue(iterator.next())); + + assertFalse(iterator.hasNext()); + } + @Test public void shouldNotThrowExceptionOnHasNextWhenStoreClosed() { iterator = new SegmentIterator<>( Collections.singletonList(segmentOne).iterator(), hasNextCondition, Bytes.wrap("a".getBytes()), - Bytes.wrap("z".getBytes())); + Bytes.wrap("z".getBytes()), + false); iterator.currentIterator = segmentOne.all(); segmentOne.close(); assertFalse(iterator.hasNext()); } + @Test + public void shouldOnlyIterateOverSegmentsInBackwardRange() { + iterator = new SegmentIterator<>( + Arrays.asList(segmentOne, segmentTwo).iterator(), + hasNextCondition, + Bytes.wrap("a".getBytes()), + Bytes.wrap("b".getBytes()), + true); + + assertTrue(iterator.hasNext()); + assertEquals("b", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("b", "2"), toStringKeyValue(iterator.next())); + + assertTrue(iterator.hasNext()); + assertEquals("a", new String(iterator.peekNextKey().get())); + assertEquals(KeyValue.pair("a", "1"), toStringKeyValue(iterator.next())); + + assertFalse(iterator.hasNext()); + } + @Test public void shouldOnlyIterateOverSegmentsInRange() { iterator = new SegmentIterator<>( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("a".getBytes()), - Bytes.wrap("b".getBytes())); + Bytes.wrap("b".getBytes()), + false); assertTrue(iterator.hasNext()); assertEquals("a", new String(iterator.peekNextKey().get())); @@ -145,7 +195,8 @@ public void shouldThrowNoSuchElementOnPeekNextKeyIfNoNext() { Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("f".getBytes()), - Bytes.wrap("h".getBytes())); + Bytes.wrap("h".getBytes()), + false); iterator.peekNextKey(); } @@ -156,7 +207,8 @@ public void shouldThrowNoSuchElementOnNextIfNoNext() { Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, Bytes.wrap("f".getBytes()), - Bytes.wrap("h".getBytes())); + Bytes.wrap("h".getBytes()), + false); iterator.next(); } diff --git a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java index afc378541c118..22fa79fdda744 100644 --- a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java @@ -30,6 +30,7 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Properties; @@ -121,7 +122,7 @@ public static List> toList(final Iterator> } public static Set> toSet(final Iterator> iterator) { - final Set> results = new HashSet<>(); + final Set> results = new LinkedHashSet<>(); while (iterator.hasNext()) { results.add(iterator.next()); From 2f881fc61ed97ba5a19780dbada0cf98cb0f8556 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 21 Jul 2020 15:33:22 +0100 Subject: [PATCH 12/24] reformat --- .../state/internals/CachingSessionStore.java | 17 ++++++----------- .../state/internals/CachingWindowStore.java | 12 ++++++++---- 2 files changed, 14 insertions(+), 15 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java index 097fe72f69b05..5a91b5c397fde 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java @@ -155,13 +155,10 @@ public KeyValueIterator, byte[]> findSessions(final Bytes key, cacheFunction.cacheKey(keySchema.upperRangeFixedSize(key, latestSessionStartTime)) ); - final KeyValueIterator, byte[]> storeIterator = wrapped().findSessions(key, - earliestSessionEndTime, - latestSessionStartTime); - final HasNextCondition hasNextCondition = keySchema.hasNextCondition(key, - key, - earliestSessionEndTime, - latestSessionStartTime); + final KeyValueIterator, byte[]> storeIterator = + wrapped().findSessions(key, earliestSessionEndTime, latestSessionStartTime); + final HasNextCondition hasNextCondition = + keySchema.hasNextCondition(key, key, earliestSessionEndTime, latestSessionStartTime); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction, false); @@ -213,10 +210,8 @@ public KeyValueIterator, byte[]> findSessions(final Bytes keyFro final KeyValueIterator, byte[]> storeIterator = wrapped().findSessions( keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime ); - final HasNextCondition hasNextCondition = keySchema.hasNextCondition(keyFrom, - keyTo, - earliestSessionEndTime, - latestSessionStartTime); + final HasNextCondition hasNextCondition = + keySchema.hasNextCondition(keyFrom, keyTo, earliestSessionEndTime, latestSessionStartTime); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); return new MergedSortedCacheSessionStoreIterator(filteredCacheIterator, storeIterator, cacheFunction, false); 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 314fcbe3257f8..8bf5cf56cbd52 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 @@ -206,7 +206,8 @@ public synchronized WindowStoreIterator fetch(final Bytes key, final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? new CacheIteratorWrapper(key, timeFrom, timeTo, false) : - context.cache().range(name, + context.cache().range( + name, cacheFunction.cacheKey(keySchema.lowerRangeFixedSize(key, timeFrom)), cacheFunction.cacheKey(keySchema.upperRangeFixedSize(key, timeTo)) ); @@ -236,7 +237,8 @@ public synchronized WindowStoreIterator backwardFetch(final Bytes key, final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? new CacheIteratorWrapper(key, timeFrom, timeTo, true) : - context.cache().reverseRange(name, + context.cache().reverseRange( + name, cacheFunction.cacheKey(keySchema.lowerRangeFixedSize(key, timeFrom)), cacheFunction.cacheKey(keySchema.upperRangeFixedSize(key, timeTo)) ); @@ -273,7 +275,8 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? new CacheIteratorWrapper(from, to, timeFrom, timeTo, false) : - context.cache().range(name, + context.cache().range( + name, cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), cacheFunction.cacheKey(keySchema.upperRange(to, timeTo)) ); @@ -318,7 +321,8 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final PeekingKeyValueIterator cacheIterator = wrapped().persistent() ? new CacheIteratorWrapper(from, to, timeFrom, timeTo, true) : - context.cache().reverseRange(name, + context.cache().reverseRange( + name, cacheFunction.cacheKey(keySchema.lowerRange(from, timeFrom)), cacheFunction.cacheKey(keySchema.upperRange(to, timeTo)) ); From 27db8f8707e559479f9923d92e54dda30a80b172 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 21 Jul 2020 15:43:35 +0100 Subject: [PATCH 13/24] test merged kv iterators --- ...rappedWindowStoreKeyValueIteratorTest.java | 69 ++++++++++++++++--- 1 file changed, 60 insertions(+), 9 deletions(-) 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 f5e34c3c95954..a160df8e2e6e3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreKeyValueIteratorTest.java @@ -63,53 +63,103 @@ public long segmentId(final Bytes key) { @Test public void shouldHaveNextFromStore() { final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = - createIterator(storeKvs, Collections.emptyIterator()); + createIterator(storeKvs, Collections.emptyIterator(), false); + assertTrue(mergeIterator.hasNext()); + } + + @Test + public void shouldHaveNextFromReverseStore() { + final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = + createIterator(storeKvs, Collections.emptyIterator(), true); assertTrue(mergeIterator.hasNext()); } @Test public void shouldGetNextFromStore() { final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = - createIterator(storeKvs, Collections.emptyIterator()); + createIterator(storeKvs, Collections.emptyIterator(), false); + assertThat(convertKeyValuePair(mergeIterator.next()), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); + } + + @Test + public void shouldGetNextFromReverseStore() { + final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = + createIterator(storeKvs, Collections.emptyIterator(), true); assertThat(convertKeyValuePair(mergeIterator.next()), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); } @Test public void shouldPeekNextKeyFromStore() { final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = - createIterator(storeKvs, Collections.emptyIterator()); + createIterator(storeKvs, Collections.emptyIterator(), false); + assertThat(convertWindowedKey(mergeIterator.peekNextKey()), equalTo(new Windowed<>(storeKey, storeWindow))); + } + + @Test + public void shouldPeekNextKeyFromReverseStore() { + final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = + createIterator(storeKvs, Collections.emptyIterator(), true); assertThat(convertWindowedKey(mergeIterator.peekNextKey()), equalTo(new Windowed<>(storeKey, storeWindow))); } @Test public void shouldHaveNextFromCache() { final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = - createIterator(Collections.emptyIterator(), cacheKvs); + createIterator(Collections.emptyIterator(), cacheKvs, false); + assertTrue(mergeIterator.hasNext()); + } + + @Test + public void shouldHaveNextFromReverseCache() { + final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = + createIterator(Collections.emptyIterator(), cacheKvs, true); assertTrue(mergeIterator.hasNext()); } @Test public void shouldGetNextFromCache() { final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = - createIterator(Collections.emptyIterator(), cacheKvs); + createIterator(Collections.emptyIterator(), cacheKvs, false); + assertThat(convertKeyValuePair(mergeIterator.next()), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); + } + + @Test + public void shouldGetNextFromReverseCache() { + final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = + createIterator(Collections.emptyIterator(), cacheKvs, true); assertThat(convertKeyValuePair(mergeIterator.next()), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); } @Test public void shouldPeekNextKeyFromCache() { final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = - createIterator(Collections.emptyIterator(), cacheKvs); + createIterator(Collections.emptyIterator(), cacheKvs, false); + assertThat(convertWindowedKey(mergeIterator.peekNextKey()), equalTo(new Windowed<>(cacheKey, cacheWindow))); + } + + @Test + public void shouldPeekNextKeyFromReverseCache() { + final MergedSortedCacheWindowStoreKeyValueIterator mergeIterator = + createIterator(Collections.emptyIterator(), cacheKvs, true); assertThat(convertWindowedKey(mergeIterator.peekNextKey()), equalTo(new Windowed<>(cacheKey, cacheWindow))); } @Test public void shouldIterateBothStoreAndCache() { - final MergedSortedCacheWindowStoreKeyValueIterator iterator = createIterator(storeKvs, cacheKvs); + final MergedSortedCacheWindowStoreKeyValueIterator iterator = createIterator(storeKvs, cacheKvs, false); assertThat(convertKeyValuePair(iterator.next()), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); assertThat(convertKeyValuePair(iterator.next()), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); assertFalse(iterator.hasNext()); } + @Test + public void shouldReverseIterateBothStoreAndCache() { + final MergedSortedCacheWindowStoreKeyValueIterator iterator = createIterator(storeKvs, cacheKvs, true); + assertThat(convertKeyValuePair(iterator.next()), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); + assertThat(convertKeyValuePair(iterator.next()), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); + assertFalse(iterator.hasNext()); + } + private KeyValue, String> convertKeyValuePair(final KeyValue, byte[]> next) { final String value = deserializer.deserialize("", next.value); return KeyValue.pair(convertWindowedKey(next.key), value); @@ -123,7 +173,8 @@ private Windowed convertWindowedKey(final Windowed bytesWindowed) private MergedSortedCacheWindowStoreKeyValueIterator createIterator( final Iterator, byte[]>> storeKvs, - final Iterator> cacheKvs + final Iterator> cacheKvs, + final boolean reverse ) { final DelegatingPeekingKeyValueIterator, byte[]> storeIterator = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(storeKvs)); @@ -136,7 +187,7 @@ private MergedSortedCacheWindowStoreKeyValueIterator createIterator( new StateSerdes<>("name", Serdes.Bytes(), Serdes.ByteArray()), WINDOW_SIZE, SINGLE_SEGMENT_CACHE_FUNCTION, - false + reverse ); } } From c1981157628219a9195b17b317d7ccddeaf50e31 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 22 Jul 2020 11:25:25 +0100 Subject: [PATCH 14/24] partially test composite stores --- .../internals/CompositeKeyValueIterator.java | 3 +- .../CompositeReadOnlyKeyValueStore.java | 16 +- .../internals/CachingWindowStoreTest.java | 6 +- .../CompositeReadOnlyKeyValueStoreTest.java | 32 ++- .../CompositeReadOnlyWindowStoreTest.java | 247 +++++++++++++++--- .../internals/ReadOnlyWindowStoreStub.java | 158 ++++++++++- 6 files changed, 405 insertions(+), 57 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeKeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeKeyValueIterator.java index 4ac6fee2e6cec..1614f9f52a15a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeKeyValueIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeKeyValueIterator.java @@ -50,8 +50,7 @@ public K peekNextKey() { @Override public boolean hasNext() { - while ((current == null || !current.hasNext()) - && storeIterator.hasNext()) { + while ((current == null || !current.hasNext()) && storeIterator.hasNext()) { close(); current = nextIteratorFunction.apply(storeIterator.next()); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java index 4b515e04d8e38..910b2a347021c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java @@ -79,7 +79,9 @@ public KeyValueIterator apply(final ReadOnlyKeyValueStore store) { } }; final List> stores = storeProvider.stores(storeName, storeType); - return new DelegatingPeekingKeyValueIterator<>(storeName, new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction)); + return new DelegatingPeekingKeyValueIterator<>( + storeName, + new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction)); } @Override @@ -97,7 +99,9 @@ public KeyValueIterator apply(final ReadOnlyKeyValueStore store) { } }; final List> stores = storeProvider.stores(storeName, storeType); - return new DelegatingPeekingKeyValueIterator<>(storeName, new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction)); + return new DelegatingPeekingKeyValueIterator<>( + storeName, + new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction)); } @Override @@ -113,7 +117,9 @@ public KeyValueIterator apply(final ReadOnlyKeyValueStore store) { } }; final List> stores = storeProvider.stores(storeName, storeType); - return new DelegatingPeekingKeyValueIterator<>(storeName, new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction)); + return new DelegatingPeekingKeyValueIterator<>( + storeName, + new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction)); } @Override @@ -129,7 +135,9 @@ public KeyValueIterator apply(final ReadOnlyKeyValueStore store) { } }; final List> stores = storeProvider.stores(storeName, storeType); - return new DelegatingPeekingKeyValueIterator<>(storeName, new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction)); + return new DelegatingPeekingKeyValueIterator<>( + storeName, + new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction)); } @Override 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 61878b5716784..7cf6293e87a12 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 @@ -521,7 +521,7 @@ public void shouldIterateAcrossWindows() { } @Test - public void shouldIterateAcrossWindowsBackward() { + public void shouldIterateBackwardAcrossWindows() { cachingStore.put(bytesKey("1"), bytesValue("a"), DEFAULT_TIMESTAMP); cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE); @@ -545,7 +545,7 @@ public void shouldIterateCacheAndStore() { } @Test - public void shouldIterateCacheBackwardAndStore() { + 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); @@ -576,7 +576,7 @@ public void shouldIterateCacheAndStoreKeyRange() { } @Test - public void shouldIterateCacheAndStoreKeyRangeBackward() { + 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); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java index 9564dca12748f..00312500fcc1b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java @@ -41,6 +41,7 @@ import static java.util.Arrays.asList; import static java.util.Collections.singletonList; import static org.apache.kafka.test.StreamsTestUtils.toList; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -213,9 +214,12 @@ public void shouldSupportReverseRange() { stubOneUnderlying.put("c", "c"); final List> results = toList(theStore.reverseRange("a", "b")); - assertTrue(results.contains(new KeyValue<>("a", "a"))); - assertTrue(results.contains(new KeyValue<>("b", "b"))); - assertEquals(2, results.size()); + assertArrayEquals( + asList( + new KeyValue<>("b", "b"), + new KeyValue<>("a", "a") + ).toArray(), + results.toArray()); } @Test @@ -232,11 +236,14 @@ public void shouldSupportRangeAcrossMultipleKVStores() { cache.put("x", "x"); final List> results = toList(theStore.range("a", "e")); - assertTrue(results.contains(new KeyValue<>("a", "a"))); - assertTrue(results.contains(new KeyValue<>("b", "b"))); - assertTrue(results.contains(new KeyValue<>("c", "c"))); - assertTrue(results.contains(new KeyValue<>("d", "d"))); - assertEquals(4, results.size()); + assertArrayEquals( + asList( + new KeyValue<>("a", "a"), + new KeyValue<>("b", "b"), + new KeyValue<>("c", "c"), + new KeyValue<>("d", "d") + ).toArray(), + results.toArray()); } @Test @@ -258,6 +265,15 @@ public void shouldSupportReverseRangeAcrossMultipleKVStores() { assertTrue(results.contains(new KeyValue<>("c", "c"))); assertTrue(results.contains(new KeyValue<>("d", "d"))); assertEquals(4, results.size()); + //FIXME: order does not hold between stores, how to validate order here? +// assertArrayEquals( +// asList( +// new KeyValue<>("d", "d"), +// new KeyValue<>("c", "c"), +// new KeyValue<>("b", "b"), +// new KeyValue<>("a", "a") +// ).toArray(), +// results.toArray()); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java index 514d0940a0c29..39313e17e3c0c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java @@ -44,6 +44,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsEqual.equalTo; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThrows; public class CompositeReadOnlyWindowStoreTest { @@ -69,8 +70,8 @@ public void before() { windowStore = new CompositeReadOnlyWindowStore<>( new WrappingStoreProvider(asList(stubProviderOne, stubProviderTwo), StoreQueryParameters.fromNameAndType(storeName, QueryableStoreTypes.windowStore())), - QueryableStoreTypes.windowStore(), - storeName + QueryableStoreTypes.windowStore(), + storeName ); } @@ -79,18 +80,41 @@ public void shouldFetchValuesFromWindowStore() { underlyingWindowStore.put("my-key", "my-value", 0L); underlyingWindowStore.put("my-key", "my-later-value", 10L); - final WindowStoreIterator iterator = windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); + final WindowStoreIterator iterator = + windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); final List> results = StreamsTestUtils.toList(iterator); - assertEquals(asList(new KeyValue<>(0L, "my-value"), - new KeyValue<>(10L, "my-later-value")), - results); + assertEquals( + asList(new KeyValue<>(0L, "my-value"), new KeyValue<>(10L, "my-later-value")), + results); + } + + @Test + public void shouldBackwardFetchValuesFromWindowStore() { + underlyingWindowStore.put("my-key", "my-value", 0L); + underlyingWindowStore.put("my-key", "my-later-value", 10L); + + final WindowStoreIterator iterator = + windowStore.backwardFetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); + final List> results = StreamsTestUtils.toList(iterator); + + assertEquals( + asList(new KeyValue<>(10L, "my-later-value"), new KeyValue<>(0L, "my-value")), + results); } @Test public void shouldReturnEmptyIteratorIfNoData() { - final WindowStoreIterator iterator = windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); - assertEquals(false, iterator.hasNext()); + final WindowStoreIterator iterator = + windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); + assertFalse(iterator.hasNext()); + } + + @Test + public void shouldReturnBackwardEmptyIteratorIfNoData() { + final WindowStoreIterator iterator = + windowStore.backwardFetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); + assertFalse(iterator.hasNext()); } @Test @@ -102,10 +126,28 @@ public void shouldFindValueForKeyWhenMultiStores() { underlyingWindowStore.put("key-one", "value-one", 0L); secondUnderlying.put("key-two", "value-two", 10L); - final List> keyOneResults = StreamsTestUtils.toList(windowStore.fetch("key-one", ofEpochMilli(0L), - ofEpochMilli(1L))); - final List> keyTwoResults = StreamsTestUtils.toList(windowStore.fetch("key-two", ofEpochMilli(10L), - ofEpochMilli(11L))); + final List> keyOneResults = + StreamsTestUtils.toList(windowStore.fetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L))); + final List> keyTwoResults = + StreamsTestUtils.toList(windowStore.fetch("key-two", ofEpochMilli(10L), ofEpochMilli(11L))); + + assertEquals(Collections.singletonList(KeyValue.pair(0L, "value-one")), keyOneResults); + assertEquals(Collections.singletonList(KeyValue.pair(10L, "value-two")), keyTwoResults); + } + + @Test + public void shouldFindValueForKeyWhenMultiStoresBackwards() { + final ReadOnlyWindowStoreStub secondUnderlying = new + ReadOnlyWindowStoreStub<>(WINDOW_SIZE); + stubProviderTwo.addStore(storeName, secondUnderlying); + + underlyingWindowStore.put("key-one", "value-one", 0L); + secondUnderlying.put("key-two", "value-two", 10L); + + final List> keyOneResults = + StreamsTestUtils.toList(windowStore.backwardFetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L))); + final List> keyTwoResults = + StreamsTestUtils.toList(windowStore.backwardFetch("key-two", ofEpochMilli(10L), ofEpochMilli(11L))); assertEquals(Collections.singletonList(KeyValue.pair(0L, "value-one")), keyOneResults); assertEquals(Collections.singletonList(KeyValue.pair(10L, "value-two")), keyTwoResults); @@ -116,7 +158,18 @@ public void shouldNotGetValuesFromOtherStores() { otherUnderlyingStore.put("some-key", "some-value", 0L); underlyingWindowStore.put("some-key", "my-value", 1L); - final List> results = StreamsTestUtils.toList(windowStore.fetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L))); + final List> results = + StreamsTestUtils.toList(windowStore.fetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L))); + assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results); + } + + @Test + public void shouldNotGetValuesBackwardFromOtherStores() { + otherUnderlyingStore.put("some-key", "some-value", 0L); + underlyingWindowStore.put("some-key", "my-value", 1L); + + final List> results = + StreamsTestUtils.toList(windowStore.backwardFetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L))); assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results); } @@ -124,7 +177,7 @@ public void shouldNotGetValuesFromOtherStores() { public void shouldThrowInvalidStateStoreExceptionOnRebalance() { final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); EasyMock.expect(storeProvider.stores(anyString(), anyObject())) - .andThrow(new InvalidStateStoreException("store is unavailable")); + .andThrow(new InvalidStateStoreException("store is unavailable")); EasyMock.replay(storeProvider); final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>( @@ -135,24 +188,74 @@ public void shouldThrowInvalidStateStoreExceptionOnRebalance() { store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); } + @Test(expected = InvalidStateStoreException.class) + public void shouldThrowInvalidStateStoreExceptionOnRebalanceWhenBackwards() { + final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); + EasyMock.expect(storeProvider.stores(anyString(), anyObject())) + .andThrow(new InvalidStateStoreException("store is unavailable")); + EasyMock.replay(storeProvider); + + final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>( + storeProvider, + QueryableStoreTypes.windowStore(), + "foo" + ); + store.backwardFetch("key", ofEpochMilli(1), ofEpochMilli(10)); + } + @Test public void shouldThrowInvalidStateStoreExceptionIfFetchThrows() { underlyingWindowStore.setOpen(false); final CompositeReadOnlyWindowStore store = - new CompositeReadOnlyWindowStore<>( - new WrappingStoreProvider(singletonList(stubProviderOne), StoreQueryParameters.fromNameAndType("window-store", QueryableStoreTypes.windowStore())), - QueryableStoreTypes.windowStore(), - "window-store" - ); + new CompositeReadOnlyWindowStore<>( + new WrappingStoreProvider(singletonList(stubProviderOne), StoreQueryParameters.fromNameAndType("window-store", QueryableStoreTypes.windowStore())), + QueryableStoreTypes.windowStore(), + "window-store" + ); try { store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); Assert.fail("InvalidStateStoreException was expected"); } catch (final InvalidStateStoreException e) { Assert.assertEquals("State store is not available anymore and may have been migrated to another instance; " + - "please re-discover its location from the state metadata.", e.getMessage()); + "please re-discover its location from the state metadata.", e.getMessage()); + } + } + + @Test + public void shouldThrowInvalidStateStoreExceptionIfBackwardFetchThrows() { + underlyingWindowStore.setOpen(false); + final CompositeReadOnlyWindowStore store = + new CompositeReadOnlyWindowStore<>( + new WrappingStoreProvider(singletonList(stubProviderOne), StoreQueryParameters.fromNameAndType("window-store", QueryableStoreTypes.windowStore())), + QueryableStoreTypes.windowStore(), + "window-store" + ); + try { + store.backwardFetch("key", ofEpochMilli(1), ofEpochMilli(10)); + Assert.fail("InvalidStateStoreException was expected"); + } catch (final InvalidStateStoreException e) { + Assert.assertEquals("State store is not available anymore and may have been migrated to another instance; " + + "please re-discover its location from the state metadata.", e.getMessage()); } } + @Test + public void emptyBackwardIteratorAlwaysReturnsFalse() { + final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); + EasyMock.expect(storeProvider.stores(anyString(), anyObject())).andReturn(emptyList()); + EasyMock.replay(storeProvider); + + final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>( + storeProvider, + QueryableStoreTypes.windowStore(), + "foo" + ); + final WindowStoreIterator windowStoreIterator = + store.backwardFetch("key", ofEpochMilli(1), ofEpochMilli(10)); + + Assert.assertFalse(windowStoreIterator.hasNext()); + } + @Test public void emptyIteratorAlwaysReturnsFalse() { final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); @@ -164,11 +267,28 @@ public void emptyIteratorAlwaysReturnsFalse() { QueryableStoreTypes.windowStore(), "foo" ); - final WindowStoreIterator windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); + final WindowStoreIterator windowStoreIterator = + store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); Assert.assertFalse(windowStoreIterator.hasNext()); } + @Test + public void emptyBackwardIteratorPeekNextKeyShouldThrowNoSuchElementException() { + final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); + EasyMock.expect(storeProvider.stores(anyString(), anyObject())).andReturn(emptyList()); + EasyMock.replay(storeProvider); + + final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>( + storeProvider, + QueryableStoreTypes.windowStore(), + "foo" + ); + final WindowStoreIterator windowStoreIterator = store.backwardFetch("key", ofEpochMilli(1), ofEpochMilli(10)); + assertThrows(NoSuchElementException.class, windowStoreIterator::peekNextKey); + } + + @Test public void emptyIteratorPeekNextKeyShouldThrowNoSuchElementException() { final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); @@ -180,7 +300,8 @@ public void emptyIteratorPeekNextKeyShouldThrowNoSuchElementException() { QueryableStoreTypes.windowStore(), "foo" ); - final WindowStoreIterator windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); + final WindowStoreIterator windowStoreIterator = + store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); assertThrows(NoSuchElementException.class, windowStoreIterator::peekNextKey); } @@ -195,7 +316,24 @@ public void emptyIteratorNextShouldThrowNoSuchElementException() { QueryableStoreTypes.windowStore(), "foo" ); - final WindowStoreIterator windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); + final WindowStoreIterator windowStoreIterator = + store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); + assertThrows(NoSuchElementException.class, windowStoreIterator::next); + } + + @Test + public void emptyBackwardIteratorNextShouldThrowNoSuchElementException() { + final StateStoreProvider storeProvider = EasyMock.createNiceMock(StateStoreProvider.class); + EasyMock.expect(storeProvider.stores(anyString(), anyObject())).andReturn(emptyList()); + EasyMock.replay(storeProvider); + + final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>( + storeProvider, + QueryableStoreTypes.windowStore(), + "foo" + ); + final WindowStoreIterator windowStoreIterator = + store.backwardFetch("key", ofEpochMilli(1), ofEpochMilli(10)); assertThrows(NoSuchElementException.class, windowStoreIterator::next); } @@ -205,10 +343,24 @@ public void shouldFetchKeyRangeAcrossStores() { stubProviderTwo.addStore(storeName, secondUnderlying); underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); - final List, String>> results = StreamsTestUtils.toList(windowStore.fetch("a", "b", ofEpochMilli(0), ofEpochMilli(10))); + final List, String>> results = + StreamsTestUtils.toList(windowStore.fetch("a", "b", ofEpochMilli(0), ofEpochMilli(10))); + assertThat(results, equalTo(Arrays.asList( + KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), + KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); + } + + @Test + public void shouldBackwardFetchKeyRangeAcrossStores() { + final ReadOnlyWindowStoreStub secondUnderlying = new ReadOnlyWindowStoreStub<>(WINDOW_SIZE); + stubProviderTwo.addStore(storeName, secondUnderlying); + underlyingWindowStore.put("a", "a", 0L); + secondUnderlying.put("b", "b", 10L); + final List, String>> results = + StreamsTestUtils.toList(windowStore.backwardFetch("a", "b", ofEpochMilli(0), ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( - KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), - KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); + KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), + KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); } @Test @@ -223,31 +375,58 @@ public void shouldFetchKeyValueAcrossStores() { assertThat(windowStore.fetch("a", 10L), equalTo(null)); } - @Test public void shouldGetAllAcrossStores() { final ReadOnlyWindowStoreStub secondUnderlying = new - ReadOnlyWindowStoreStub<>(WINDOW_SIZE); + ReadOnlyWindowStoreStub<>(WINDOW_SIZE); stubProviderTwo.addStore(storeName, secondUnderlying); underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); final List, String>> results = StreamsTestUtils.toList(windowStore.all()); assertThat(results, equalTo(Arrays.asList( - KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), - KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); + KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), + KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); + } + + @Test + public void shouldGetBackwardAllAcrossStores() { + final ReadOnlyWindowStoreStub secondUnderlying = new + ReadOnlyWindowStoreStub<>(WINDOW_SIZE); + stubProviderTwo.addStore(storeName, secondUnderlying); + underlyingWindowStore.put("a", "a", 0L); + secondUnderlying.put("b", "b", 10L); + final List, String>> results = StreamsTestUtils.toList(windowStore.backwardAll()); + assertThat(results, equalTo(Arrays.asList( + KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), + KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); } @Test public void shouldFetchAllAcrossStores() { final ReadOnlyWindowStoreStub secondUnderlying = new - ReadOnlyWindowStoreStub<>(WINDOW_SIZE); + ReadOnlyWindowStoreStub<>(WINDOW_SIZE); + stubProviderTwo.addStore(storeName, secondUnderlying); + underlyingWindowStore.put("a", "a", 0L); + secondUnderlying.put("b", "b", 10L); + final List, String>> results = + StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(0), ofEpochMilli(10))); + assertThat(results, equalTo(Arrays.asList( + KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), + KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); + } + + @Test + public void shouldBackwardFetchAllAcrossStores() { + final ReadOnlyWindowStoreStub secondUnderlying = new + ReadOnlyWindowStoreStub<>(WINDOW_SIZE); stubProviderTwo.addStore(storeName, secondUnderlying); underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); - final List, String>> results = StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(0), ofEpochMilli(10))); + final List, String>> results = + StreamsTestUtils.toList(windowStore.backwardFetchAll(ofEpochMilli(0), ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( - KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), - KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); + KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), + KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); } @Test(expected = NullPointerException.class) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java index 468d551d793c0..59866ab467eec 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 @@ -81,11 +81,28 @@ public WindowStoreIterator fetch(final K key, final long timeFrom, final long @Override public WindowStoreIterator fetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException { return fetch( - key, + key, ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")), ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to"))); } + @Override + public WindowStoreIterator backwardFetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + if (!open) { + throw new InvalidStateStoreException("Store is not open"); + } + final List> results = new ArrayList<>(); + for (long now = timeTo; now >= timeFrom; now--) { + final Map kvMap = data.get(now); + if (kvMap != null && kvMap.containsKey(key)) { + results.add(new KeyValue<>(now, kvMap.get(key))); + } + } + return new TheWindowStoreIterator<>(results.iterator()); + } + @Override public KeyValueIterator, V> all() { if (!open) { @@ -102,6 +119,45 @@ public KeyValueIterator, V> all() { } 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(); + } + + }; + } + + @Override + public KeyValueIterator, V> backwardAll() { + if (!open) { + throw new InvalidStateStoreException("Store is not open"); + } + final List, V>> results = new ArrayList<>(); + for (final long now : data.keySet()) { + final NavigableMap kvMap = data.get(now); + if (kvMap != null) { + for (final Entry entry : kvMap.descendingMap().entrySet()) { + results.add(new KeyValue<>(new Windowed<>(entry.getKey(), new TimeWindow(now, now + windowSize)), entry.getValue())); + } + } + } + final Iterator, V>> iterator = results.iterator(); + return new KeyValueIterator, V>() { @Override public void close() {} @@ -173,6 +229,51 @@ public KeyValueIterator, V> fetchAll(final Instant from, final Insta ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to"))); } + @Override + public KeyValueIterator, V> backwardFetchAll(final Instant from, final Instant to) throws IllegalArgumentException { + final long timeFrom = ApiUtils.validateMillisecondInstant(from, prepareMillisCheckFailMsgPrefix(from, "from")); + final long timeTo = ApiUtils.validateMillisecondInstant(to, prepareMillisCheckFailMsgPrefix(to, "to")); + + if (!open) { + throw new InvalidStateStoreException("Store is not open"); + } + final List, V>> results = new ArrayList<>(); + for (final long now : data.keySet()) { + if (!(now >= timeFrom && now <= timeTo)) { + continue; + } + final NavigableMap kvMap = data.get(now); + if (kvMap != null) { + for (final Entry entry : kvMap.descendingMap().entrySet()) { + results.add(new KeyValue<>(new Windowed<>(entry.getKey(), new TimeWindow(now, now + windowSize)), entry.getValue())); + } + } + } + final Iterator, V>> iterator = results.iterator(); + + return new KeyValueIterator, V>() { + @Override + public void close() { + } + + @Override + public Windowed peekNextKey() { + throw new UnsupportedOperationException("peekNextKey() not supported in " + getClass().getName()); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public KeyValue, V> next() { + return iterator.next(); + } + + }; + } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { @@ -212,17 +313,62 @@ public KeyValue, V> next() { }; } - @Override public KeyValueIterator, V> fetch(final K from, - final K to, - final Instant fromTime, - final Instant toTime) throws IllegalArgumentException { + @Override + public KeyValueIterator, V> fetch(final K from, + final K to, + final Instant fromTime, + final Instant toTime) throws IllegalArgumentException { return fetch( from, - to, + to, ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")), ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime"))); } + @Override + public KeyValueIterator, V> backwardFetch(final K from, + final K to, + final Instant fromTimeInstant, + final Instant toTimeInstant) throws IllegalArgumentException { + final long timeFrom = ApiUtils.validateMillisecondInstant(fromTimeInstant, prepareMillisCheckFailMsgPrefix(fromTimeInstant, "fromTimeInstant")); + final long timeTo = ApiUtils.validateMillisecondInstant(toTimeInstant, prepareMillisCheckFailMsgPrefix(toTimeInstant, "toTimeInstant")); + if (!open) { + throw new InvalidStateStoreException("Store is not open"); + } + final List, V>> results = new ArrayList<>(); + for (long now = timeFrom; now <= timeTo; now++) { + final NavigableMap kvMap = data.get(now); + if (kvMap != null) { + for (final Entry entry : kvMap.subMap(from, true, to, true).descendingMap().entrySet()) { + results.add(new KeyValue<>(new Windowed<>(entry.getKey(), new TimeWindow(now, now + windowSize)), entry.getValue())); + } + } + } + final Iterator, V>> iterator = results.iterator(); + + return new KeyValueIterator, V>() { + @Override + public void close() { + } + + @Override + public Windowed peekNextKey() { + throw new UnsupportedOperationException("peekNextKey() not supported in " + getClass().getName()); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public KeyValue, V> next() { + return iterator.next(); + } + + }; + } + public void put(final K key, final V value, final long timestamp) { if (!data.containsKey(timestamp)) { data.put(timestamp, new TreeMap<>()); From 026e762953a6ae9ec1e24917981df96c1a7f80d3 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 23 Jul 2020 14:36:14 +0100 Subject: [PATCH 15/24] default new methods --- .../streams/state/ReadOnlySessionStore.java | 38 +++++++++++++------ 1 file changed, 27 insertions(+), 11 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java index a0af5bc1a2325..9874d4902ddc5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java @@ -41,41 +41,57 @@ public interface ReadOnlySessionStore { * @return iterator of sessions with the matching key and aggregated values * @throws NullPointerException If null is used for key. */ - KeyValueIterator, AGG> findSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime); + default KeyValueIterator, AGG> findSessions(final K key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + throw new UnsupportedOperationException("Operation moved from SessionStore"); + } - default KeyValueIterator, AGG> backwardFindSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime) { + default KeyValueIterator, AGG> backwardFindSessions(final K key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { throw new UnsupportedOperationException(); } /** * Fetch any sessions in the given range of keys and the sessions end is ≥ earliestSessionEndTime and the sessions * start is ≤ latestSessionStartTime - * + *

* This iterator must be closed after use. * - * @param keyFrom The first key that could be in the range - * @param keyTo The last key that could be in the range + * @param keyFrom The first key that could be in the range + * @param keyTo The last key that could be in the range * @param earliestSessionEndTime the end timestamp of the earliest session to search for * @param latestSessionStartTime the end timestamp of the latest session to search for * @return iterator of sessions with the matching keys and aggregated values * @throws NullPointerException If null is used for any key. */ - KeyValueIterator, AGG> findSessions(final K keyFrom, final K keyTo, final long earliestSessionEndTime, final long latestSessionStartTime); + default KeyValueIterator, AGG> findSessions(final K keyFrom, + final K keyTo, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + throw new UnsupportedOperationException("Operation moved from SessionStore"); + } - default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, final K keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { + default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, + final K keyTo, + final long earliestSessionEndTime, + final long latestSessionStartTime) { throw new UnsupportedOperationException(); } /** * Get the value of key from a single session. * - * @param key the key to fetch - * @param startTime start timestamp of the session - * @param endTime end timestamp of the session + * @param key the key to fetch + * @param startTime start timestamp of the session + * @param endTime end timestamp of the session * @return The value or {@code null} if no session associated with the key can be found * @throws NullPointerException If {@code null} is used for any key. */ - AGG fetchSession(final K key, final long startTime, final long endTime); + default AGG fetchSession(final K key, final long startTime, final long endTime) { + throw new UnsupportedOperationException("Operation moved from SessionStore"); + } /** * Retrieve all aggregated sessions for the provided key. From de93a5476ef61c8b63316632db72a1a666d772db Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Sat, 25 Jul 2020 15:01:41 +0100 Subject: [PATCH 16/24] remove wrong deprecation --- .../kafka/streams/state/internals/InMemorySessionStore.java | 1 - 1 file changed, 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index c499c67bbfbf1..25d11d1a1c551 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -207,7 +207,6 @@ public KeyValueIterator, byte[]> findSessions(final Bytes keyFro endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator()); } - @Deprecated @Override public KeyValueIterator, byte[]> backwardFindSessions(final Bytes keyFrom, final Bytes keyTo, From 41954532e4c8eced7ee16a4df5207a6930adf6de Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 28 Jul 2020 11:49:29 +0100 Subject: [PATCH 17/24] docs: improve ro kv store ordering description --- .../streams/state/ReadOnlyKeyValueStore.java | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java index c02b78b8c907d..7a7df9c7f0b35 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java @@ -46,19 +46,22 @@ public interface ReadOnlyKeyValueStore { /** * Get an iterator over a given range of keys. This iterator must be closed after use. * The returned iterator must be safe from {@link java.util.ConcurrentModificationException}s - * and must not return null values. No ordering guarantees are provided. + * and must not return null values. + * Order is not guaranteed as bytes lexicographical ordering might not represent key order. + * * @param from The first key that could be in the range - * @param to The last key that could be in the range + * @param to The last key that could be in the range * @return The iterator for this range. - * @throws NullPointerException If null is used for from or to. + * @throws NullPointerException If null is used for from or to. * @throws InvalidStateStoreException if the store is not initialized */ KeyValueIterator range(K from, K to); /** - * Get an reverse iterator over a given range of keys. This iterator must be closed after use. + * Get a reverse iterator over a given range of keys. This iterator must be closed after use. * The returned iterator must be safe from {@link java.util.ConcurrentModificationException}s - * and must not return null values. No ordering guarantees are provided. + * and must not return null values. + * Order is not guaranteed as bytes lexicographical ordering might not represent key order. * @param from The last key that could be in the range * @param to The first key that could be in the range * @return The reverse iterator for this range. @@ -72,7 +75,8 @@ default KeyValueIterator reverseRange(K from, K to) { /** * Return an iterator over all keys in this store. This iterator must be closed after use. * The returned iterator must be safe from {@link java.util.ConcurrentModificationException}s - * and must not return null values. No ordering guarantees are provided. + * and must not return null values. + * Order is not guaranteed as bytes lexicographical ordering might not represent key order. * @return An iterator of all key/value pairs in the store. * @throws InvalidStateStoreException if the store is not initialized */ @@ -81,7 +85,8 @@ default KeyValueIterator reverseRange(K from, K to) { /** * Return an reverse iterator over all keys in this store. This iterator must be closed after use. * The returned iterator must be safe from {@link java.util.ConcurrentModificationException}s - * and must not return null values. No ordering guarantees are provided. + * and must not return null values. + * Order is not guaranteed as bytes lexicographical ordering might not represent key order. * @return An reverse iterator of all key/value pairs in the store. * @throws InvalidStateStoreException if the store is not initialized */ From 4edac651198f6e6027904454dda9078a03d83034 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 28 Jul 2020 12:09:53 +0100 Subject: [PATCH 18/24] docs: improve reverse/backward ops dscription --- .../streams/state/ReadOnlyKeyValueStore.java | 15 ++-- .../streams/state/ReadOnlySessionStore.java | 84 +++++++++++++---- .../streams/state/ReadOnlyWindowStore.java | 90 +++++++++---------- 3 files changed, 120 insertions(+), 69 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java index 7a7df9c7f0b35..ff7602ccfa18c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java @@ -21,7 +21,7 @@ /** * A key-value store that only supports read operations. * Implementations should be thread-safe as concurrent reads and writes are expected. - * + *

* Please note that this contract defines the thread-safe read functionality only; it does not * guarantee anything about whether the actual instance is writable by another thread, or * whether it uses some locking mechanism under the hood. For this reason, making dependencies @@ -38,7 +38,7 @@ public interface ReadOnlyKeyValueStore { * * @param key The key to fetch * @return The value or null if no value is found. - * @throws NullPointerException If null is used for key. + * @throws NullPointerException If null is used for key. * @throws InvalidStateStoreException if the store is not initialized */ V get(K key); @@ -62,10 +62,11 @@ public interface ReadOnlyKeyValueStore { * The returned iterator must be safe from {@link java.util.ConcurrentModificationException}s * and must not return null values. * Order is not guaranteed as bytes lexicographical ordering might not represent key order. + * * @param from The last key that could be in the range - * @param to The first key that could be in the range + * @param to The first key that could be in the range * @return The reverse iterator for this range. - * @throws NullPointerException If null is used for from or to. + * @throws NullPointerException If null is used for from or to. * @throws InvalidStateStoreException if the store is not initialized */ default KeyValueIterator reverseRange(K from, K to) { @@ -77,16 +78,18 @@ default KeyValueIterator reverseRange(K from, K to) { * The returned iterator must be safe from {@link java.util.ConcurrentModificationException}s * and must not return null values. * Order is not guaranteed as bytes lexicographical ordering might not represent key order. + * * @return An iterator of all key/value pairs in the store. * @throws InvalidStateStoreException if the store is not initialized */ KeyValueIterator all(); /** - * Return an reverse iterator over all keys in this store. This iterator must be closed after use. + * Return a reverse iterator over all keys in this store. This iterator must be closed after use. * The returned iterator must be safe from {@link java.util.ConcurrentModificationException}s * and must not return null values. * Order is not guaranteed as bytes lexicographical ordering might not represent key order. + * * @return An reverse iterator of all key/value pairs in the store. * @throws InvalidStateStoreException if the store is not initialized */ @@ -96,7 +99,7 @@ default KeyValueIterator reverseAll() { /** * Return an approximate count of key-value mappings in this store. - * + *

* The count is not guaranteed to be exact in order to accommodate stores * where an exact count is expensive to calculate. * diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java index 9874d4902ddc5..018dbedb3cabc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java @@ -24,18 +24,18 @@ * Implementations should be thread-safe as concurrent reads and writes * are expected. * - * @param the key type + * @param the key type * @param the aggregated value type */ public interface ReadOnlySessionStore { /** * Fetch any sessions with the matching key and the sessions end is ≥ earliestSessionEndTime and the sessions - * start is ≤ latestSessionStartTime - * + * start is ≤ latestSessionStartTime iterating from earliest to latest. + *

* This iterator must be closed after use. * - * @param key the key to return sessions for + * @param key the key to return sessions for * @param earliestSessionEndTime the end timestamp of the earliest session to search for * @param latestSessionStartTime the end timestamp of the latest session to search for * @return iterator of sessions with the matching key and aggregated values @@ -44,9 +44,21 @@ public interface ReadOnlySessionStore { default KeyValueIterator, AGG> findSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime) { - throw new UnsupportedOperationException("Operation moved from SessionStore"); + throw new UnsupportedOperationException("Moved from SessionStore"); } + /** + * Fetch any sessions with the matching key and the sessions end is ≥ earliestSessionEndTime and the sessions + * start is ≤ latestSessionStartTime iterating from latest to earliest. + *

+ * This iterator must be closed after use. + * + * @param key the key to return sessions for + * @param earliestSessionEndTime the end timestamp of the earliest session to search for + * @param latestSessionStartTime the end timestamp of the latest session to search for + * @return backward iterator of sessions with the matching key and aggregated values + * @throws NullPointerException If null is used for key. + */ default KeyValueIterator, AGG> backwardFindSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime) { @@ -55,7 +67,7 @@ default KeyValueIterator, AGG> backwardFindSessions(final K key, /** * Fetch any sessions in the given range of keys and the sessions end is ≥ earliestSessionEndTime and the sessions - * start is ≤ latestSessionStartTime + * start is ≤ latestSessionStartTime iterating from earliest to latest. *

* This iterator must be closed after use. * @@ -70,9 +82,23 @@ default KeyValueIterator, AGG> findSessions(final K keyFrom, final K keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { - throw new UnsupportedOperationException("Operation moved from SessionStore"); + throw new UnsupportedOperationException("Moved from SessionStore"); } + + /** + * Fetch any sessions in the given range of keys and the sessions end is ≥ earliestSessionEndTime and the sessions + * start is ≤ latestSessionStartTime iterating from latest to earliest. + *

+ * This iterator must be closed after use. + * + * @param keyFrom The first key that could be in the range + * @param keyTo The last key that could be in the range + * @param earliestSessionEndTime the end timestamp of the earliest session to search for + * @param latestSessionStartTime the end timestamp of the latest session to search for + * @return backward iterator of sessions with the matching keys and aggregated values + * @throws NullPointerException If null is used for any key. + */ default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, final K keyTo, final long earliestSessionEndTime, @@ -90,23 +116,33 @@ default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, * @throws NullPointerException If {@code null} is used for any key. */ default AGG fetchSession(final K key, final long startTime, final long endTime) { - throw new UnsupportedOperationException("Operation moved from SessionStore"); + throw new UnsupportedOperationException("Moved from SessionStore"); } /** * Retrieve all aggregated sessions for the provided key. * This iterator must be closed after use. - * + *

* For each key, the iterator guarantees ordering of sessions, starting from the oldest/earliest * available session to the newest/latest session. * - * @param key record key to find aggregated session values for - * @return KeyValueIterator containing all sessions for the provided key. - * @throws NullPointerException If null is used for key. - * + * @param key record key to find aggregated session values for + * @return KeyValueIterator containing all sessions for the provided key. + * @throws NullPointerException If null is used for key. */ KeyValueIterator, AGG> fetch(final K key); + /** + * Retrieve all aggregated sessions for the provided key. + * This iterator must be closed after use. + *

+ * For each key, the iterator guarantees ordering of sessions, starting from the newest/latest + * available session to the oldest/earliest session. + * + * @param key record key to find aggregated session values for + * @return backward KeyValueIterator containing all sessions for the provided key. + * @throws NullPointerException If null is used for key. + */ default KeyValueIterator, AGG> backwardFetch(final K key) { throw new UnsupportedOperationException(); } @@ -114,17 +150,29 @@ default KeyValueIterator, AGG> backwardFetch(final K key) { /** * Retrieve all aggregated sessions for the given range of keys. * This iterator must be closed after use. - * + *

* For each key, the iterator guarantees ordering of sessions, starting from the oldest/earliest * available session to the newest/latest session. * - * @param from first key in the range to find aggregated session values for - * @param to last key in the range to find aggregated session values for - * @return KeyValueIterator containing all sessions for the provided key. - * @throws NullPointerException If null is used for any of the keys. + * @param from first key in the range to find aggregated session values for + * @param to last key in the range to find aggregated session values for + * @return KeyValueIterator containing all sessions for the provided key. + * @throws NullPointerException If null is used for any of the keys. */ KeyValueIterator, AGG> fetch(final K from, final K to); + /** + * Retrieve all aggregated sessions for the given range of keys. + * This iterator must be closed after use. + *

+ * For each key, the iterator guarantees ordering of sessions, starting from the newest/latest + * available session to the oldest/earliest session. + * + * @param from first key in the range to find aggregated session values for + * @param to last key in the range to find aggregated session values for + * @return backward KeyValueIterator containing all sessions for the provided key. + * @throws NullPointerException If null is used for any of the keys. + */ default KeyValueIterator, AGG> backwardFetch(final K from, final K to) { throw new UnsupportedOperationException(); } 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 b06a4e7b2eb7a..9c44123866dd9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java @@ -33,11 +33,11 @@ public interface ReadOnlyWindowStore { /** * Get the value of key from a window. * - * @param key the key to fetch - * @param time start timestamp (inclusive) of the window + * @param key the key to fetch + * @param time start timestamp (inclusive) of the window * @return The value or {@code null} if no value is found in the window * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for any key. + * @throws NullPointerException If {@code null} is used for any key. */ V fetch(K key, long time); @@ -67,12 +67,12 @@ public interface ReadOnlyWindowStore { * For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest * available window to the newest/latest window. * - * @param key the key to fetch - * @param timeFrom time range start (inclusive) - * @param timeTo time range end (inclusive) + * @param key the key to fetch + * @param timeFrom time range start (inclusive) + * @param timeTo time range end (inclusive) * @return an iterator over key-value pairs {@code } * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for key. + * @throws NullPointerException If {@code null} is used for key. * @deprecated Use {@link #fetch(Object, Instant, Instant)} instead */ @Deprecated @@ -104,13 +104,13 @@ public interface ReadOnlyWindowStore { * For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest * available window to the newest/latest window. * - * @param key the key to fetch - * @param from time range start (inclusive) - * @param to time range end (inclusive) + * @param key the key to fetch + * @param from time range start (inclusive) + * @param to time range end (inclusive) * @return an iterator over key-value pairs {@code } * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for key. - * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + * @throws NullPointerException If {@code null} is used for key. + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ WindowStoreIterator fetch(K key, Instant from, Instant to) throws IllegalArgumentException; @@ -142,13 +142,13 @@ public interface ReadOnlyWindowStore { * For each key, the iterator guarantees ordering of windows, starting from the newest/latest * available window to the oldest/earliest window. * - * @param key the key to fetch - * @param from time range start (inclusive) - * @param to time range end (inclusive) + * @param key the key to fetch + * @param from time range start (inclusive) + * @param to time range end (inclusive) * @return an iterator over key-value pairs {@code } * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for key. - * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + * @throws NullPointerException If {@code null} is used for key. + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ default WindowStoreIterator backwardFetch(K key, Instant from, Instant to) throws IllegalArgumentException { throw new UnsupportedOperationException(); @@ -159,13 +159,13 @@ default WindowStoreIterator backwardFetch(K key, Instant from, Instant to) th *

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

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

* This iterator must be closed after use. * - * @param from the first key in the range - * @param to the last key in the range - * @param fromTime time range start (inclusive) - * @param toTime time range end (inclusive) + * @param from the first key in the range + * @param to the last key in the range + * @param fromTime time range start (inclusive) + * @param toTime time range end (inclusive) * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException If {@code null} is used for any key. - * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + * @throws NullPointerException If {@code null} is used for any key. + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ default KeyValueIterator, V> backwardFetch(K from, K to, Instant fromTime, Instant toTime) - throws IllegalArgumentException { + throws IllegalArgumentException { throw new UnsupportedOperationException(); } /** - * Gets all the key-value pairs in the existing windows. - * - * @return an iterator over windowed key-value pairs {@code , value>} - * @throws InvalidStateStoreException if the store is not initialized - */ + * Gets all the key-value pairs in the existing windows. + * + * @return an iterator over windowed key-value pairs {@code , value>} + * @throws InvalidStateStoreException if the store is not initialized + */ KeyValueIterator, V> all(); /** @@ -234,7 +234,7 @@ default KeyValueIterator, V> backwardAll() { * @param timeTo the end of the time slot from which to search (inclusive) * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException if {@code null} is used for any key + * @throws NullPointerException if {@code null} is used for any key * @deprecated Use {@link #fetchAll(Instant, Instant)} instead */ @Deprecated @@ -247,8 +247,8 @@ default KeyValueIterator, V> backwardAll() { * @param to the end of the time slot from which to search (inclusive) * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException if {@code null} is used for any key - * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + * @throws NullPointerException if {@code null} is used for any key + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ KeyValueIterator, V> fetchAll(Instant from, Instant to) throws IllegalArgumentException; @@ -260,8 +260,8 @@ default KeyValueIterator, V> backwardAll() { * @param to the end of the time slot from which to search (inclusive) * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException if {@code null} is used for any key - * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + * @throws NullPointerException if {@code null} is used for any key + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} */ default KeyValueIterator, V> backwardFetchAll(Instant from, Instant to) throws IllegalArgumentException { throw new UnsupportedOperationException(); From d97fb60bbf91fe649b26c86bff5152dcfade79ff Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 28 Jul 2020 12:13:47 +0100 Subject: [PATCH 19/24] docs: improve reverse/backward ops dscription --- .../kafka/streams/state/KeyValueStore.java | 4 +-- .../kafka/streams/state/SessionStore.java | 2 ++ .../kafka/streams/state/WindowStore.java | 35 ++++++++++--------- 3 files changed, 22 insertions(+), 19 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java index b104ad488db3a..3af8d901d849e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java @@ -32,7 +32,7 @@ public interface KeyValueStore extends StateStore, ReadOnlyKeyValueStore extends StateStore, ReadOnlyKeyValueStore extends StateStore, ReadOnlySessionStore extends StateStore, ReadOnlySessionStore extends StateStore, ReadOnlyWindowStore * It's highly recommended to use {@link WindowStore#put(Object, Object, long)} instead, as the record timestamp * is unlikely to be the correct windowStartTimestamp in general. * - * @param key The key to associate the value to + * @param key The key to associate the value to * @param value The value to update, it can be null; * if the serialized bytes are also null it is interpreted as delete * @throws NullPointerException if the given key is {@code null} - * * @deprecated as timestamp is not provided for the key-value pair, this causes inconsistency * to identify the window frame to which the key belongs. * Use {@link #put(Object, Object, long)} instead. - * */ @Deprecated void put(K key, V value); @@ -61,8 +59,8 @@ public interface WindowStore extends StateStore, ReadOnlyWindowStore * If serialized value bytes are null it is interpreted as delete. Note that deletes will be * ignored in the case of an underlying store that retains duplicates. * - * @param key The key to associate the value to - * @param value The value; can be null + * @param key The key to associate the value to + * @param value The value; can be null * @param windowStartTimestamp The timestamp of the beginning of the window to put the key/value into * @throws NullPointerException if the given key is {@code null} */ @@ -94,14 +92,15 @@ public interface WindowStore extends StateStore, ReadOnlyWindowStore * For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest * available window to the newest/latest window. * - * @param key the key to fetch - * @param timeFrom time range start (inclusive) - * @param timeTo time range end (inclusive) + * @param key the key to fetch + * @param timeFrom time range start (inclusive) + * @param timeTo time range end (inclusive) * @return an iterator over key-value pairs {@code } * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException if the given key is {@code null} + * @throws NullPointerException if the given key is {@code null} */ - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed + // note, this method must be kept if super#fetch(...) is removed + @SuppressWarnings("deprecation") WindowStoreIterator fetch(K key, long timeFrom, long timeTo); @Override @@ -119,15 +118,16 @@ default WindowStoreIterator fetch(final K key, *

* This iterator must be closed after use. * - * @param from the first key in the range - * @param to the last key in the range - * @param timeFrom time range start (inclusive) - * @param timeTo time range end (inclusive) + * @param from the first key in the range + * @param to the last key in the range + * @param timeFrom time range start (inclusive) + * @param timeTo time range end (inclusive) * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException if one of the given keys is {@code null} + * @throws NullPointerException if one of the given keys is {@code null} */ - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed + // note, this method must be kept if super#fetch(...) is removed + @SuppressWarnings("deprecation") KeyValueIterator, V> fetch(K from, K to, long timeFrom, long timeTo); @Override @@ -150,7 +150,8 @@ default KeyValueIterator, V> fetch(final K from, * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized */ - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed + // note, this method must be kept if super#fetchAll(...) is removed + @SuppressWarnings("deprecation") KeyValueIterator, V> fetchAll(long timeFrom, long timeTo); @Override From a5e6e8aaf91255d6d4091d431117815680ebbd31 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 28 Jul 2020 13:18:33 +0100 Subject: [PATCH 20/24] testing caching and more stores --- .../state/internals/CachingKeyValueStore.java | 18 +--- .../state/internals/CachingSessionStore.java | 18 +--- .../state/internals/CachingWindowStore.java | 18 +--- .../internals/InMemoryKeyValueStore.java | 21 ++--- .../state/internals/InMemorySessionStore.java | 20 ++-- .../state/internals/InMemoryWindowStore.java | 12 +-- .../internals/StateStoreRangeValidator.java | 36 +++++++ .../AbstractSessionBytesStoreTest.java | 27 ++++++ .../internals/CachingSessionStoreTest.java | 93 ++++++++++++++++++- .../internals/CachingWindowStoreTest.java | 4 +- .../ChangeLoggingSessionBytesStoreTest.java | 42 ++++++++- .../ChangeLoggingWindowBytesStoreTest.java | 26 ++++++ 12 files changed, 244 insertions(+), 91 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/StateStoreRangeValidator.java diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java index 7c581c91e6ca4..3bb6a8c3a4b49 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java @@ -24,8 +24,6 @@ import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.LinkedList; import java.util.List; @@ -41,8 +39,6 @@ public class CachingKeyValueStore extends WrappedStateStore, byte[], byte[]> implements KeyValueStore, CachedStateStore { - private static final Logger LOG = LoggerFactory.getLogger(CachingKeyValueStore.class); - private CacheFlushListener flushListener; private boolean sendOldValues; private String cacheName; @@ -239,12 +235,7 @@ private byte[] getInternal(final Bytes key) { @Override public KeyValueIterator range(final Bytes from, final Bytes to) { - if (from.compareTo(to) > 0) { - LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " - + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + - "Note that the built-in numerical serdes do not follow this for negative numbers"); - return KeyValueIterators.emptyIterator(); - } + if (StateStoreRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); validateStoreOpen(); final KeyValueIterator storeIterator = wrapped().range(from, to); @@ -255,12 +246,7 @@ public KeyValueIterator range(final Bytes from, @Override public KeyValueIterator reverseRange(final Bytes from, final Bytes to) { - if (from.compareTo(to) > 0) { - LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " - + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + - "Note that the built-in numerical serdes do not follow this for negative numbers"); - return KeyValueIterators.emptyIterator(); - } + if (StateStoreRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); validateStoreOpen(); final KeyValueIterator storeIterator = wrapped().reverseRange(from, to); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java index 5a91b5c397fde..d3854a09505ba 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java @@ -27,8 +27,6 @@ import org.apache.kafka.streams.processor.internals.RecordQueue; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.SessionStore; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.LinkedList; import java.util.NoSuchElementException; @@ -41,8 +39,6 @@ class CachingSessionStore extends WrappedStateStore, byte[], byte[]> implements SessionStore, CachedStateStore { - private static final Logger LOG = LoggerFactory.getLogger(CachingSessionStore.class); - private final SessionKeySchema keySchema; private final SegmentedCacheFunction cacheFunction; private String cacheName; @@ -194,12 +190,7 @@ public KeyValueIterator, byte[]> findSessions(final Bytes keyFro final Bytes keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { - 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(); - } + if (StateStoreRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); validateStoreOpen(); @@ -222,12 +213,7 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte final Bytes keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { - 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(); - } + if (StateStoreRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); validateStoreOpen(); 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 8bf5cf56cbd52..48bbb8f58a463 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 @@ -31,8 +31,6 @@ import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.time.Instant; import java.util.LinkedList; @@ -47,8 +45,6 @@ class CachingWindowStore extends WrappedStateStore, byte[], byte[]> implements WindowStore, CachedStateStore { - private static final Logger LOG = LoggerFactory.getLogger(CachingWindowStore.class); - private final long windowSize; private final SegmentedBytesStore.KeySchema keySchema = new WindowKeySchema(); @@ -256,12 +252,7 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to, final long timeFrom, final long timeTo) { - if (from.compareTo(to) > 0) { - LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " - + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + - "Note that the built-in numerical serdes do not follow this for negative numbers"); - return KeyValueIterators.emptyIterator(); - } + if (StateStoreRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); // since this function may not access the underlying inner store, we need to validate // if store is open outside as well. @@ -299,12 +290,7 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Bytes to, final Instant fromTime, final Instant toTime) { - if (from.compareTo(to) > 0) { - LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " - + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + - "Note that the built-in numerical serdes do not follow this for negative numbers"); - return KeyValueIterators.emptyIterator(); - } + if (StateStoreRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")); final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime")); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java index 137edfec36a69..1b71b45b4d3d3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java @@ -16,11 +16,6 @@ */ package org.apache.kafka.streams.state.internals; -import java.util.List; -import java.util.NavigableMap; -import java.util.Set; -import java.util.TreeMap; -import java.util.TreeSet; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; @@ -29,8 +24,11 @@ import org.apache.kafka.streams.state.KeyValueStore; import java.util.Iterator; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import java.util.List; +import java.util.NavigableMap; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; public class InMemoryKeyValueStore implements KeyValueStore { private final String name; @@ -38,8 +36,6 @@ public class InMemoryKeyValueStore implements KeyValueStore { private volatile boolean open = false; private long size = 0L; // SkipListMap#size is O(N) so we just do our best to track it - private static final Logger LOG = LoggerFactory.getLogger(InMemoryKeyValueStore.class); - public InMemoryKeyValueStore(final String name) { this.name = name; } @@ -119,12 +115,7 @@ public KeyValueIterator reverseRange(final Bytes from, final Byte } KeyValueIterator range(final Bytes from, final Bytes to, final boolean reverse) { - if (from.compareTo(to) > 0) { - LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " - + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + - "Note that the built-in numerical serdes do not follow this for negative numbers"); - return KeyValueIterators.emptyIterator(); - } + if (StateStoreRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); return new DelegatingPeekingKeyValueIterator<>( name, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index 25d11d1a1c551..5b94975a7bd34 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -164,7 +164,8 @@ public KeyValueIterator, byte[]> findSessions(final Bytes key, removeExpiredSegments(); - return registerNewIterator(key, + return registerNewIterator( + key, key, latestSessionStartTime, endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator()); @@ -178,7 +179,8 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte removeExpiredSegments(); - return registerNewIterator(key, + return registerNewIterator( + key, key, earliestSessionEndTime, endTimeMap.headMap(latestSessionStartTime, true).descendingMap().entrySet().iterator()); @@ -194,12 +196,7 @@ public KeyValueIterator, byte[]> findSessions(final Bytes keyFro removeExpiredSegments(); - 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(); - } + if (StateStoreRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); return registerNewIterator(keyFrom, keyTo, @@ -217,12 +214,7 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte removeExpiredSegments(); - 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(); - } + if (StateStoreRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); return registerNewIterator(keyFrom, keyTo, 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 25d077c477964..0fa9986c3734d 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 @@ -104,9 +104,8 @@ public void init(final ProcessorContext context, final StateStore root) { ); if (root != null) { - context.register(root, (key, value) -> { - put(Bytes.wrap(extractStoreKeyBytes(key)), value, extractStoreTimestamp(key)); - }); + context.register(root, (key, value) -> + put(Bytes.wrap(extractStoreKeyBytes(key)), value, extractStoreTimestamp(key))); } open = true; } @@ -223,12 +222,7 @@ KeyValueIterator, byte[]> fetch(final Bytes from, removeExpiredSegments(); - if (from.compareTo(to) > 0) { - LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " - + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + - "Note that the built-in numerical serdes do not follow this for negative numbers"); - return KeyValueIterators.emptyIterator(); - } + if (StateStoreRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); // add one b/c records expire exactly retentionPeriod ms after created final long minTime = Math.max(timeFrom, observedStreamTime - retentionPeriod + 1); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StateStoreRangeValidator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StateStoreRangeValidator.java new file mode 100644 index 0000000000000..0d2eca943587b --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StateStoreRangeValidator.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.common.utils.Bytes; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class StateStoreRangeValidator { + private static final Logger LOG = LoggerFactory.getLogger(StateStoreRangeValidator.class); + + static boolean isInvalid(final Bytes from, final Bytes to) { + if (from.compareTo(to) > 0) { + LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + + "This may be due to serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + + "Note that the built-in numerical serdes do not follow this for negative numbers"); + return true; + } + return false; + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java index 3147c80f3f012..5ba79034ff296 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java @@ -133,6 +133,33 @@ public void shouldPutAndFindSessionsInRange() { } } + @Test + public void shouldPutAndBackwardFindSessionsInRange() { + final String key = "a"; + final Windowed a1 = new Windowed<>(key, new SessionWindow(10, 10L)); + final Windowed a2 = new Windowed<>(key, new SessionWindow(500L, 1000L)); + sessionStore.put(a1, 1L); + sessionStore.put(a2, 2L); + sessionStore.put(new Windowed<>(key, new SessionWindow(1500L, 2000L)), 1L); + sessionStore.put(new Windowed<>(key, new SessionWindow(2500L, 3000L)), 2L); + + final List, Long>> expected = + Arrays.asList(KeyValue.pair(a1, 1L), KeyValue.pair(a2, 2L)); + + try (final KeyValueIterator, Long> values = sessionStore.backwardFindSessions(key, 0, 1000L) + ) { + assertEquals(new HashSet<>(expected), toSet(values)); + } + + final List, Long>> expected2 = + Collections.singletonList(KeyValue.pair(a2, 2L)); + + try (final KeyValueIterator, Long> values2 = sessionStore.backwardFindSessions(key, 400L, 600L) + ) { + assertEquals(new HashSet<>(expected2), toSet(values2)); + } + } + @Test public void shouldFetchAllSessionsWithSameRecordKey() { final List, Long>> expected = Arrays.asList( diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java index 5283c38e7eebd..ba41b4b1804da 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java @@ -130,6 +130,21 @@ public void shouldPutFetchAllKeysFromCache() { assertFalse(all.hasNext()); } + @Test + public void shouldPutBackwardFetchAllKeysFromCache() { + cachingStore.put(new Windowed<>(keyA, new SessionWindow(0, 0)), "1".getBytes()); + cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "1".getBytes()); + cachingStore.put(new Windowed<>(keyB, new SessionWindow(0, 0)), "1".getBytes()); + + assertEquals(3, cache.size()); + + final KeyValueIterator, byte[]> all = cachingStore.backwardFindSessions(keyA, keyB, 0, 0); + verifyWindowedKeyValue(all.next(), new Windowed<>(keyB, new SessionWindow(0, 0)), "1"); + verifyWindowedKeyValue(all.next(), new Windowed<>(keyAA, new SessionWindow(0, 0)), "1"); + verifyWindowedKeyValue(all.next(), new Windowed<>(keyA, new SessionWindow(0, 0)), "1"); + assertFalse(all.hasNext()); + } + @Test public void shouldCloseWrappedStoreAndCacheAfterErrorDuringCacheFlush() { setUpCloseTests(); @@ -203,6 +218,20 @@ public void shouldPutFetchRangeFromCache() { assertFalse(some.hasNext()); } + @Test + public void shouldPutBackwardFetchRangeFromCache() { + cachingStore.put(new Windowed<>(keyA, new SessionWindow(0, 0)), "1".getBytes()); + cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "1".getBytes()); + cachingStore.put(new Windowed<>(keyB, new SessionWindow(0, 0)), "1".getBytes()); + + assertEquals(3, cache.size()); + + final KeyValueIterator, byte[]> some = cachingStore.backwardFindSessions(keyAA, keyB, 0, 0); + verifyWindowedKeyValue(some.next(), new Windowed<>(keyB, new SessionWindow(0, 0)), "1"); + verifyWindowedKeyValue(some.next(), new Windowed<>(keyAA, new SessionWindow(0, 0)), "1"); + assertFalse(some.hasNext()); + } + @Test public void shouldFetchAllSessionsWithSameRecordKey() { final List, byte[]>> expected = asList( @@ -222,6 +251,26 @@ public void shouldFetchAllSessionsWithSameRecordKey() { verifyKeyValueList(expected, results); } + @Test + public void shouldBackwardFetchAllSessionsWithSameRecordKey() { + final List, byte[]>> expected = asList( + KeyValue.pair(new Windowed<>(keyA, new SessionWindow(0, 0)), "1".getBytes()), + KeyValue.pair(new Windowed<>(keyA, new SessionWindow(10, 10)), "2".getBytes()), + KeyValue.pair(new Windowed<>(keyA, new SessionWindow(100, 100)), "3".getBytes()), + KeyValue.pair(new Windowed<>(keyA, new SessionWindow(1000, 1000)), "4".getBytes()) + ); + for (final KeyValue, byte[]> kv : expected) { + cachingStore.put(kv.key, kv.value); + } + + // add one that shouldn't appear in the results + cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "5".getBytes()); + + final List, byte[]>> results = toList(cachingStore.backwardFetch(keyA)); + Collections.reverse(results); + verifyKeyValueList(expected, results); + } + @Test public void shouldFlushItemsToStoreOnEviction() { final List, byte[]>> added = addSessionsUntilOverflow("a", "b", "c", "d"); @@ -300,6 +349,29 @@ public void shouldFetchRangeCorrectlyAcrossSegments() { assertEquals(mkSet(a1, a2, a3, aa1, aa3), keys); } + @Test + public void shouldBackwardFetchRangeCorrectlyAcrossSegments() { + final Windowed a1 = new Windowed<>(keyA, new SessionWindow(SEGMENT_INTERVAL * 0, SEGMENT_INTERVAL * 0)); + final Windowed aa1 = new Windowed<>(keyAA, new SessionWindow(SEGMENT_INTERVAL * 0, SEGMENT_INTERVAL * 0)); + final Windowed a2 = new Windowed<>(keyA, new SessionWindow(SEGMENT_INTERVAL * 1, SEGMENT_INTERVAL * 1)); + final Windowed a3 = new Windowed<>(keyA, new SessionWindow(SEGMENT_INTERVAL * 2, SEGMENT_INTERVAL * 2)); + final Windowed aa3 = new Windowed<>(keyAA, new SessionWindow(SEGMENT_INTERVAL * 2, SEGMENT_INTERVAL * 2)); + cachingStore.put(a1, "1".getBytes()); + cachingStore.put(aa1, "1".getBytes()); + cachingStore.put(a2, "2".getBytes()); + cachingStore.put(a3, "3".getBytes()); + cachingStore.put(aa3, "3".getBytes()); + + final KeyValueIterator, byte[]> rangeResults = + cachingStore.backwardFindSessions(keyA, keyAA, 0, SEGMENT_INTERVAL * 2); + final Set> keys = new HashSet<>(); + while (rangeResults.hasNext()) { + keys.add(rangeResults.next().key); + } + rangeResults.close(); + assertEquals(mkSet(a1, a2, a3, aa1, aa3), keys); + } + @Test public void shouldSetFlushListener() { assertTrue(cachingStore.setFlushListener(null, true)); @@ -519,12 +591,31 @@ public void shouldThrowNullPointerExceptionOnPutNullKey() { cachingStore.put(null, "1".getBytes()); } + @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(StateStoreRangeValidator.class)) { + final KeyValueIterator, byte[]> iterator = cachingStore.backwardFindSessions(keyFrom, keyTo, 0L, 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 shouldNotThrowInvalidRangeExceptionWithNegativeFromKey() { final Bytes keyFrom = Bytes.wrap(Serdes.Integer().serializer().serialize("", -1)); final Bytes keyTo = Bytes.wrap(Serdes.Integer().serializer().serialize("", 1)); - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(CachingSessionStore.class)) { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StateStoreRangeValidator.class)) { final KeyValueIterator, byte[]> iterator = cachingStore.findSessions(keyFrom, keyTo, 0L, 10L); assertFalse(iterator.hasNext()); 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 7cf6293e87a12..a82cf0e498d45 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 @@ -796,7 +796,7 @@ public void shouldNotThrowInvalidRangeExceptionWithNegativeFromKey() { final Bytes keyFrom = Bytes.wrap(Serdes.Integer().serializer().serialize("", -1)); final Bytes keyTo = Bytes.wrap(Serdes.Integer().serializer().serialize("", 1)); - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(CachingWindowStore.class)) { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StateStoreRangeValidator.class)) { final KeyValueIterator, byte[]> iterator = cachingStore.fetch(keyFrom, keyTo, 0L, 10L); assertFalse(iterator.hasNext()); @@ -815,7 +815,7 @@ public void shouldNotThrowInvalidBackwardRangeExceptionWithNegativeFromKey() { final Bytes keyFrom = Bytes.wrap(Serdes.Integer().serializer().serialize("", -1)); final Bytes keyTo = Bytes.wrap(Serdes.Integer().serializer().serialize("", 1)); - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(CachingWindowStore.class)) { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StateStoreRangeValidator.class)) { final KeyValueIterator, byte[]> iterator = cachingStore.backwardFetch(keyFrom, keyTo, Instant.ofEpochMilli(0L), Instant.ofEpochMilli(10L)); assertFalse(iterator.hasNext()); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java index 5ab035cf74d27..fd1252761a8d5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java @@ -109,6 +109,16 @@ public void shouldDelegateToUnderlyingStoreWhenFetching() { EasyMock.verify(inner); } + @Test + public void shouldDelegateToUnderlyingStoreWhenBackwardFetching() { + EasyMock.expect(inner.backwardFetch(bytesKey)).andReturn(KeyValueIterators.emptyIterator()); + + init(); + + store.backwardFetch(bytesKey); + EasyMock.verify(inner); + } + @Test public void shouldDelegateToUnderlyingStoreWhenFetchingRange() { EasyMock.expect(inner.fetch(bytesKey, bytesKey)).andReturn(KeyValueIterators.emptyIterator()); @@ -119,6 +129,16 @@ public void shouldDelegateToUnderlyingStoreWhenFetchingRange() { EasyMock.verify(inner); } + @Test + public void shouldDelegateToUnderlyingStoreWhenBackwardFetchingRange() { + EasyMock.expect(inner.backwardFetch(bytesKey, bytesKey)).andReturn(KeyValueIterators.emptyIterator()); + + init(); + + store.backwardFetch(bytesKey, bytesKey); + EasyMock.verify(inner); + } + @Test public void shouldDelegateToUnderlyingStoreWhenFindingSessions() { EasyMock.expect(inner.findSessions(bytesKey, 0, 1)).andReturn(KeyValueIterators.emptyIterator()); @@ -129,6 +149,16 @@ public void shouldDelegateToUnderlyingStoreWhenFindingSessions() { EasyMock.verify(inner); } + @Test + public void shouldDelegateToUnderlyingStoreWhenBackwardFindingSessions() { + EasyMock.expect(inner.backwardFindSessions(bytesKey, 0, 1)).andReturn(KeyValueIterators.emptyIterator()); + + init(); + + store.backwardFindSessions(bytesKey, 0, 1); + EasyMock.verify(inner); + } + @Test public void shouldDelegateToUnderlyingStoreWhenFindingSessionRange() { EasyMock.expect(inner.findSessions(bytesKey, bytesKey, 0, 1)).andReturn(KeyValueIterators.emptyIterator()); @@ -139,6 +169,16 @@ public void shouldDelegateToUnderlyingStoreWhenFindingSessionRange() { EasyMock.verify(inner); } + @Test + public void shouldDelegateToUnderlyingStoreWhenBackwardFindingSessionRange() { + EasyMock.expect(inner.backwardFindSessions(bytesKey, bytesKey, 0, 1)).andReturn(KeyValueIterators.emptyIterator()); + + init(); + + store.backwardFindSessions(bytesKey, bytesKey, 0, 1); + EasyMock.verify(inner); + } + @Test public void shouldFlushUnderlyingStore() { inner.flush(); @@ -160,6 +200,4 @@ public void shouldCloseUnderlyingStore() { store.close(); EasyMock.verify(inner); } - - } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java index f4cb5232746eb..7bc1e92cd3472 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java @@ -30,6 +30,8 @@ import org.junit.Test; import org.junit.runner.RunWith; +import java.time.Instant; + import static java.time.Instant.ofEpochMilli; @RunWith(EasyMockRunner.class) @@ -95,6 +97,18 @@ public void shouldDelegateToUnderlyingStoreWhenFetching() { EasyMock.verify(inner); } + @Test + public void shouldDelegateToUnderlyingStoreWhenBackwardFetching() { + EasyMock + .expect(inner.backwardFetch(bytesKey, Instant.ofEpochMilli(0), Instant.ofEpochMilli(10))) + .andReturn(KeyValueIterators.emptyWindowStoreIterator()); + + init(); + + store.backwardFetch(bytesKey, ofEpochMilli(0), ofEpochMilli(10)); + EasyMock.verify(inner); + } + @Test public void shouldDelegateToUnderlyingStoreWhenFetchingRange() { EasyMock @@ -107,6 +121,18 @@ public void shouldDelegateToUnderlyingStoreWhenFetchingRange() { EasyMock.verify(inner); } + @Test + public void shouldDelegateToUnderlyingStoreWhenBackwardFetchingRange() { + EasyMock + .expect(inner.backwardFetch(bytesKey, bytesKey, Instant.ofEpochMilli(0), Instant.ofEpochMilli(1))) + .andReturn(KeyValueIterators.emptyIterator()); + + init(); + + store.backwardFetch(bytesKey, bytesKey, ofEpochMilli(0), ofEpochMilli(1)); + EasyMock.verify(inner); + } + @Test @SuppressWarnings("deprecation") public void shouldRetainDuplicatesWhenSet() { From 9d6e147fd85ad277bba7ea73932a6c245966f271 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 29 Jul 2020 13:04:48 +0100 Subject: [PATCH 21/24] fix in memmory session store backward iterators --- .../state/internals/InMemorySessionStore.java | 74 +++++-- .../AbstractSessionBytesStoreTest.java | 183 ++++++++++++++++-- 2 files changed, 226 insertions(+), 31 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index 5b94975a7bd34..3fb18fe03e6da 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -168,7 +168,8 @@ public KeyValueIterator, byte[]> findSessions(final Bytes key, key, key, latestSessionStartTime, - endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator()); + endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator(), + false); } @Override @@ -182,8 +183,9 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte return registerNewIterator( key, key, - earliestSessionEndTime, - endTimeMap.headMap(latestSessionStartTime, true).descendingMap().entrySet().iterator()); + latestSessionStartTime, + endTimeMap.tailMap(earliestSessionEndTime, true).descendingMap().entrySet().iterator(), + true); } @Override @@ -198,10 +200,12 @@ public KeyValueIterator, byte[]> findSessions(final Bytes keyFro if (StateStoreRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); - return registerNewIterator(keyFrom, + return registerNewIterator( + keyFrom, keyTo, latestSessionStartTime, - endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator()); + endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator(), + false); } @Override @@ -216,10 +220,12 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte if (StateStoreRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); - return registerNewIterator(keyFrom, + return registerNewIterator( + keyFrom, keyTo, - earliestSessionEndTime, - endTimeMap.headMap(latestSessionStartTime, true).entrySet().iterator()); + latestSessionStartTime, + endTimeMap.tailMap(earliestSessionEndTime, true).descendingMap().entrySet().iterator(), + true); } @Override @@ -229,7 +235,7 @@ public KeyValueIterator, byte[]> fetch(final Bytes key) { removeExpiredSegments(); - return registerNewIterator(key, key, Long.MAX_VALUE, endTimeMap.entrySet().iterator()); + return registerNewIterator(key, key, Long.MAX_VALUE, endTimeMap.entrySet().iterator(), false); } @Override @@ -239,7 +245,7 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes key) removeExpiredSegments(); - return registerNewIterator(key, key, Long.MAX_VALUE, endTimeMap.descendingMap().entrySet().iterator()); + return registerNewIterator(key, key, Long.MAX_VALUE, endTimeMap.descendingMap().entrySet().iterator(), true); } @Override @@ -250,7 +256,7 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, final B removeExpiredSegments(); - return registerNewIterator(from, to, Long.MAX_VALUE, endTimeMap.entrySet().iterator()); + return registerNewIterator(from, to, Long.MAX_VALUE, endTimeMap.entrySet().iterator(), false); } @Override @@ -260,7 +266,7 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes from, removeExpiredSegments(); - return registerNewIterator(from, to, Long.MAX_VALUE, endTimeMap.descendingMap().entrySet().iterator()); + return registerNewIterator(from, to, Long.MAX_VALUE, endTimeMap.descendingMap().entrySet().iterator(), true); } @Override @@ -305,8 +311,16 @@ private void removeExpiredSegments() { private InMemorySessionStoreIterator registerNewIterator(final Bytes keyFrom, final Bytes keyTo, final long latestSessionStartTime, - final Iterator>>> endTimeIterator) { - final InMemorySessionStoreIterator iterator = new InMemorySessionStoreIterator(keyFrom, keyTo, latestSessionStartTime, endTimeIterator, openIterators::remove); + final Iterator>>> endTimeIterator, + final boolean backward) { + final InMemorySessionStoreIterator iterator = + new InMemorySessionStoreIterator( + keyFrom, + keyTo, + latestSessionStartTime, + endTimeIterator, + openIterators::remove, + backward); openIterators.add(iterator); return iterator; } @@ -331,17 +345,20 @@ private static class InMemorySessionStoreIterator implements KeyValueIterator>>> endTimeIterator, - final ClosingCallback callback) { + final ClosingCallback callback, + final boolean backward) { this.keyFrom = keyFrom; this.keyTo = keyTo; this.latestSessionStartTime = latestSessionStartTime; - this.endTimeIterator = endTimeIterator; this.callback = callback; + this.backward = backward; setAllIterators(); } @@ -412,7 +429,15 @@ private void setAllIterators() { while (endTimeIterator.hasNext()) { final Entry>> nextEndTimeEntry = endTimeIterator.next(); currentEndTime = nextEndTimeEntry.getKey(); - keyIterator = nextEndTimeEntry.getValue().subMap(keyFrom, true, keyTo, true).entrySet().iterator(); + final Set>> entries; + if (backward) entries = nextEndTimeEntry.getValue() + .subMap(keyFrom, true, keyTo, true) + .descendingMap() + .entrySet(); + else entries = nextEndTimeEntry.getValue() + .subMap(keyFrom, true, keyTo, true) + .entrySet(); + keyIterator = entries.iterator(); if (setInnerIterators()) { return; @@ -429,9 +454,20 @@ private boolean setInnerIterators() { currentKey = nextKeyEntry.getKey(); if (latestSessionStartTime == Long.MAX_VALUE) { - recordIterator = nextKeyEntry.getValue().entrySet().iterator(); + final Set> entries; + if (backward) entries = nextKeyEntry.getValue().descendingMap().entrySet(); + else entries = nextKeyEntry.getValue().entrySet(); + recordIterator = entries.iterator(); } else { - recordIterator = nextKeyEntry.getValue().headMap(latestSessionStartTime, true).entrySet().iterator(); + final Set> entries; + if (backward) entries = nextKeyEntry.getValue() + .headMap(latestSessionStartTime, true) + .descendingMap() + .entrySet(); + else entries = nextKeyEntry.getValue() + .headMap(latestSessionStartTime, true) + .entrySet(); + recordIterator = entries.iterator(); } if (recordIterator.hasNext()) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java index 5ba79034ff296..97eff79334891 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java @@ -119,16 +119,16 @@ public void shouldPutAndFindSessionsInRange() { final List, Long>> expected = Arrays.asList(KeyValue.pair(a1, 1L), KeyValue.pair(a2, 2L)); - try (final KeyValueIterator, Long> values = sessionStore.findSessions(key, 0, 1000L) - ) { + try (final KeyValueIterator, Long> values = + sessionStore.findSessions(key, 0, 1000L)) { assertEquals(new HashSet<>(expected), toSet(values)); } final List, Long>> expected2 = Collections.singletonList(KeyValue.pair(a2, 2L)); - try (final KeyValueIterator, Long> values2 = sessionStore.findSessions(key, 400L, 600L) - ) { + try (final KeyValueIterator, Long> values2 = + sessionStore.findSessions(key, 400L, 600L)) { assertEquals(new HashSet<>(expected2), toSet(values2)); } } @@ -146,16 +146,16 @@ public void shouldPutAndBackwardFindSessionsInRange() { final List, Long>> expected = Arrays.asList(KeyValue.pair(a1, 1L), KeyValue.pair(a2, 2L)); - try (final KeyValueIterator, Long> values = sessionStore.backwardFindSessions(key, 0, 1000L) - ) { + try (final KeyValueIterator, Long> values = + sessionStore.backwardFindSessions(key, 0, 1000L)) { assertEquals(new HashSet<>(expected), toSet(values)); } final List, Long>> expected2 = Collections.singletonList(KeyValue.pair(a2, 2L)); - try (final KeyValueIterator, Long> values2 = sessionStore.backwardFindSessions(key, 400L, 600L) - ) { + try (final KeyValueIterator, Long> values2 = + sessionStore.backwardFindSessions(key, 400L, 600L)) { assertEquals(new HashSet<>(expected2), toSet(values2)); } } @@ -180,6 +180,26 @@ public void shouldFetchAllSessionsWithSameRecordKey() { } } + @Test + public void shouldBackwardFetchAllSessionsWithSameRecordKey() { + final List, Long>> expected = Arrays.asList( + KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 1L), + KeyValue.pair(new Windowed<>("a", new SessionWindow(10, 10)), 2L), + KeyValue.pair(new Windowed<>("a", new SessionWindow(100, 100)), 3L), + KeyValue.pair(new Windowed<>("a", new SessionWindow(1000, 1000)), 4L)); + + for (final KeyValue, Long> kv : expected) { + sessionStore.put(kv.key, kv.value); + } + + // add one that shouldn't appear in the results + sessionStore.put(new Windowed<>("aa", new SessionWindow(0, 0)), 5L); + + try (final KeyValueIterator, Long> values = sessionStore.backwardFetch("a")) { + assertEquals(new HashSet<>(expected), toSet(values)); + } + } + @Test public void shouldFetchAllSessionsWithinKeyRange() { final List, Long>> expected = Arrays.asList( @@ -202,6 +222,28 @@ public void shouldFetchAllSessionsWithinKeyRange() { } } + @Test + public void shouldBackwardFetchAllSessionsWithinKeyRange() { + final List, Long>> expected = Arrays.asList( + KeyValue.pair(new Windowed<>("aa", new SessionWindow(10, 10)), 2L), + KeyValue.pair(new Windowed<>("b", new SessionWindow(1000, 1000)), 4L), + + KeyValue.pair(new Windowed<>("aaa", new SessionWindow(100, 100)), 3L), + KeyValue.pair(new Windowed<>("bb", new SessionWindow(1500, 2000)), 5L)); + + for (final KeyValue, Long> kv : expected) { + sessionStore.put(kv.key, kv.value); + } + + // add some that shouldn't appear in the results + sessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L); + sessionStore.put(new Windowed<>("bbb", new SessionWindow(2500, 3000)), 6L); + + try (final KeyValueIterator, Long> values = sessionStore.backwardFetch("aa", "bb")) { + assertEquals(new HashSet<>(expected), toSet(values)); + } + } + @Test public void shouldFetchExactSession() { sessionStore.put(new Windowed<>("a", new SessionWindow(0, 4)), 1L); @@ -234,6 +276,21 @@ public void shouldFindValuesWithinMergingSessionWindowRange() { } } + @Test + public void shouldBackwardFindValuesWithinMergingSessionWindowRange() { + final String key = "a"; + sessionStore.put(new Windowed<>(key, new SessionWindow(0L, 0L)), 1L); + sessionStore.put(new Windowed<>(key, new SessionWindow(1000L, 1000L)), 2L); + + final List, Long>> expected = Arrays.asList( + KeyValue.pair(new Windowed<>(key, new SessionWindow(0L, 0L)), 1L), + KeyValue.pair(new Windowed<>(key, new SessionWindow(1000L, 1000L)), 2L)); + + try (final KeyValueIterator, Long> results = sessionStore.backwardFindSessions(key, -1, 1000L)) { + assertEquals(new HashSet<>(expected), toSet(results)); + } + } + @Test public void shouldRemove() { sessionStore.put(new Windowed<>("a", new SessionWindow(0, 1000)), 1L); @@ -287,6 +344,27 @@ public void shouldFindSessionsToMerge() { } } + @Test + public void shouldBackwardFindSessionsToMerge() { + final Windowed session1 = new Windowed<>("a", new SessionWindow(0, 100)); + final Windowed session2 = new Windowed<>("a", new SessionWindow(101, 200)); + final Windowed session3 = new Windowed<>("a", new SessionWindow(201, 300)); + final Windowed session4 = new Windowed<>("a", new SessionWindow(301, 400)); + final Windowed session5 = new Windowed<>("a", new SessionWindow(401, 500)); + sessionStore.put(session1, 1L); + sessionStore.put(session2, 2L); + sessionStore.put(session3, 3L); + sessionStore.put(session4, 4L); + sessionStore.put(session5, 5L); + + final List, Long>> expected = + Arrays.asList(KeyValue.pair(session2, 2L), KeyValue.pair(session3, 3L)); + + try (final KeyValueIterator, Long> results = sessionStore.backwardFindSessions("a", 150, 300)) { + assertEquals(new HashSet<>(expected), toSet(results)); + } + } + @Test public void shouldFetchExactKeys() { sessionStore = buildSessionStore(0x7a00000000000000L, Serdes.String(), Serdes.Long()); @@ -300,25 +378,62 @@ public void shouldFetchExactKeys() { new SessionWindow(0x7a00000000000000L - 2, 0x7a00000000000000L - 1)), 5L); try (final KeyValueIterator, Long> iterator = - sessionStore.findSessions("a", 0, Long.MAX_VALUE) + sessionStore.findSessions("a", 0, Long.MAX_VALUE) + ) { + assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 3L, 5L)))); + } + + try (final KeyValueIterator, Long> iterator = + sessionStore.findSessions("aa", 0, Long.MAX_VALUE) + ) { + assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(2L, 4L)))); + } + + try (final KeyValueIterator, Long> iterator = + sessionStore.findSessions("a", "aa", 0, Long.MAX_VALUE) + ) { + assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 2L, 3L, 4L, 5L)))); + } + + try (final KeyValueIterator, Long> iterator = + sessionStore.findSessions("a", "aa", 10, 0) + ) { + assertThat(valuesToSet(iterator), equalTo(new HashSet<>(Collections.singletonList(2L)))); + } + } + + @Test + public void shouldBackwardFetchExactKeys() { + sessionStore = buildSessionStore(0x7a00000000000000L, Serdes.String(), Serdes.Long()); + sessionStore.init(context, sessionStore); + + sessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L); + sessionStore.put(new Windowed<>("aa", new SessionWindow(0, 10)), 2L); + sessionStore.put(new Windowed<>("a", new SessionWindow(10, 20)), 3L); + sessionStore.put(new Windowed<>("aa", new SessionWindow(10, 20)), 4L); + sessionStore.put(new Windowed<>("a", + new SessionWindow(0x7a00000000000000L - 2, 0x7a00000000000000L - 1)), 5L); + + try (final KeyValueIterator, Long> iterator = + sessionStore.backwardFindSessions("a", 0, Long.MAX_VALUE) ) { assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 3L, 5L)))); } try (final KeyValueIterator, Long> iterator = - sessionStore.findSessions("aa", 0, Long.MAX_VALUE) + sessionStore.backwardFindSessions("aa", 0, Long.MAX_VALUE) ) { assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(2L, 4L)))); } try (final KeyValueIterator, Long> iterator = - sessionStore.findSessions("a", "aa", 0, Long.MAX_VALUE) + sessionStore.backwardFindSessions("a", "aa", 0, Long.MAX_VALUE) ) { assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 2L, 3L, 4L, 5L)))); } try (final KeyValueIterator, Long> iterator = - sessionStore.findSessions("a", "aa", 10, 0) + sessionStore.backwardFindSessions("a", "aa", 10, 0) ) { assertThat(valuesToSet(iterator), equalTo(new HashSet<>(Collections.singletonList(2L)))); } @@ -353,6 +468,35 @@ public void shouldFetchAndIterateOverExactBinaryKeys() { assertThat(valuesToSet(sessionStore.findSessions(key3, 0L, Long.MAX_VALUE)), equalTo(expectedKey3)); } + @Test + public void shouldBackwardFetchAndIterateOverExactBinaryKeys() { + final SessionStore sessionStore = + buildSessionStore(RETENTION_PERIOD, Serdes.Bytes(), Serdes.String()); + + sessionStore.init(context, sessionStore); + + final Bytes key1 = Bytes.wrap(new byte[]{0}); + final Bytes key2 = Bytes.wrap(new byte[]{0, 0}); + final Bytes key3 = Bytes.wrap(new byte[]{0, 0, 0}); + + sessionStore.put(new Windowed<>(key1, new SessionWindow(1, 100)), "1"); + sessionStore.put(new Windowed<>(key2, new SessionWindow(2, 100)), "2"); + sessionStore.put(new Windowed<>(key3, new SessionWindow(3, 100)), "3"); + sessionStore.put(new Windowed<>(key1, new SessionWindow(4, 100)), "4"); + sessionStore.put(new Windowed<>(key2, new SessionWindow(5, 100)), "5"); + sessionStore.put(new Windowed<>(key3, new SessionWindow(6, 100)), "6"); + sessionStore.put(new Windowed<>(key1, new SessionWindow(7, 100)), "7"); + sessionStore.put(new Windowed<>(key2, new SessionWindow(8, 100)), "8"); + sessionStore.put(new Windowed<>(key3, new SessionWindow(9, 100)), "9"); + + final Set expectedKey1 = new HashSet<>(asList("1", "4", "7")); + assertThat(valuesToSet(sessionStore.backwardFindSessions(key1, 0L, Long.MAX_VALUE)), equalTo(expectedKey1)); + final Set expectedKey2 = new HashSet<>(asList("2", "5", "8")); + assertThat(valuesToSet(sessionStore.backwardFindSessions(key2, 0L, Long.MAX_VALUE)), equalTo(expectedKey2)); + final Set expectedKey3 = new HashSet<>(asList("3", "6", "9")); + assertThat(valuesToSet(sessionStore.backwardFindSessions(key3, 0L, Long.MAX_VALUE)), equalTo(expectedKey3)); + } + @Test public void testIteratorPeek() { sessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L); @@ -368,6 +512,21 @@ public void testIteratorPeek() { assertFalse(iterator.hasNext()); } + @Test + public void testIteratorPeekBackward() { + sessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L); + sessionStore.put(new Windowed<>("aa", new SessionWindow(0, 10)), 2L); + sessionStore.put(new Windowed<>("a", new SessionWindow(10, 20)), 3L); + sessionStore.put(new Windowed<>("aa", new SessionWindow(10, 20)), 4L); + + final KeyValueIterator, Long> iterator = sessionStore.backwardFindSessions("a", 0L, 20); + + assertEquals(iterator.peekNextKey(), new Windowed<>("a", new SessionWindow(10L, 20L))); + assertEquals(iterator.peekNextKey(), iterator.next().key); + assertEquals(iterator.peekNextKey(), iterator.next().key); + assertFalse(iterator.hasNext()); + } + @Test public void shouldRestore() { final List, Long>> expected = Arrays.asList( From 1833ace99659475d04f15697ccc654ee0833696e Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 29 Jul 2020 13:15:06 +0100 Subject: [PATCH 22/24] reformat inmem stores and tests --- .../state/internals/InMemorySessionStore.java | 2 +- .../state/internals/InMemoryWindowStore.java | 8 ++--- .../internals/AbstractKeyValueStoreTest.java | 4 +-- .../AbstractSessionBytesStoreTest.java | 21 ++++++------ .../AbstractWindowBytesStoreTest.java | 32 +++++++++---------- .../internals/InMemoryKeyValueStoreTest.java | 6 ++-- .../internals/InMemoryWindowStoreTest.java | 25 ++++++++------- 7 files changed, 49 insertions(+), 49 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index 3fb18fe03e6da..b83454e314916 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -331,7 +331,6 @@ interface ClosingCallback { private static class InMemorySessionStoreIterator implements KeyValueIterator, byte[]> { - private final Iterator>>> endTimeIterator; private Iterator>> keyIterator; private Iterator> recordIterator; @@ -342,6 +341,7 @@ private static class InMemorySessionStoreIterator implements KeyValueIterator>>> endTimeIterator; private final ClosingCallback callback; 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 0fa9986c3734d..9267e5c9787a0 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 @@ -204,9 +204,9 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, @Override public KeyValueIterator, byte[]> backwardFetch(final Bytes from, - final Bytes to, - final Instant fromTime, - final Instant toTime) { + final Bytes to, + final Instant fromTime, + final Instant toTime) { final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")); final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime")); return fetch(from, to, timeFrom, timeTo, true); @@ -378,7 +378,6 @@ interface ClosingCallback { private static abstract class InMemoryWindowStoreIteratorWrapper { - private Iterator>> segmentIterator; private Iterator> recordIterator; private KeyValue next; private long currentTime; @@ -386,6 +385,7 @@ private static abstract class InMemoryWindowStoreIteratorWrapper { private final boolean allKeys; private final Bytes keyFrom; private final Bytes keyTo; + private final Iterator>> segmentIterator; private final boolean retainDuplicates; private final ClosingCallback callback; 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 55c43c2756881..9f6e70e6a13c4 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 @@ -40,12 +40,12 @@ import java.util.Map; import static org.hamcrest.CoreMatchers.hasItem; -import static org.hamcrest.core.IsEqual.equalTo; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsEqual.equalTo; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; public abstract class AbstractKeyValueStoreTest { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java index 97eff79334891..9b72f4c9e2324 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java @@ -65,7 +65,6 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; - public abstract class AbstractSessionBytesStoreTest { static final long SEGMENT_INTERVAL = 60_000L; @@ -415,25 +414,25 @@ public void shouldBackwardFetchExactKeys() { new SessionWindow(0x7a00000000000000L - 2, 0x7a00000000000000L - 1)), 5L); try (final KeyValueIterator, Long> iterator = - sessionStore.backwardFindSessions("a", 0, Long.MAX_VALUE) + sessionStore.backwardFindSessions("a", 0, Long.MAX_VALUE) ) { assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 3L, 5L)))); } try (final KeyValueIterator, Long> iterator = - sessionStore.backwardFindSessions("aa", 0, Long.MAX_VALUE) + sessionStore.backwardFindSessions("aa", 0, Long.MAX_VALUE) ) { assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(2L, 4L)))); } try (final KeyValueIterator, Long> iterator = - sessionStore.backwardFindSessions("a", "aa", 0, Long.MAX_VALUE) + sessionStore.backwardFindSessions("a", "aa", 0, Long.MAX_VALUE) ) { assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 2L, 3L, 4L, 5L)))); } try (final KeyValueIterator, Long> iterator = - sessionStore.backwardFindSessions("a", "aa", 10, 0) + sessionStore.backwardFindSessions("a", "aa", 10, 0) ) { assertThat(valuesToSet(iterator), equalTo(new HashSet<>(Collections.singletonList(2L)))); } @@ -446,9 +445,9 @@ public void shouldFetchAndIterateOverExactBinaryKeys() { sessionStore.init(context, sessionStore); - final Bytes key1 = Bytes.wrap(new byte[]{0}); - final Bytes key2 = Bytes.wrap(new byte[]{0, 0}); - final Bytes key3 = Bytes.wrap(new byte[]{0, 0, 0}); + final Bytes key1 = Bytes.wrap(new byte[] {0}); + final Bytes key2 = Bytes.wrap(new byte[] {0, 0}); + final Bytes key3 = Bytes.wrap(new byte[] {0, 0, 0}); sessionStore.put(new Windowed<>(key1, new SessionWindow(1, 100)), "1"); sessionStore.put(new Windowed<>(key2, new SessionWindow(2, 100)), "2"); @@ -475,9 +474,9 @@ public void shouldBackwardFetchAndIterateOverExactBinaryKeys() { sessionStore.init(context, sessionStore); - final Bytes key1 = Bytes.wrap(new byte[]{0}); - final Bytes key2 = Bytes.wrap(new byte[]{0, 0}); - final Bytes key3 = Bytes.wrap(new byte[]{0, 0, 0}); + final Bytes key1 = Bytes.wrap(new byte[] {0}); + final Bytes key2 = Bytes.wrap(new byte[] {0, 0}); + final Bytes key3 = Bytes.wrap(new byte[] {0, 0, 0}); sessionStore.put(new Windowed<>(key1, new SessionWindow(1, 100)), "1"); sessionStore.put(new Windowed<>(key2, new SessionWindow(2, 100)), "2"); 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 aefd1ed380e99..4d01bf8158501 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 @@ -792,10 +792,10 @@ public void shouldFetchAndIterateOverExactKeys() { final long windowSize = 0x7a00000000000000L; final long retentionPeriod = 0x7a00000000000000L; final WindowStore windowStore = buildWindowStore(retentionPeriod, - windowSize, - false, - Serdes.String(), - Serdes.String()); + windowSize, + false, + Serdes.String(), + Serdes.String()); windowStore.init(context, windowStore); @@ -878,15 +878,15 @@ public void shouldThrowNullPointerExceptionOnRangeNullToKey() { @Test public void shouldFetchAndIterateOverExactBinaryKeys() { final WindowStore windowStore = buildWindowStore(RETENTION_PERIOD, - WINDOW_SIZE, - true, - Serdes.Bytes(), - Serdes.String()); + WINDOW_SIZE, + true, + Serdes.Bytes(), + Serdes.String()); windowStore.init(context, windowStore); - final Bytes key1 = Bytes.wrap(new byte[]{0}); - final Bytes key2 = Bytes.wrap(new byte[]{0, 0}); - final Bytes key3 = Bytes.wrap(new byte[]{0, 0, 0}); + final Bytes key1 = Bytes.wrap(new byte[] {0}); + final Bytes key2 = Bytes.wrap(new byte[] {0, 0}); + final Bytes key3 = Bytes.wrap(new byte[] {0, 0, 0}); windowStore.put(key1, "1", 0); windowStore.put(key2, "2", 0); windowStore.put(key3, "3", 0); @@ -1139,8 +1139,8 @@ public void testFetchDuplicates() { @SuppressWarnings("deprecation") private void putFirstBatch(final WindowStore store, - @SuppressWarnings("SameParameterValue") final long startTime, - final InternalMockProcessorContext context) { + @SuppressWarnings("SameParameterValue") final long startTime, + final InternalMockProcessorContext context) { context.setRecordContext(createRecordContext(startTime)); store.put(0, "zero"); context.setRecordContext(createRecordContext(startTime + 1L)); @@ -1155,8 +1155,8 @@ private void putFirstBatch(final WindowStore store, @SuppressWarnings("deprecation") private void putSecondBatch(final WindowStore store, - @SuppressWarnings("SameParameterValue") final long startTime, - final InternalMockProcessorContext context) { + @SuppressWarnings("SameParameterValue") final long startTime, + final InternalMockProcessorContext context) { context.setRecordContext(createRecordContext(startTime + 3L)); store.put(2, "two+1"); context.setRecordContext(createRecordContext(startTime + 4L)); @@ -1172,7 +1172,7 @@ private void putSecondBatch(final WindowStore store, } private Map> entriesByKey(final List> changeLog, - @SuppressWarnings("SameParameterValue") final long startTime) { + @SuppressWarnings("SameParameterValue") final long startTime) { final HashMap> entriesByKey = new HashMap<>(); for (final KeyValue entry : changeLog) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java index 62f89495c7f9d..6dc90eac8d417 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java @@ -33,9 +33,9 @@ public class InMemoryKeyValueStoreTest extends AbstractKeyValueStoreTest { @Override protected KeyValueStore createKeyValueStore(final ProcessorContext context) { final StoreBuilder> storeBuilder = Stores.keyValueStoreBuilder( - Stores.inMemoryKeyValueStore("my-store"), - (Serde) context.keySerde(), - (Serde) context.valueSerde()); + Stores.inMemoryKeyValueStore("my-store"), + (Serde) context.keySerde(), + (Serde) context.valueSerde()); final KeyValueStore store = storeBuilder.build(); store.init(context, store); 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()); } - + } From f36d126bc7512cb6708e814ab99cd2cf9706d61a Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 29 Jul 2020 14:11:54 +0100 Subject: [PATCH 23/24] improve tests --- ...alidator.java => BytesRangeValidator.java} | 5 +- .../state/internals/CachingKeyValueStore.java | 4 +- .../state/internals/CachingSessionStore.java | 4 +- .../state/internals/CachingWindowStore.java | 4 +- .../internals/InMemoryKeyValueStore.java | 2 +- .../state/internals/InMemorySessionStore.java | 4 +- .../state/internals/InMemoryWindowStore.java | 2 +- ...SortedCacheKeyValueBytesStoreIterator.java | 3 +- ...MergedSortedCacheSessionStoreIterator.java | 4 +- .../state/internals/RocksDBRangeIterator.java | 15 +-- .../state/internals/RocksDBSessionStore.java | 1 - .../state/internals/SegmentIterator.java | 2 +- .../state/internals/SessionKeySchema.java | 1 - .../TimestampedWindowStoreBuilder.java | 10 +- .../state/internals/WindowKeySchema.java | 4 +- .../internals/CachingSessionStoreTest.java | 4 +- .../internals/CachingWindowStoreTest.java | 4 +- ...edCacheKeyValueBytesStoreIteratorTest.java | 85 +++++++++++++---- ...dCacheWrappedSessionStoreIteratorTest.java | 65 +++++++++++-- ...edCacheWrappedWindowStoreIteratorTest.java | 80 +++++++++++++++- .../internals/MeteredSessionStoreTest.java | 95 ++++++++++++++++--- .../internals/MeteredWindowStoreTest.java | 11 ++- .../internals/ReadOnlyWindowStoreStub.java | 23 +++-- .../state/internals/SegmentIteratorTest.java | 16 ++-- .../state/internals/WindowKeySchemaTest.java | 64 ++++++------- .../kafka/test/ReadOnlySessionStoreStub.java | 4 +- 26 files changed, 381 insertions(+), 135 deletions(-) rename streams/src/main/java/org/apache/kafka/streams/state/internals/{StateStoreRangeValidator.java => BytesRangeValidator.java} (91%) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StateStoreRangeValidator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/BytesRangeValidator.java similarity index 91% rename from streams/src/main/java/org/apache/kafka/streams/state/internals/StateStoreRangeValidator.java rename to streams/src/main/java/org/apache/kafka/streams/state/internals/BytesRangeValidator.java index 0d2eca943587b..c55e18eea4a30 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StateStoreRangeValidator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/BytesRangeValidator.java @@ -21,8 +21,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class StateStoreRangeValidator { - private static final Logger LOG = LoggerFactory.getLogger(StateStoreRangeValidator.class); +public class BytesRangeValidator { + + private static final Logger LOG = LoggerFactory.getLogger(BytesRangeValidator.class); static boolean isInvalid(final Bytes from, final Bytes to) { if (from.compareTo(to) > 0) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java index 3bb6a8c3a4b49..48802f37e3165 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java @@ -235,7 +235,7 @@ private byte[] getInternal(final Bytes key) { @Override public KeyValueIterator range(final Bytes from, final Bytes to) { - if (StateStoreRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); + if (BytesRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); validateStoreOpen(); final KeyValueIterator storeIterator = wrapped().range(from, to); @@ -246,7 +246,7 @@ public KeyValueIterator range(final Bytes from, @Override public KeyValueIterator reverseRange(final Bytes from, final Bytes to) { - if (StateStoreRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); + if (BytesRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); validateStoreOpen(); final KeyValueIterator storeIterator = wrapped().reverseRange(from, to); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java index d3854a09505ba..ca5fe44444377 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java @@ -190,7 +190,7 @@ public KeyValueIterator, byte[]> findSessions(final Bytes keyFro final Bytes keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { - if (StateStoreRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); + if (BytesRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); validateStoreOpen(); @@ -213,7 +213,7 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte final Bytes keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { - if (StateStoreRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); + if (BytesRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); validateStoreOpen(); 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 48bbb8f58a463..1a34ebc88eedb 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 @@ -252,7 +252,7 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to, final long timeFrom, final long timeTo) { - if (StateStoreRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); + if (BytesRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); // since this function may not access the underlying inner store, we need to validate // if store is open outside as well. @@ -290,7 +290,7 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Bytes to, final Instant fromTime, final Instant toTime) { - if (StateStoreRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); + if (BytesRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); final long timeFrom = ApiUtils.validateMillisecondInstant(fromTime, prepareMillisCheckFailMsgPrefix(fromTime, "fromTime")); final long timeTo = ApiUtils.validateMillisecondInstant(toTime, prepareMillisCheckFailMsgPrefix(toTime, "toTime")); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java index 1b71b45b4d3d3..889e08c45b34d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java @@ -115,7 +115,7 @@ public KeyValueIterator reverseRange(final Bytes from, final Byte } KeyValueIterator range(final Bytes from, final Bytes to, final boolean reverse) { - if (StateStoreRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); + if (BytesRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); return new DelegatingPeekingKeyValueIterator<>( name, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index b83454e314916..062edcf618e5c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -198,7 +198,7 @@ public KeyValueIterator, byte[]> findSessions(final Bytes keyFro removeExpiredSegments(); - if (StateStoreRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); + if (BytesRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); return registerNewIterator( keyFrom, @@ -218,7 +218,7 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte removeExpiredSegments(); - if (StateStoreRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); + if (BytesRangeValidator.isInvalid(keyFrom, keyTo)) return KeyValueIterators.emptyIterator(); return registerNewIterator( keyFrom, 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 9267e5c9787a0..a8b5600ddbb39 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 @@ -222,7 +222,7 @@ KeyValueIterator, byte[]> fetch(final Bytes from, removeExpiredSegments(); - if (StateStoreRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); + if (BytesRangeValidator.isInvalid(from, to)) return KeyValueIterators.emptyIterator(); // add one b/c records expire exactly retentionPeriod ms after created final long minTime = Math.max(timeFrom, observedStreamTime - retentionPeriod + 1); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIterator.java index 54174aab33adc..5cb426edb08ca 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIterator.java @@ -24,7 +24,8 @@ * Merges two iterators. Assumes each of them is sorted by key * */ -class MergedSortedCacheKeyValueBytesStoreIterator extends AbstractMergedSortedCacheStoreIterator { +class MergedSortedCacheKeyValueBytesStoreIterator + extends AbstractMergedSortedCacheStoreIterator { MergedSortedCacheKeyValueBytesStoreIterator(final PeekingKeyValueIterator cacheIterator, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java index e1abcba77747a..abc62a04efe89 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java @@ -24,9 +24,9 @@ /** * Merges two iterators. Assumes each of them is sorted by key - * */ -class MergedSortedCacheSessionStoreIterator extends AbstractMergedSortedCacheStoreIterator, Windowed, byte[], byte[]> { +class MergedSortedCacheSessionStoreIterator + extends AbstractMergedSortedCacheStoreIterator, Windowed, byte[], byte[]> { private final SegmentedCacheFunction cacheFunction; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java index 8e058cc9b5ca9..66df25a0d3927 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java @@ -55,22 +55,15 @@ class RocksDBRangeIterator extends RocksDbIterator { @Override public KeyValue makeNext() { final KeyValue next = super.makeNext(); - if (next == null) { return allDone(); } else { if (!reverse) { - if (comparator.compare(next.key.get(), rawLastKey) <= 0) { - return next; - } else { - return allDone(); - } + if (comparator.compare(next.key.get(), rawLastKey) <= 0) return next; + else return allDone(); } else { - if (comparator.compare(next.key.get(), rawLastKey) >= 0) { - return next; - } else { - return allDone(); - } + if (comparator.compare(next.key.get(), rawLastKey) >= 0) return next; + else return allDone(); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java index 338769abea4a5..e208ac12d56d3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java @@ -21,7 +21,6 @@ import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.SessionStore; - public class RocksDBSessionStore extends WrappedStateStore implements SessionStore { 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 5d0cb3bb7437e..c0fbefbc4a7a7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java @@ -70,7 +70,7 @@ 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 { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java index 568297e0d7825..154e54a8f1a29 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 @@ -26,7 +26,6 @@ import java.nio.ByteBuffer; import java.util.List; - public class SessionKeySchema implements SegmentedBytesStore.KeySchema { private static final int TIMESTAMP_SIZE = 8; 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 a27fb4923020b..ebb1be5aa77b0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java @@ -29,16 +29,16 @@ import org.apache.kafka.streams.state.WindowBytesStoreSupplier; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.time.Instant; import java.util.Objects; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - 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; @@ -62,7 +62,7 @@ public TimestampedWindowStore build() { } } if (storeSupplier.retainDuplicates() && enableCaching) { - log.warn("Disabling caching for {} since store was configured to retain duplicates", storeSupplier.name()); + LOG.warn("Disabling caching for {} since store was configured to retain duplicates", storeSupplier.name()); enableCaching = false; } 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 0437c7a904294..a675ae490f8c9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java @@ -23,11 +23,11 @@ import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.TimeWindow; import org.apache.kafka.streams.state.StateSerdes; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; import java.util.List; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class WindowKeySchema implements RocksDBSegmentedBytesStore.KeySchema { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java index ba41b4b1804da..1babefb51313f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java @@ -596,7 +596,7 @@ public void shouldNotThrowInvalidBackwardRangeExceptionWithNegativeFromKey() { final Bytes keyFrom = Bytes.wrap(Serdes.Integer().serializer().serialize("", -1)); final Bytes keyTo = Bytes.wrap(Serdes.Integer().serializer().serialize("", 1)); - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StateStoreRangeValidator.class)) { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(BytesRangeValidator.class)) { final KeyValueIterator, byte[]> iterator = cachingStore.backwardFindSessions(keyFrom, keyTo, 0L, 10L); assertFalse(iterator.hasNext()); @@ -615,7 +615,7 @@ public void shouldNotThrowInvalidRangeExceptionWithNegativeFromKey() { final Bytes keyFrom = Bytes.wrap(Serdes.Integer().serializer().serialize("", -1)); final Bytes keyTo = Bytes.wrap(Serdes.Integer().serializer().serialize("", 1)); - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StateStoreRangeValidator.class)) { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(BytesRangeValidator.class)) { final KeyValueIterator, byte[]> iterator = cachingStore.findSessions(keyFrom, keyTo, 0L, 10L); assertFalse(iterator.hasNext()); 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 a82cf0e498d45..398744e9881b0 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 @@ -796,7 +796,7 @@ public void shouldNotThrowInvalidRangeExceptionWithNegativeFromKey() { final Bytes keyFrom = Bytes.wrap(Serdes.Integer().serializer().serialize("", -1)); final Bytes keyTo = Bytes.wrap(Serdes.Integer().serializer().serialize("", 1)); - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StateStoreRangeValidator.class)) { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(BytesRangeValidator.class)) { final KeyValueIterator, byte[]> iterator = cachingStore.fetch(keyFrom, keyTo, 0L, 10L); assertFalse(iterator.hasNext()); @@ -815,7 +815,7 @@ public void shouldNotThrowInvalidBackwardRangeExceptionWithNegativeFromKey() { final Bytes keyFrom = Bytes.wrap(Serdes.Integer().serializer().serialize("", -1)); final Bytes keyTo = Bytes.wrap(Serdes.Integer().serializer().serialize("", 1)); - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StateStoreRangeValidator.class)) { + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(BytesRangeValidator.class)) { final KeyValueIterator, byte[]> iterator = cachingStore.backwardFetch(keyFrom, keyTo, Instant.ofEpochMilli(0L), Instant.ofEpochMilli(10L)); assertFalse(iterator.hasNext()); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIteratorTest.java index 1226148538132..4fdca17da4373 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueBytesStoreIteratorTest.java @@ -17,13 +17,11 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.StateSerdes; import org.junit.Before; import org.junit.Test; @@ -33,18 +31,16 @@ public class MergedSortedCacheKeyValueBytesStoreIteratorTest { private final String namespace = "0.0-one"; - private final StateSerdes serdes = new StateSerdes<>("dummy", Serdes.ByteArray(), Serdes.ByteArray()); private KeyValueStore store; private ThreadCache cache; @Before - public void setUp() throws Exception { + public void setUp() { store = new InMemoryKeyValueStore(namespace); cache = new ThreadCache(new LogContext("testCache "), 10000L, new MockStreamsMetrics(new Metrics())); } - @Test - public void shouldIterateOverRange() throws Exception { + public void shouldIterateOverRange() { final byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}, {10}, {11}}; for (int i = 0; i < bytes.length; i += 2) { store.put(Bytes.wrap(bytes[i]), bytes[i]); @@ -53,7 +49,8 @@ public void shouldIterateOverRange() throws Exception { final Bytes from = Bytes.wrap(new byte[] {2}); final Bytes to = Bytes.wrap(new byte[] {9}); - final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("store", store.range(from, to)); + final KeyValueIterator storeIterator = + new DelegatingPeekingKeyValueIterator<>("store", store.range(from, to)); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(namespace, from, to); final MergedSortedCacheKeyValueBytesStoreIterator iterator = @@ -71,7 +68,34 @@ public void shouldIterateOverRange() throws Exception { @Test - public void shouldSkipLargerDeletedCacheValue() throws Exception { + public void shouldReverseIterateOverRange() { + final byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}, {10}, {11}}; + for (int i = 0; i < bytes.length; i += 2) { + store.put(Bytes.wrap(bytes[i]), bytes[i]); + cache.put(namespace, Bytes.wrap(bytes[i + 1]), new LRUCacheEntry(bytes[i + 1])); + } + + final Bytes from = Bytes.wrap(new byte[] {2}); + final Bytes to = Bytes.wrap(new byte[] {9}); + final KeyValueIterator storeIterator = + new DelegatingPeekingKeyValueIterator<>("store", store.reverseRange(from, to)); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.reverseRange(namespace, from, to); + + final MergedSortedCacheKeyValueBytesStoreIterator iterator = + new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator, true); + final byte[][] values = new byte[8][]; + int index = 0; + int bytesIndex = 9; + while (iterator.hasNext()) { + final byte[] value = iterator.next().value; + values[index++] = value; + assertArrayEquals(bytes[bytesIndex--], value); + } + iterator.close(); + } + + @Test + public void shouldSkipLargerDeletedCacheValue() { final byte[][] bytes = {{0}, {1}}; store.put(Bytes.wrap(bytes[0]), bytes[0]); cache.put(namespace, Bytes.wrap(bytes[1]), new LRUCacheEntry(null)); @@ -81,7 +105,7 @@ public void shouldSkipLargerDeletedCacheValue() throws Exception { } @Test - public void shouldSkipSmallerDeletedCachedValue() throws Exception { + public void shouldSkipSmallerDeletedCachedValue() { final byte[][] bytes = {{0}, {1}}; cache.put(namespace, Bytes.wrap(bytes[0]), new LRUCacheEntry(null)); store.put(Bytes.wrap(bytes[1]), bytes[1]); @@ -91,7 +115,7 @@ public void shouldSkipSmallerDeletedCachedValue() throws Exception { } @Test - public void shouldIgnoreIfDeletedInCacheButExistsInStore() throws Exception { + public void shouldIgnoreIfDeletedInCacheButExistsInStore() { final byte[][] bytes = {{0}}; cache.put(namespace, Bytes.wrap(bytes[0]), new LRUCacheEntry(null)); store.put(Bytes.wrap(bytes[0]), bytes[0]); @@ -100,7 +124,7 @@ public void shouldIgnoreIfDeletedInCacheButExistsInStore() throws Exception { } @Test - public void shouldNotHaveNextIfAllCachedItemsDeleted() throws Exception { + public void shouldNotHaveNextIfAllCachedItemsDeleted() { final byte[][] bytes = {{0}, {1}, {2}}; for (final byte[] aByte : bytes) { final Bytes aBytes = Bytes.wrap(aByte); @@ -111,7 +135,7 @@ public void shouldNotHaveNextIfAllCachedItemsDeleted() throws Exception { } @Test - public void shouldNotHaveNextIfOnlyCacheItemsAndAllDeleted() throws Exception { + public void shouldNotHaveNextIfOnlyCacheItemsAndAllDeleted() { final byte[][] bytes = {{0}, {1}, {2}}; for (final byte[] aByte : bytes) { cache.put(namespace, Bytes.wrap(aByte), new LRUCacheEntry(null)); @@ -120,7 +144,7 @@ public void shouldNotHaveNextIfOnlyCacheItemsAndAllDeleted() throws Exception { } @Test - public void shouldSkipAllDeletedFromCache() throws Exception { + public void shouldSkipAllDeletedFromCache() { final byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}, {10}, {11}}; for (final byte[] aByte : bytes) { final Bytes aBytes = Bytes.wrap(aByte); @@ -146,7 +170,7 @@ public void shouldSkipAllDeletedFromCache() throws Exception { } @Test - public void shouldPeekNextKey() throws Exception { + public void shouldPeekNextKey() { final KeyValueStore kv = new InMemoryKeyValueStore("one"); final ThreadCache cache = new ThreadCache(new LogContext("testCache "), 1000000L, new MockStreamsMetrics(new Metrics())); final byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}, {10}}; @@ -155,8 +179,8 @@ public void shouldPeekNextKey() throws Exception { cache.put(namespace, Bytes.wrap(bytes[i + 1]), new LRUCacheEntry(bytes[i + 1])); } - final Bytes from = Bytes.wrap(new byte[]{2}); - final Bytes to = Bytes.wrap(new byte[]{9}); + final Bytes from = Bytes.wrap(new byte[] {2}); + final Bytes to = Bytes.wrap(new byte[] {9}); final KeyValueIterator storeIterator = kv.range(from, to); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(namespace, from, to); @@ -174,6 +198,35 @@ public void shouldPeekNextKey() throws Exception { iterator.close(); } + @Test + public void shouldPeekNextKeyReverse() { + final KeyValueStore kv = new InMemoryKeyValueStore("one"); + final ThreadCache cache = new ThreadCache(new LogContext("testCache "), 1000000L, new MockStreamsMetrics(new Metrics())); + final byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}, {10}}; + for (int i = 0; i < bytes.length - 1; i += 2) { + kv.put(Bytes.wrap(bytes[i]), bytes[i]); + cache.put(namespace, Bytes.wrap(bytes[i + 1]), new LRUCacheEntry(bytes[i + 1])); + } + + final Bytes from = Bytes.wrap(new byte[] {2}); + final Bytes to = Bytes.wrap(new byte[] {9}); + final KeyValueIterator storeIterator = kv.reverseRange(from, to); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.reverseRange(namespace, from, to); + + final MergedSortedCacheKeyValueBytesStoreIterator iterator = + new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, storeIterator, true); + final byte[][] values = new byte[8][]; + int index = 0; + int bytesIndex = 9; + while (iterator.hasNext()) { + final byte[] keys = iterator.peekNextKey().get(); + values[index++] = keys; + assertArrayEquals(bytes[bytesIndex--], keys); + iterator.next(); + } + iterator.close(); + } + private MergedSortedCacheKeyValueBytesStoreIterator createIterator() { final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.all(namespace); final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("store", store.all()); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java index 1909da770d83d..219009a13dfb3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java @@ -45,7 +45,7 @@ public long segmentId(final Bytes key) { private final SessionWindow storeWindow = new SessionWindow(0, 1); private final Iterator, byte[]>> storeKvs = Collections.singleton( - KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey.get())).iterator(); + KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey.get())).iterator(); private final SessionWindow cacheWindow = new SessionWindow(10, 20); private final Iterator> cacheKvs = Collections.singleton( KeyValue.pair( @@ -55,56 +55,101 @@ public long segmentId(final Bytes key) { @Test public void shouldHaveNextFromStore() { - final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(storeKvs, Collections.emptyIterator()); + final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(storeKvs, Collections.emptyIterator(), false); + assertTrue(mergeIterator.hasNext()); + } + + @Test + public void shouldHaveNextFromReverseStore() { + final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(storeKvs, Collections.emptyIterator(), true); assertTrue(mergeIterator.hasNext()); } @Test public void shouldGetNextFromStore() { - final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(storeKvs, Collections.emptyIterator()); + final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(storeKvs, Collections.emptyIterator(), false); + assertThat(mergeIterator.next(), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey.get()))); + } + + @Test + public void shouldGetNextFromReverseStore() { + final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(storeKvs, Collections.emptyIterator(), true); assertThat(mergeIterator.next(), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey.get()))); } @Test public void shouldPeekNextKeyFromStore() { - final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(storeKvs, Collections.emptyIterator()); + final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(storeKvs, Collections.emptyIterator(), false); + assertThat(mergeIterator.peekNextKey(), equalTo(new Windowed<>(storeKey, storeWindow))); + } + + @Test + public void shouldPeekNextKeyFromReverseStore() { + final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(storeKvs, Collections.emptyIterator(), true); assertThat(mergeIterator.peekNextKey(), equalTo(new Windowed<>(storeKey, storeWindow))); } @Test public void shouldHaveNextFromCache() { - final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(Collections.emptyIterator(), cacheKvs); + final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(Collections.emptyIterator(), cacheKvs, false); + assertTrue(mergeIterator.hasNext()); + } + + @Test + public void shouldHaveNextFromReverseCache() { + final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(Collections.emptyIterator(), cacheKvs, true); assertTrue(mergeIterator.hasNext()); } @Test public void shouldGetNextFromCache() { - final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(Collections.emptyIterator(), cacheKvs); + final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(Collections.emptyIterator(), cacheKvs, false); + assertThat(mergeIterator.next(), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey.get()))); + } + + @Test + public void shouldGetNextFromReverseCache() { + final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(Collections.emptyIterator(), cacheKvs, true); assertThat(mergeIterator.next(), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey.get()))); } @Test public void shouldPeekNextKeyFromCache() { - final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(Collections.emptyIterator(), cacheKvs); + final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(Collections.emptyIterator(), cacheKvs, false); + assertThat(mergeIterator.peekNextKey(), equalTo(new Windowed<>(cacheKey, cacheWindow))); + } + + @Test + public void shouldPeekNextKeyFromReverseCache() { + final MergedSortedCacheSessionStoreIterator mergeIterator = createIterator(Collections.emptyIterator(), cacheKvs, true); assertThat(mergeIterator.peekNextKey(), equalTo(new Windowed<>(cacheKey, cacheWindow))); } @Test public void shouldIterateBothStoreAndCache() { - final MergedSortedCacheSessionStoreIterator iterator = createIterator(storeKvs, cacheKvs); + final MergedSortedCacheSessionStoreIterator iterator = createIterator(storeKvs, cacheKvs, false); assertThat(iterator.next(), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey.get()))); assertThat(iterator.next(), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey.get()))); assertFalse(iterator.hasNext()); } + @Test + public void shouldReverseIterateBothStoreAndCache() { + final MergedSortedCacheSessionStoreIterator iterator = createIterator(storeKvs, cacheKvs, true); + assertThat(iterator.next(), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey.get()))); + assertThat(iterator.next(), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey.get()))); + assertFalse(iterator.hasNext()); + } + private MergedSortedCacheSessionStoreIterator createIterator(final Iterator, byte[]>> storeKvs, - final Iterator> cacheKvs) { + final Iterator> cacheKvs, + final boolean reverse) { final DelegatingPeekingKeyValueIterator, byte[]> storeIterator = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(storeKvs)); final PeekingKeyValueIterator cacheIterator = new DelegatingPeekingKeyValueIterator<>("cache", new KeyValueIteratorStub<>(cacheKvs)); - return new MergedSortedCacheSessionStoreIterator(cacheIterator, storeIterator, SINGLE_SEGMENT_CACHE_FUNCTION, false); + return new MergedSortedCacheSessionStoreIterator(cacheIterator, storeIterator, SINGLE_SEGMENT_CACHE_FUNCTION, reverse); } } 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 6f12ea40d2ecc..7559b846badbf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java @@ -28,6 +28,7 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import static org.hamcrest.CoreMatchers.equalTo; @@ -84,6 +85,45 @@ public void shouldIterateOverValueFromBothIterators() { iterator.close(); } + + @Test + public void shouldReverseIterateOverValueFromBothIterators() { + final List> expectedKvPairs = new ArrayList<>(); + for (long t = 0; t < 100; t += 20) { + final byte[] v1Bytes = String.valueOf(t).getBytes(); + final KeyValue v1 = KeyValue.pair(t, v1Bytes); + windowStoreKvPairs.add(v1); + expectedKvPairs.add(KeyValue.pair(t, v1Bytes)); + final Bytes keyBytes = WindowKeySchema.toStoreKeyBinary("a", t + 10, 0, stateSerdes); + final byte[] valBytes = String.valueOf(t + 10).getBytes(); + expectedKvPairs.add(KeyValue.pair(t + 10, valBytes)); + cache.put(namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(keyBytes), new LRUCacheEntry(valBytes)); + } + + final Bytes fromBytes = WindowKeySchema.toStoreKeyBinary("a", 0, 0, stateSerdes); + final Bytes toBytes = WindowKeySchema.toStoreKeyBinary("a", 100, 0, stateSerdes); + Collections.reverse(windowStoreKvPairs); + final KeyValueIterator storeIterator = + new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); + + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.reverseRange( + namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes) + ); + + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( + cacheIterator, storeIterator, true + ); + int index = 0; + Collections.reverse(expectedKvPairs); + while (iterator.hasNext()) { + final KeyValue next = iterator.next(); + final KeyValue expected = expectedKvPairs.get(index++); + assertArrayEquals(expected.value, next.value); + assertEquals(expected.key, next.key); + } + iterator.close(); + } + @Test public void shouldPeekNextStoreKey() { windowStoreKvPairs.add(KeyValue.pair(10L, "a".getBytes())); @@ -103,6 +143,27 @@ public void shouldPeekNextStoreKey() { iterator.close(); } + @Test + public void shouldPeekNextStoreKeyReverse() { + windowStoreKvPairs.add(KeyValue.pair(10L, "a".getBytes())); + cache.put(namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(WindowKeySchema.toStoreKeyBinary("a", 0, 0, stateSerdes)), new LRUCacheEntry("b".getBytes())); + final Bytes fromBytes = WindowKeySchema.toStoreKeyBinary("a", 0, 0, stateSerdes); + final Bytes toBytes = WindowKeySchema.toStoreKeyBinary("a", 100, 0, stateSerdes); + final KeyValueIterator storeIterator = + new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.reverseRange( + namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), + SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes) + ); + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator( + cacheIterator, storeIterator, true + ); + assertThat(iterator.peekNextKey(), equalTo(10L)); + iterator.next(); + assertThat(iterator.peekNextKey(), equalTo(0L)); + iterator.close(); + } + @Test public void shouldPeekNextCacheKey() { windowStoreKvPairs.add(KeyValue.pair(0L, "a".getBytes())); @@ -117,4 +178,21 @@ public void shouldPeekNextCacheKey() { assertThat(iterator.peekNextKey(), equalTo(10L)); iterator.close(); } -} + + @Test + public void shouldPeekNextCacheKeyReverse() { + windowStoreKvPairs.add(KeyValue.pair(0L, "a".getBytes())); + cache.put(namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(WindowKeySchema.toStoreKeyBinary("a", 10L, 0, stateSerdes)), new LRUCacheEntry("b".getBytes())); + final Bytes fromBytes = WindowKeySchema.toStoreKeyBinary("a", 0, 0, stateSerdes); + final Bytes toBytes = WindowKeySchema.toStoreKeyBinary("a", 100, 0, stateSerdes); + final KeyValueIterator storeIterator = + new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = + cache.reverseRange(namespace, SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(fromBytes), SINGLE_SEGMENT_CACHE_FUNCTION.cacheKey(toBytes)); + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator(cacheIterator, storeIterator, true); + assertThat(iterator.peekNextKey(), equalTo(10L)); + iterator.next(); + assertThat(iterator.peekNextKey(), equalTo(0L)); + iterator.close(); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java index 83136f4690ace..8741694a8781e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java @@ -246,8 +246,8 @@ public void shouldWriteBytesToInnerStoreAndRecordPutMetric() { @Test public void shouldFindSessionsFromStoreAndRecordFetchMetric() { expect(innerStore.findSessions(KEY_BYTES, 0, 0)) - .andReturn(new KeyValueIteratorStub<>( - Collections.singleton(KeyValue.pair(WINDOWED_KEY_BYTES, VALUE_BYTES)).iterator())); + .andReturn(new KeyValueIteratorStub<>( + Collections.singleton(KeyValue.pair(WINDOWED_KEY_BYTES, VALUE_BYTES)).iterator())); init(); final KeyValueIterator, String> iterator = store.findSessions(KEY, 0, 0); @@ -260,11 +260,28 @@ public void shouldFindSessionsFromStoreAndRecordFetchMetric() { verify(innerStore); } + @Test + public void shouldBackwardFindSessionsFromStoreAndRecordFetchMetric() { + expect(innerStore.backwardFindSessions(KEY_BYTES, 0, 0)) + .andReturn(new KeyValueIteratorStub<>( + Collections.singleton(KeyValue.pair(WINDOWED_KEY_BYTES, VALUE_BYTES)).iterator())); + init(); + + final KeyValueIterator, String> iterator = store.backwardFindSessions(KEY, 0, 0); + assertThat(iterator.next().value, equalTo(VALUE)); + assertFalse(iterator.hasNext()); + iterator.close(); + + final KafkaMetric metric = metric("fetch-rate"); + assertTrue((Double) metric.metricValue() > 0); + verify(innerStore); + } + @Test public void shouldFindSessionRangeFromStoreAndRecordFetchMetric() { expect(innerStore.findSessions(KEY_BYTES, KEY_BYTES, 0, 0)) - .andReturn(new KeyValueIteratorStub<>( - Collections.singleton(KeyValue.pair(WINDOWED_KEY_BYTES, VALUE_BYTES)).iterator())); + .andReturn(new KeyValueIteratorStub<>( + Collections.singleton(KeyValue.pair(WINDOWED_KEY_BYTES, VALUE_BYTES)).iterator())); init(); final KeyValueIterator, String> iterator = store.findSessions(KEY, KEY, 0, 0); @@ -277,6 +294,23 @@ public void shouldFindSessionRangeFromStoreAndRecordFetchMetric() { verify(innerStore); } + @Test + public void shouldBackwardFindSessionRangeFromStoreAndRecordFetchMetric() { + expect(innerStore.backwardFindSessions(KEY_BYTES, KEY_BYTES, 0, 0)) + .andReturn(new KeyValueIteratorStub<>( + Collections.singleton(KeyValue.pair(WINDOWED_KEY_BYTES, VALUE_BYTES)).iterator())); + init(); + + final KeyValueIterator, String> iterator = store.backwardFindSessions(KEY, KEY, 0, 0); + assertThat(iterator.next().value, equalTo(VALUE)); + assertFalse(iterator.hasNext()); + iterator.close(); + + final KafkaMetric metric = metric("fetch-rate"); + assertTrue((Double) metric.metricValue() > 0); + verify(innerStore); + } + @Test public void shouldRemoveFromStoreAndRecordRemoveMetric() { innerStore.remove(WINDOWED_KEY_BYTES); @@ -294,8 +328,8 @@ public void shouldRemoveFromStoreAndRecordRemoveMetric() { @Test public void shouldFetchForKeyAndRecordFetchMetric() { expect(innerStore.fetch(KEY_BYTES)) - .andReturn(new KeyValueIteratorStub<>( - Collections.singleton(KeyValue.pair(WINDOWED_KEY_BYTES, VALUE_BYTES)).iterator())); + .andReturn(new KeyValueIteratorStub<>( + Collections.singleton(KeyValue.pair(WINDOWED_KEY_BYTES, VALUE_BYTES)).iterator())); init(); final KeyValueIterator, String> iterator = store.fetch(KEY); @@ -308,11 +342,28 @@ public void shouldFetchForKeyAndRecordFetchMetric() { verify(innerStore); } + @Test + public void shouldBackwardFetchForKeyAndRecordFetchMetric() { + expect(innerStore.backwardFetch(KEY_BYTES)) + .andReturn(new KeyValueIteratorStub<>( + Collections.singleton(KeyValue.pair(WINDOWED_KEY_BYTES, VALUE_BYTES)).iterator())); + init(); + + final KeyValueIterator, String> iterator = store.backwardFetch(KEY); + assertThat(iterator.next().value, equalTo(VALUE)); + assertFalse(iterator.hasNext()); + iterator.close(); + + final KafkaMetric metric = metric("fetch-rate"); + assertTrue((Double) metric.metricValue() > 0); + verify(innerStore); + } + @Test public void shouldFetchRangeFromStoreAndRecordFetchMetric() { expect(innerStore.fetch(KEY_BYTES, KEY_BYTES)) - .andReturn(new KeyValueIteratorStub<>( - Collections.singleton(KeyValue.pair(WINDOWED_KEY_BYTES, VALUE_BYTES)).iterator())); + .andReturn(new KeyValueIteratorStub<>( + Collections.singleton(KeyValue.pair(WINDOWED_KEY_BYTES, VALUE_BYTES)).iterator())); init(); final KeyValueIterator, String> iterator = store.fetch(KEY, KEY); @@ -325,6 +376,23 @@ public void shouldFetchRangeFromStoreAndRecordFetchMetric() { verify(innerStore); } + @Test + public void shouldBackwardFetchRangeFromStoreAndRecordFetchMetric() { + expect(innerStore.backwardFetch(KEY_BYTES, KEY_BYTES)) + .andReturn(new KeyValueIteratorStub<>( + Collections.singleton(KeyValue.pair(WINDOWED_KEY_BYTES, VALUE_BYTES)).iterator())); + init(); + + final KeyValueIterator, String> iterator = store.backwardFetch(KEY, KEY); + assertThat(iterator.next().value, equalTo(VALUE)); + assertFalse(iterator.hasNext()); + iterator.close(); + + final KafkaMetric metric = metric("fetch-rate"); + assertTrue((Double) metric.metricValue() > 0); + verify(innerStore); + } + @Test public void shouldRecordRestoreTimeOnInit() { init(); @@ -380,7 +448,8 @@ public void shouldThrowNullPointerOnFindSessionsRangeIfToIsNull() { store.findSessions("a", null, 0, 0); } - private interface CachedSessionStore extends SessionStore, CachedStateStore { } + private interface CachedSessionStore extends SessionStore, CachedStateStore { + } @SuppressWarnings("unchecked") @Test @@ -437,9 +506,9 @@ private KafkaMetric metric(final String name) { 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/MeteredWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java index 3f7eb1de2dbb8..9fc3f24ae59e1 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 @@ -358,7 +358,8 @@ public void shouldNotThrowNullPointerExceptionIfFetchReturnsNull() { assertNull(store.fetch("a", 0)); } - private interface CachedWindowStore extends WindowStore, CachedStateStore { } + private interface CachedWindowStore extends WindowStore, CachedStateStore { + } @SuppressWarnings("unchecked") @Test @@ -426,9 +427,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 59866ab467eec..cc26a454ba7a7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java @@ -46,7 +46,7 @@ public class ReadOnlyWindowStoreStub implements ReadOnlyWindowStore, private final long windowSize; private final Map> data = new HashMap<>(); - private boolean open = true; + private boolean open = true; ReadOnlyWindowStoreStub(final long windowSize) { this.windowSize = windowSize; @@ -160,7 +160,8 @@ public KeyValueIterator, V> backwardAll() { return new KeyValueIterator, V>() { @Override - public void close() {} + public void close() { + } @Override public Windowed peekNextKey() { @@ -202,7 +203,8 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, final long return new KeyValueIterator, V>() { @Override - public void close() {} + public void close() { + } @Override public Windowed peekNextKey() { @@ -293,7 +295,8 @@ public KeyValueIterator, V> fetch(final K from, final K to, final lo return new KeyValueIterator, V>() { @Override - public void close() {} + public void close() { + } @Override public Windowed peekNextKey() { @@ -382,13 +385,16 @@ public String name() { } @Override - public void init(final ProcessorContext context, final StateStore root) {} + public void init(final ProcessorContext context, final StateStore root) { + } @Override - public void flush() {} + public void flush() { + } @Override - public void close() {} + public void close() { + } @Override public boolean persistent() { @@ -413,7 +419,8 @@ private class TheWindowStoreIterator implements WindowStoreIterator { } @Override - public void close() {} + public void close() { + } @Override public Long peekNextKey() { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java index ab16b4eb76e00..74ef7b7a531c9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java @@ -54,14 +54,14 @@ public class SegmentIteratorTest { @Before public void before() { final InternalMockProcessorContext context = new InternalMockProcessorContext( - TestUtils.tempDirectory(), - Serdes.String(), - Serdes.String(), - new MockRecordCollector(), - new ThreadCache( - new LogContext("testCache "), - 0, - new MockStreamsMetrics(new Metrics()))); + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + new MockRecordCollector(), + new ThreadCache( + new LogContext("testCache "), + 0, + new MockStreamsMetrics(new Metrics()))); segmentOne.init(context, segmentOne); segmentTwo.init(context, segmentTwo); segmentOne.put(Bytes.wrap("a".getBytes()), "1".getBytes()); 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/ReadOnlySessionStoreStub.java b/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java index 5298894dd34eb..81df1337ea76e 100644 --- a/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java @@ -32,7 +32,7 @@ import java.util.TreeMap; public class ReadOnlySessionStoreStub implements ReadOnlySessionStore, StateStore { - private NavigableMap, V>>> sessions = new TreeMap<>(); + private final NavigableMap, V>>> sessions = new TreeMap<>(); private boolean open = true; public void put(final Windowed sessionKey, final V value) { @@ -97,7 +97,7 @@ public KeyValueIterator, V> fetch(final K from, final K to) { if (sessions.subMap(from, true, to, true).isEmpty()) { return new KeyValueIteratorStub<>(Collections., V>>emptyIterator()); } - final Iterator, V>>> keysIterator = sessions.subMap(from, true, to, true).values().iterator(); + final Iterator, V>>> keysIterator = sessions.subMap(from, true, to, true).values().iterator(); return new KeyValueIteratorStub<>( new Iterator, V>>() { From bcfe0075826b003078759f710ef48697f2521aba Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 29 Jul 2020 14:12:09 +0100 Subject: [PATCH 24/24] improve tests --- .../main/java/org/apache/kafka/connect/runtime/SessionKey.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SessionKey.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SessionKey.java index ab5476e4b9000..2ff0cc84c5ac1 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SessionKey.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SessionKey.java @@ -29,7 +29,8 @@ public class SessionKey { /** * Create a new session key with the given key value and creation timestamp - * @param key the actual cryptographic key to use for request validation; may not be null + * + * @param key the actual cryptographic key to use for request validation; may not be null * @param creationTimestamp the time at which the key was generated */ public SessionKey(SecretKey key, long creationTimestamp) {