You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by da...@apache.org on 2017/06/22 14:00:47 UTC
kafka git commit: KAFKA-4655;
Improve test coverage of CompositeReadOnlySessionStore
Repository: kafka
Updated Branches:
refs/heads/trunk 785d8e20c -> adfaa1161
KAFKA-4655; Improve test coverage of CompositeReadOnlySessionStore
Author: Jeyhun Karimov <je...@gmail.com>
Reviewers: Matthias J. Sax <ma...@confluent.io>, Damian Guy <da...@gmail.com>
Closes #3290 from jeyhunkarimov/KAFKA-4655
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/adfaa116
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/adfaa116
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/adfaa116
Branch: refs/heads/trunk
Commit: adfaa1161150635b9bb0e36a573382c5b68960e2
Parents: 785d8e2
Author: Jeyhun Karimov <je...@gmail.com>
Authored: Thu Jun 22 15:00:41 2017 +0100
Committer: Damian Guy <da...@gmail.com>
Committed: Thu Jun 22 15:00:41 2017 +0100
----------------------------------------------------------------------
.../streams/state/internals/CachingSessionStore.java | 4 ++++
.../internals/CompositeReadOnlySessionStoreTest.java | 14 +++++++++++---
2 files changed, 15 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/adfaa116/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java
----------------------------------------------------------------------
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 37d0c20..566d5c1 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
@@ -32,6 +32,7 @@ import org.apache.kafka.streams.state.SessionStore;
import org.apache.kafka.streams.state.StateSerdes;
import java.util.List;
+import java.util.Objects;
class CachingSessionStore<K, AGG> extends WrappedStateStore.AbstractStateStore implements SessionStore<K, AGG>, CachedStateStore<Windowed<K>, AGG> {
@@ -150,11 +151,14 @@ class CachingSessionStore<K, AGG> extends WrappedStateStore.AbstractStateStore i
@Override
public KeyValueIterator<Windowed<K>, AGG> fetch(final K key) {
+ Objects.requireNonNull(key, "key cannot be null");
return findSessions(key, 0, Long.MAX_VALUE);
}
@Override
public KeyValueIterator<Windowed<K>, AGG> fetch(K from, K to) {
+ Objects.requireNonNull(from, "from cannot be null");
+ Objects.requireNonNull(to, "to cannot be null");
return findSessions(from, to, 0, Long.MAX_VALUE);
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/adfaa116/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java
index e2646ad..f077321 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java
@@ -34,6 +34,7 @@ import java.util.List;
import static org.apache.kafka.test.StreamsTestUtils.toList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
public class CompositeReadOnlySessionStoreTest {
@@ -111,10 +112,17 @@ public class CompositeReadOnlySessionStoreTest {
store.fetch("a");
}
- @Test(expected = InvalidStateStoreException.class)
- public void shouldThrowInvalidStateStoreExceptionIfFetchThrows() throws Exception {
+ @Test
+ public void shouldThrowInvalidStateStoreExceptionIfSessionFetchThrows() {
underlyingSessionStore.setOpen(false);
- underlyingSessionStore.fetch("key");
+ try {
+ sessionStore.fetch("key");
+ fail("Should have thrown InvalidStateStoreException with session store");
+ } catch (InvalidStateStoreException e) { }
}
+ @Test(expected = NullPointerException.class)
+ public void shouldThrowNullPointerExceptionIfFetchingNullKey() {
+ sessionStore.fetch(null);
+ }
}
\ No newline at end of file