Skip to content

KAFKA-4863: Querying window store may return unwanted keys#2662

Closed
dguy wants to merge 2 commits intoapache:trunkfrom
dguy:kafka-4863
Closed

KAFKA-4863: Querying window store may return unwanted keys#2662
dguy wants to merge 2 commits intoapache:trunkfrom
dguy:kafka-4863

Conversation

@dguy
Copy link
Copy Markdown
Contributor

@dguy dguy commented Mar 9, 2017

Make sure that the iterator returned from WindowStore.fetch(..) only returns matching keys, rather than all keys that are a prefix match.

@dguy
Copy link
Copy Markdown
Contributor Author

dguy commented Mar 9, 2017

@asfbot
Copy link
Copy Markdown

asfbot commented Mar 9, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/2106/
Test PASSed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Mar 9, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/2103/
Test PASSed (JDK 7 and Scala 2.10).

@asfbot
Copy link
Copy Markdown

asfbot commented Mar 9, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/2104/
Test PASSed (JDK 8 and Scala 2.12).

@enothereska
Copy link
Copy Markdown
Contributor

@dguy thanks for the patch. I can't immediately tell which change fixed the problem since there is a lot of refactoring in there. Any chance you could point me at the change? It'd be super useful (for me) to perhaps have 2 commits for such PRs, one that shows the immediate fix and another for the refactoring (although this is in theory, in practice it might be a pain). Thanks.

@dguy
Copy link
Copy Markdown
Contributor Author

dguy commented Mar 10, 2017

Sorry @enothereska, the key part of the change is in WindowKeySchema.hasNextCondition
The FilteredCacheIterator was extracted from CachingSessionStore as it is now also required by CachingWindowStore. The remainder of the changes are test related. Some tidy ups.

Copy link
Copy Markdown
Contributor

@guozhangwang guozhangwang left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A few comments.

if (iterator.hasNext()) {
final Bytes bytes = iterator.peekNextKey();
final Bytes keyBytes = WindowStoreUtils.bytesKeyFromBinaryKey(bytes.get());
return keyBytes.equals(binaryKey);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we also check that

final long start = SessionKeySerde.extractStart(bytes.get());
final long end = SessionKeySerde.extractEnd(bytes.get());
return end >= from && start <= to;

Although the current impl of RocksDBWIndowStore naturally checked that for us, it does not guarantee all underlying store impls guarantee that.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah makes sense to add the same check to the WindowKeySchema

assertEquals(Utils.mkSet("five@5"), entriesByKey.get(5));
assertNull(entriesByKey.get(6));

windowStore.close();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this still needed?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nope


private final File baseDir = TestUtils.tempDirectory("test");
private final MockProcessorContext context = new MockProcessorContext(baseDir, Serdes.ByteArray(), Serdes.ByteArray(), recordCollector, cache);
private WindowStore windowStore;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems all test cases are creating the store with the same key-value type: <Integer, String>, the difference only lies in the constructor parameters. In this case could we still declare it as typed and hence remove the @SuppressWarnings("unchecked")?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not all tests. There is at least one that uses <String, Integer>, hence the change

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you point me to the constructor of <String, Integer>? Can't found it in this file..

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is actually <String, String>, line 622

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@asfbot
Copy link
Copy Markdown

asfbot commented Mar 11, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/2127/
Test PASSed (JDK 7 and Scala 2.10).

@asfbot
Copy link
Copy Markdown

asfbot commented Mar 11, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/2130/
Test PASSed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Mar 11, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/2128/
Test FAILed (JDK 8 and Scala 2.12).

@asfgit asfgit closed this in 9e4548d Mar 14, 2017
@guozhangwang
Copy link
Copy Markdown
Contributor

Merged to trunk. @dguy Could you file another PR against 0.10.2 branch? I tried to cherry-pick this one but found it too hard to resolve conflicts.

@dguy
Copy link
Copy Markdown
Contributor Author

dguy commented Mar 15, 2017

@guozhangwang sure - no probs

if (iterator.hasNext()) {
final Bytes bytes = iterator.peekNextKey();
final Bytes keyBytes = WindowStoreUtils.bytesKeyFromBinaryKey(bytes.get());
if (!keyBytes.equals(binaryKey)) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@guozhangwang @dguy
we cannot guarantee that all the entries for one key will necessarily precede the entries for the next key.

The following code still fails with this patch, and only returns 0001 and 0003, since the key for ("a", "0005") will come after the key for ("aa", "0004")

        final RocksDBWindowStoreSupplier<String, String> supplier =
                new RocksDBWindowStoreSupplier<>(
                        "window",
                        0x7a00000000000000L, 2,
                        true,
                        Serdes.String(),
                        Serdes.String(),
                        0x7a00000000000000L,
                        true,
                        Collections.<String, String>emptyMap(),
                        false);

        windowStore = supplier.get();
        windowStore.init(context, windowStore);

        windowStore.put("a", "0001", 0);
        windowStore.put("aa", "0002", 0);
        windowStore.put("a", "0003", 1);
        windowStore.put("aa", "0004", 1);
        windowStore.put("a", "0005", 0x7a00000000000000L - 1);

        final List expected = Utils.mkList("0001", "0003", "0005");
        assertThat(toList(windowStore.fetch("a", 0, Long.MAX_VALUE)), equalTo(expected));

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Note that WindowStore.fetch() should return WindowStoreIterator where the key is Long indicating timestamp and value is the value.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a slightly different test. The window is much larger to ensure all the entries are in the same window and the last timestamp for (a, 0005) is such that it causes the resuling composite key to come after the one for (aa, 0004)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see, it is because the binary key becomes:
ay\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x00\x00\x00\x05
which is greater than the previous one:
aa\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x04

So the lexicographic comparator will return aa\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x04 first, hence we stop iterating

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nice catch - btw @xvrl

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants