MINOR: fix warnings in Kafka Streams state store tests (#17855)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Matthias J. Sax 2024-11-26 09:54:21 -08:00 committed by GitHub
parent 98d47f47ef
commit f5d712396b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
22 changed files with 887 additions and 946 deletions

View File

@ -227,7 +227,7 @@ public class SessionWindowedKStreamImplTest {
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
processData(driver); processData(driver);
final SessionStore<String, Long> store = driver.getSessionStore("count-store"); final SessionStore<String, Long> store = driver.getSessionStore("count-store");
final List<KeyValue<Windowed<String>, Long>> data = StreamsTestUtils.toList(store.fetch("1", "2")); final List<KeyValue<Windowed<String>, Long>> data = StreamsTestUtils.toListAndCloseIterator(store.fetch("1", "2"));
if (!emitFinal) { if (!emitFinal) {
assertThat( assertThat(
data, data,
@ -255,7 +255,7 @@ public class SessionWindowedKStreamImplTest {
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
processData(driver); processData(driver);
final SessionStore<String, String> sessionStore = driver.getSessionStore("reduced"); final SessionStore<String, String> sessionStore = driver.getSessionStore("reduced");
final List<KeyValue<Windowed<String>, String>> data = StreamsTestUtils.toList(sessionStore.fetch("1", "2")); final List<KeyValue<Windowed<String>, String>> data = StreamsTestUtils.toListAndCloseIterator(sessionStore.fetch("1", "2"));
if (!emitFinal) { if (!emitFinal) {
assertThat( assertThat(
@ -288,7 +288,7 @@ public class SessionWindowedKStreamImplTest {
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
processData(driver); processData(driver);
final SessionStore<String, String> sessionStore = driver.getSessionStore("aggregated"); final SessionStore<String, String> sessionStore = driver.getSessionStore("aggregated");
final List<KeyValue<Windowed<String>, String>> data = StreamsTestUtils.toList(sessionStore.fetch("1", "2")); final List<KeyValue<Windowed<String>, String>> data = StreamsTestUtils.toListAndCloseIterator(sessionStore.fetch("1", "2"));
if (!emitFinal) { if (!emitFinal) {
assertThat( assertThat(
data, data,

View File

@ -208,7 +208,7 @@ public class SlidingWindowedKStreamImplTest {
{ {
final WindowStore<String, Long> windowStore = driver.getWindowStore("count-store"); final WindowStore<String, Long> windowStore = driver.getWindowStore("count-store");
final List<KeyValue<Windowed<String>, Long>> data = final List<KeyValue<Windowed<String>, Long>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
assertThat(data, equalTo(Arrays.asList( assertThat(data, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), 1L), KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), 1L),
@ -223,7 +223,7 @@ public class SlidingWindowedKStreamImplTest {
final WindowStore<String, ValueAndTimestamp<Long>> windowStore = final WindowStore<String, ValueAndTimestamp<Long>> windowStore =
driver.getTimestampedWindowStore("count-store"); driver.getTimestampedWindowStore("count-store");
final List<KeyValue<Windowed<String>, ValueAndTimestamp<Long>>> data = final List<KeyValue<Windowed<String>, ValueAndTimestamp<Long>>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
assertThat(data, equalTo(Arrays.asList( assertThat(data, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), ValueAndTimestamp.make(1L, 100L)), KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), ValueAndTimestamp.make(1L, 100L)),
KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), ValueAndTimestamp.make(2L, 150L)), KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), ValueAndTimestamp.make(2L, 150L)),
@ -248,7 +248,7 @@ public class SlidingWindowedKStreamImplTest {
{ {
final WindowStore<String, String> windowStore = driver.getWindowStore("reduced"); final WindowStore<String, String> windowStore = driver.getWindowStore("reduced");
final List<KeyValue<Windowed<String>, String>> data = final List<KeyValue<Windowed<String>, String>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
assertThat(data, equalTo(Arrays.asList( assertThat(data, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), "1"), KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), "1"),
KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), "1+2"), KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), "1+2"),
@ -262,7 +262,7 @@ public class SlidingWindowedKStreamImplTest {
final WindowStore<String, ValueAndTimestamp<Long>> windowStore = final WindowStore<String, ValueAndTimestamp<Long>> windowStore =
driver.getTimestampedWindowStore("reduced"); driver.getTimestampedWindowStore("reduced");
final List<KeyValue<Windowed<String>, ValueAndTimestamp<Long>>> data = final List<KeyValue<Windowed<String>, ValueAndTimestamp<Long>>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
assertThat(data, equalTo(Arrays.asList( assertThat(data, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), ValueAndTimestamp.make("1", 100L)), KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), ValueAndTimestamp.make("1", 100L)),
KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), ValueAndTimestamp.make("1+2", 150L)), KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), ValueAndTimestamp.make("1+2", 150L)),
@ -289,7 +289,7 @@ public class SlidingWindowedKStreamImplTest {
{ {
final WindowStore<String, String> windowStore = driver.getWindowStore("aggregated"); final WindowStore<String, String> windowStore = driver.getWindowStore("aggregated");
final List<KeyValue<Windowed<String>, String>> data = final List<KeyValue<Windowed<String>, String>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
assertThat(data, equalTo(Arrays.asList( assertThat(data, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), "0+1"), KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), "0+1"),
KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), "0+1+2"), KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), "0+1+2"),
@ -303,7 +303,7 @@ public class SlidingWindowedKStreamImplTest {
final WindowStore<String, ValueAndTimestamp<Long>> windowStore = final WindowStore<String, ValueAndTimestamp<Long>> windowStore =
driver.getTimestampedWindowStore("aggregated"); driver.getTimestampedWindowStore("aggregated");
final List<KeyValue<Windowed<String>, ValueAndTimestamp<Long>>> data = final List<KeyValue<Windowed<String>, ValueAndTimestamp<Long>>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
assertThat(data, equalTo(Arrays.asList( assertThat(data, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), ValueAndTimestamp.make("0+1", 100L)), KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), ValueAndTimestamp.make("0+1", 100L)),
KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), ValueAndTimestamp.make("0+1+2", 150L)), KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), ValueAndTimestamp.make("0+1+2", 150L)),
@ -410,7 +410,7 @@ public class SlidingWindowedKStreamImplTest {
{ {
final WindowStore<String, String> windowStore = driver.getWindowStore("aggregated"); final WindowStore<String, String> windowStore = driver.getWindowStore("aggregated");
final List<KeyValue<Windowed<String>, String>> data = final List<KeyValue<Windowed<String>, String>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "1", ofEpochMilli(0), ofEpochMilli(10000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "1", ofEpochMilli(0), ofEpochMilli(10000L)));
assertThat(data, equalTo(Arrays.asList( assertThat(data, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("1", new TimeWindow(900, 1000)), "0+4"), KeyValue.pair(new Windowed<>("1", new TimeWindow(900, 1000)), "0+4"),
KeyValue.pair(new Windowed<>("1", new TimeWindow(1900, 2000)), "0+5")))); KeyValue.pair(new Windowed<>("1", new TimeWindow(1900, 2000)), "0+5"))));
@ -419,7 +419,7 @@ public class SlidingWindowedKStreamImplTest {
final WindowStore<String, ValueAndTimestamp<Long>> windowStore = final WindowStore<String, ValueAndTimestamp<Long>> windowStore =
driver.getTimestampedWindowStore("aggregated"); driver.getTimestampedWindowStore("aggregated");
final List<KeyValue<Windowed<String>, ValueAndTimestamp<Long>>> data = final List<KeyValue<Windowed<String>, ValueAndTimestamp<Long>>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "1", ofEpochMilli(0), ofEpochMilli(2000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "1", ofEpochMilli(0), ofEpochMilli(2000L)));
assertThat(data, equalTo(Arrays.asList( assertThat(data, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("1", new TimeWindow(900, 1000)), ValueAndTimestamp.make("0+4", 1000L)), KeyValue.pair(new Windowed<>("1", new TimeWindow(900, 1000)), ValueAndTimestamp.make("0+4", 1000L)),
KeyValue.pair(new Windowed<>("1", new TimeWindow(1900, 2000)), ValueAndTimestamp.make("0+5", 2000L))))); KeyValue.pair(new Windowed<>("1", new TimeWindow(1900, 2000)), ValueAndTimestamp.make("0+5", 2000L)))));

View File

@ -239,7 +239,7 @@ public class TimeWindowedKStreamImplTest {
{ {
final WindowStore<String, Long> windowStore = driver.getWindowStore("count-store"); final WindowStore<String, Long> windowStore = driver.getWindowStore("count-store");
final List<KeyValue<Windowed<String>, Long>> data = final List<KeyValue<Windowed<String>, Long>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
if (withCache) { if (withCache) {
// with cache returns all records (expired from underneath as well) as part of // with cache returns all records (expired from underneath as well) as part of
@ -266,7 +266,7 @@ public class TimeWindowedKStreamImplTest {
final WindowStore<String, ValueAndTimestamp<Long>> windowStore = final WindowStore<String, ValueAndTimestamp<Long>> windowStore =
driver.getTimestampedWindowStore("count-store"); driver.getTimestampedWindowStore("count-store");
final List<KeyValue<Windowed<String>, ValueAndTimestamp<Long>>> data = final List<KeyValue<Windowed<String>, ValueAndTimestamp<Long>>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
// the same values and logic described above applies here as well. // the same values and logic described above applies here as well.
if (withCache) { if (withCache) {
@ -305,7 +305,7 @@ public class TimeWindowedKStreamImplTest {
{ {
final WindowStore<String, String> windowStore = driver.getWindowStore("reduced"); final WindowStore<String, String> windowStore = driver.getWindowStore("reduced");
final List<KeyValue<Windowed<String>, String>> data = final List<KeyValue<Windowed<String>, String>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
if (withCache) { if (withCache) {
// with cache returns all records (expired from underneath as well) as part of // with cache returns all records (expired from underneath as well) as part of
@ -325,7 +325,7 @@ public class TimeWindowedKStreamImplTest {
{ {
final WindowStore<String, ValueAndTimestamp<String>> windowStore = driver.getTimestampedWindowStore("reduced"); final WindowStore<String, ValueAndTimestamp<String>> windowStore = driver.getTimestampedWindowStore("reduced");
final List<KeyValue<Windowed<String>, ValueAndTimestamp<String>>> data = final List<KeyValue<Windowed<String>, ValueAndTimestamp<String>>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
// same logic/data as explained above. // same logic/data as explained above.
if (withCache) { if (withCache) {
@ -358,7 +358,7 @@ public class TimeWindowedKStreamImplTest {
{ {
final WindowStore<String, String> windowStore = driver.getWindowStore("aggregated"); final WindowStore<String, String> windowStore = driver.getWindowStore("aggregated");
final List<KeyValue<Windowed<String>, String>> data = final List<KeyValue<Windowed<String>, String>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
if (withCache) { if (withCache) {
// with cache returns all records (expired from underneath as well) as part of // with cache returns all records (expired from underneath as well) as part of
@ -379,7 +379,7 @@ public class TimeWindowedKStreamImplTest {
{ {
final WindowStore<String, ValueAndTimestamp<String>> windowStore = driver.getTimestampedWindowStore("aggregated"); final WindowStore<String, ValueAndTimestamp<String>> windowStore = driver.getTimestampedWindowStore("aggregated");
final List<KeyValue<Windowed<String>, ValueAndTimestamp<String>>> data = final List<KeyValue<Windowed<String>, ValueAndTimestamp<String>>> data =
StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
if (withCache) { if (withCache) {
assertThat(data, equalTo(asList( assertThat(data, equalTo(asList(
KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), ValueAndTimestamp.make("0+1+2", 15L)), KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), ValueAndTimestamp.make("0+1+2", 15L)),

View File

@ -38,11 +38,11 @@ import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator;
import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.CoreMatchers.hasItem;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.core.IsEqual.equalTo; import static org.hamcrest.core.IsEqual.equalTo;
@ -53,18 +53,17 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail; import static org.junit.jupiter.api.Assertions.fail;
@SuppressWarnings("unchecked")
public abstract class AbstractKeyValueStoreTest { public abstract class AbstractKeyValueStoreTest {
protected abstract <K, V> KeyValueStore<K, V> createKeyValueStore(final StateStoreContext context); protected abstract <K, V> KeyValueStore<K, V> createKeyValueStore(final StateStoreContext context);
protected InternalMockProcessorContext context; protected InternalMockProcessorContext<?, ?> context;
protected KeyValueStore<Integer, String> store; protected KeyValueStore<Integer, String> store;
protected KeyValueStoreTestDriver<Integer, String> driver; protected KeyValueStoreTestDriver<Integer, String> driver;
@BeforeEach @BeforeEach
public void before() { public void before() {
driver = KeyValueStoreTestDriver.create(Integer.class, String.class); driver = KeyValueStoreTestDriver.create(Integer.class, String.class);
context = (InternalMockProcessorContext) driver.context(); context = (InternalMockProcessorContext<?, ?>) driver.context();
context.setTime(10); context.setTime(10);
store = createKeyValueStore(context); store = createKeyValueStore(context);
} }
@ -75,7 +74,8 @@ public abstract class AbstractKeyValueStoreTest {
driver.clear(); driver.clear();
} }
private static Map<Integer, String> getContents(final KeyValueIterator<Integer, String> iter) { private static Map<Integer, String> getContentsAndCloseIterator(final KeyValueIterator<Integer, String> iter) {
try (iter) {
final HashMap<Integer, String> result = new HashMap<>(); final HashMap<Integer, String> result = new HashMap<>();
while (iter.hasNext()) { while (iter.hasNext()) {
final KeyValue<Integer, String> entry = iter.next(); final KeyValue<Integer, String> entry = iter.next();
@ -83,8 +83,8 @@ public abstract class AbstractKeyValueStoreTest {
} }
return result; return result;
} }
}
@SuppressWarnings("unchecked")
@Test @Test
public void shouldNotIncludeDeletedFromRangeResult() { public void shouldNotIncludeDeletedFromRangeResult() {
store.close(); store.close();
@ -113,7 +113,7 @@ public abstract class AbstractKeyValueStoreTest {
// should not include deleted records in iterator // should not include deleted records in iterator
final Map<Integer, String> expectedContents = Collections.singletonMap(2, "two"); final Map<Integer, String> expectedContents = Collections.singletonMap(2, "two");
assertEquals(expectedContents, getContents(store.all())); assertEquals(expectedContents, getContentsAndCloseIterator(store.all()));
} }
@Test @Test
@ -142,7 +142,7 @@ public abstract class AbstractKeyValueStoreTest {
// should not include deleted records in iterator // should not include deleted records in iterator
final Map<Integer, String> expectedContents = Collections.singletonMap(2, "two"); final Map<Integer, String> expectedContents = Collections.singletonMap(2, "two");
assertEquals(expectedContents, getContents(store.all())); assertEquals(expectedContents, getContentsAndCloseIterator(store.all()));
} }
@Test @Test
@ -184,13 +184,13 @@ public abstract class AbstractKeyValueStoreTest {
expectedContents.put(4, "four"); expectedContents.put(4, "four");
// Check range iteration ... // Check range iteration ...
assertEquals(expectedContents, getContents(store.range(2, 4))); assertEquals(expectedContents, getContentsAndCloseIterator(store.range(2, 4)));
assertEquals(expectedContents, getContents(store.range(2, 6))); assertEquals(expectedContents, getContentsAndCloseIterator(store.range(2, 6)));
// Check all iteration ... // Check all iteration ...
expectedContents.put(0, "zero"); expectedContents.put(0, "zero");
expectedContents.put(1, "one"); expectedContents.put(1, "one");
assertEquals(expectedContents, getContents(store.all())); assertEquals(expectedContents, getContentsAndCloseIterator(store.all()));
} }
@Test @Test
@ -232,13 +232,13 @@ public abstract class AbstractKeyValueStoreTest {
expectedContents.put(4, "four"); expectedContents.put(4, "four");
// Check range iteration ... // Check range iteration ...
assertEquals(expectedContents, getContents(store.reverseRange(2, 4))); assertEquals(expectedContents, getContentsAndCloseIterator(store.reverseRange(2, 4)));
assertEquals(expectedContents, getContents(store.reverseRange(2, 6))); assertEquals(expectedContents, getContentsAndCloseIterator(store.reverseRange(2, 6)));
// Check all iteration ... // Check all iteration ...
expectedContents.put(0, "zero"); expectedContents.put(0, "zero");
expectedContents.put(1, "one"); expectedContents.put(1, "one");
assertEquals(expectedContents, getContents(store.reverseAll())); assertEquals(expectedContents, getContentsAndCloseIterator(store.reverseAll()));
} }
@Test @Test
@ -498,14 +498,10 @@ public abstract class AbstractKeyValueStoreTest {
store.putAll(entries); store.putAll(entries);
final List<KeyValue<Integer, String>> allReturned = new ArrayList<>(); final List<KeyValue<Integer, String>> allReturned = toListAndCloseIterator(store.all());
final List<KeyValue<Integer, String>> expectedReturned = final List<KeyValue<Integer, String>> expectedReturned =
Arrays.asList(KeyValue.pair(1, "one"), KeyValue.pair(2, "two")); Arrays.asList(KeyValue.pair(1, "one"), KeyValue.pair(2, "two"));
final Iterator<KeyValue<Integer, String>> iterator = store.all();
while (iterator.hasNext()) {
allReturned.add(iterator.next());
}
assertThat(allReturned, equalTo(expectedReturned)); assertThat(allReturned, equalTo(expectedReturned));
} }
@ -517,14 +513,10 @@ public abstract class AbstractKeyValueStoreTest {
store.putAll(entries); store.putAll(entries);
final List<KeyValue<Integer, String>> allReturned = new ArrayList<>(); final List<KeyValue<Integer, String>> allReturned = toListAndCloseIterator(store.reverseAll());
final List<KeyValue<Integer, String>> expectedReturned = final List<KeyValue<Integer, String>> expectedReturned =
Arrays.asList(KeyValue.pair(2, "two"), KeyValue.pair(1, "one")); Arrays.asList(KeyValue.pair(2, "two"), KeyValue.pair(1, "one"));
final Iterator<KeyValue<Integer, String>> iterator = store.reverseAll();
while (iterator.hasNext()) {
allReturned.add(iterator.next());
}
assertThat(allReturned, equalTo(expectedReturned)); assertThat(allReturned, equalTo(expectedReturned));
} }
@ -545,11 +537,11 @@ public abstract class AbstractKeyValueStoreTest {
store.putAll(entries); store.putAll(entries);
final Iterator<KeyValue<Integer, String>> iterator = store.range(2, 2); try (final KeyValueIterator<Integer, String> iterator = store.range(2, 2)) {
assertEquals(iterator.next().value, store.get(2)); assertEquals(iterator.next().value, store.get(2));
assertFalse(iterator.hasNext()); assertFalse(iterator.hasNext());
} }
}
@Test @Test
public void shouldReturnSameResultsForGetAndReverseRangeWithEqualKeys() { public void shouldReturnSameResultsForGetAndReverseRangeWithEqualKeys() {
@ -560,11 +552,11 @@ public abstract class AbstractKeyValueStoreTest {
store.putAll(entries); store.putAll(entries);
final Iterator<KeyValue<Integer, String>> iterator = store.reverseRange(2, 2); try (final KeyValueIterator<Integer, String> iterator = store.reverseRange(2, 2)) {
assertEquals(iterator.next().value, store.get(2)); assertEquals(iterator.next().value, store.get(2));
assertFalse(iterator.hasNext()); assertFalse(iterator.hasNext());
} }
}
@Test @Test
public void shouldNotThrowConcurrentModificationException() { public void shouldNotThrowConcurrentModificationException() {

View File

@ -25,6 +25,7 @@ import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.serialization.LongSerializer;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.LogContext;
@ -36,7 +37,6 @@ import org.apache.kafka.streams.StreamsConfig.InternalConfig;
import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.Window;
import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.kstream.internals.SessionWindow; import org.apache.kafka.streams.kstream.internals.SessionWindow;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.processor.internals.ChangelogRecordDeserializationHelper; import org.apache.kafka.streams.processor.internals.ChangelogRecordDeserializationHelper;
import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
@ -88,7 +88,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment> { public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment> {
private final long windowSizeForTimeWindow = 500; private final long windowSizeForTimeWindow = 500;
private InternalMockProcessorContext context; private InternalMockProcessorContext<?, ?> context;
private AbstractRocksDBSegmentedBytesStore<S> bytesStore; private AbstractRocksDBSegmentedBytesStore<S> bytesStore;
private File stateDir; private File stateDir;
private final Window[] windows = new Window[4]; private final Window[] windows = new Window[4];
@ -140,7 +140,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
new MockRecordCollector(), new MockRecordCollector(),
new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())) new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics()))
); );
bytesStore.init((StateStoreContext) context, bytesStore); bytesStore.init(context, bytesStore);
} }
@AfterEach @AfterEach
@ -168,21 +168,21 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
Bytes.wrap(keyA.getBytes()), 0, windows[2].start())) { Bytes.wrap(keyA.getBytes()), 0, windows[2].start())) {
// All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1) // All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1)
// for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000 // for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000
assertEquals(Collections.emptyList(), toList(values)); assertEquals(Collections.emptyList(), toListAndCloseIterator(values));
} }
try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.fetch( try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.fetch(
Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), 0, windows[2].start())) { Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), 0, windows[2].start())) {
// All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1) // All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1)
// for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000 // for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000
assertEquals(Collections.emptyList(), toList(values)); assertEquals(Collections.emptyList(), toListAndCloseIterator(values));
} }
try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.fetch( try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.fetch(
null, Bytes.wrap(keyB.getBytes()), 0, windows[2].start())) { null, Bytes.wrap(keyB.getBytes()), 0, windows[2].start())) {
// All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1) // All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1)
// for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000 // for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000
assertEquals(Collections.emptyList(), toList(values)); assertEquals(Collections.emptyList(), toListAndCloseIterator(values));
} }
try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.fetch( try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.fetch(
@ -193,7 +193,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L) KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L)
); );
assertEquals(expected, toList(values)); assertEquals(expected, toListAndCloseIterator(values));
} }
try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.fetch( try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.fetch(
@ -204,7 +204,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L) KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L)
); );
assertEquals(expected, toList(values)); assertEquals(expected, toListAndCloseIterator(values));
} }
} }
@ -225,7 +225,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
// All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1) // All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1)
// for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000 // for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000
assertEquals(Collections.emptyList(), toList(values)); assertEquals(Collections.emptyList(), toListAndCloseIterator(values));
} }
try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.backwardFetch( try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.backwardFetch(
@ -233,7 +233,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
// All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1) // All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1)
// for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000 // for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000
assertEquals(Collections.emptyList(), toList(values)); assertEquals(Collections.emptyList(), toListAndCloseIterator(values));
} }
try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.backwardFetch( try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.backwardFetch(
@ -241,7 +241,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
// All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1) // All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1)
// for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000 // for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000
assertEquals(Collections.emptyList(), toList(values)); assertEquals(Collections.emptyList(), toListAndCloseIterator(values));
} }
try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.backwardFetch( try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.backwardFetch(
@ -252,7 +252,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L) KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L)
); );
assertEquals(expected, toList(values)); assertEquals(expected, toListAndCloseIterator(values));
} }
try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.backwardFetch( try (final KeyValueIterator<Bytes, byte[]> values = bytesStore.backwardFetch(
@ -263,7 +263,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L) KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L)
); );
assertEquals(expected, toList(values)); assertEquals(expected, toListAndCloseIterator(values));
} }
} }
@ -289,7 +289,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
expected.add(KeyValue.pair(new Windowed<>(key, windows[1]), 50L)); expected.add(KeyValue.pair(new Windowed<>(key, windows[1]), 50L));
} }
assertEquals(expected, toList(results)); assertEquals(expected, toListAndCloseIterator(results));
} }
} }
@ -322,7 +322,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
bytesStore.put(serializeKey(new Windowed<>(key, windows[3])), serializeValue(1000)); bytesStore.put(serializeKey(new Windowed<>(key, windows[3])), serializeValue(1000));
assertEquals(Set.of(segments.segmentName(0), segments.segmentName(1)), segmentDirs()); assertEquals(Set.of(segments.segmentName(0), segments.segmentName(1)), segmentDirs());
final List<KeyValue<Windowed<String>, Long>> results = toList(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0, 1500)); final List<KeyValue<Windowed<String>, Long>> results = toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0, 1500));
/* /*
* All records expired as observed stream time = 60,000 which sets actual-from to 59001(60,000 - 1000 + 1). to = 1500. * All records expired as observed stream time = 60,000 which sets actual-from to 59001(60,000 - 1000 + 1). to = 1500.
*/ */
@ -355,7 +355,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
/* /*
* Only 1 record returned. observed stream time = 60000, actual from = 59001 (60000 - 1000 + 1) and to = Long.MAX. * Only 1 record returned. observed stream time = 60000, actual from = 59001 (60000 - 1000 + 1) and to = Long.MAX.
*/ */
final List<KeyValue<Windowed<String>, Long>> results = toList(bytesStore.all()); final List<KeyValue<Windowed<String>, Long>> results = toListAndCloseIterator(bytesStore.all());
assertEquals( assertEquals(
Collections.singletonList( Collections.singletonList(
KeyValue.pair(new Windowed<>(key, windows[3]), 100L) KeyValue.pair(new Windowed<>(key, windows[3]), 100L)
@ -388,7 +388,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
/* /*
* Only 1 record returned. observed stream time = 60000, actual from = 59001 (60000 - 1000 + 1) and to = 60,000. * Only 1 record returned. observed stream time = 60000, actual from = 59001 (60000 - 1000 + 1) and to = 60,000.
*/ */
final List<KeyValue<Windowed<String>, Long>> results = toList(bytesStore.fetchAll(0L, 60_000L)); final List<KeyValue<Windowed<String>, Long>> results = toListAndCloseIterator(bytesStore.fetchAll(0L, 60_000L));
assertEquals( assertEquals(
Collections.singletonList( Collections.singletonList(
KeyValue.pair(new Windowed<>(key, windows[3]), 100L) KeyValue.pair(new Windowed<>(key, windows[3]), 100L)
@ -422,8 +422,8 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
bytesStore = getBytesStore(); bytesStore = getBytesStore();
bytesStore.init((StateStoreContext) context, bytesStore); bytesStore.init(context, bytesStore);
final List<KeyValue<Windowed<String>, Long>> results = toList(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0L, 60_000L)); final List<KeyValue<Windowed<String>, Long>> results = toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0L, 60_000L));
assertThat( assertThat(
results, results,
equalTo( equalTo(
@ -456,8 +456,8 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
bytesStore = getBytesStore(); bytesStore = getBytesStore();
bytesStore.init((StateStoreContext) context, bytesStore); bytesStore.init(context, bytesStore);
final List<KeyValue<Windowed<String>, Long>> results = toList(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0L, 60_000L)); final List<KeyValue<Windowed<String>, Long>> results = toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0L, 60_000L));
assertThat( assertThat(
results, results,
equalTo( equalTo(
@ -479,7 +479,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
// need to create a segment so we can attempt to write to it again. // need to create a segment so we can attempt to write to it again.
bytesStore.put(serializeKey(new Windowed<>(key, windows[0])), serializeValue(50)); bytesStore.put(serializeKey(new Windowed<>(key, windows[0])), serializeValue(50));
bytesStore.close(); bytesStore.close();
bytesStore.init((StateStoreContext) context, bytesStore); bytesStore.init(context, bytesStore);
bytesStore.put(serializeKey(new Windowed<>(key, windows[1])), serializeValue(100)); bytesStore.put(serializeKey(new Windowed<>(key, windows[1])), serializeValue(100));
} }
@ -515,7 +515,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
} }
private void shouldRestoreToByteStore() { private void shouldRestoreToByteStore() {
bytesStore.init((StateStoreContext) context, bytesStore); bytesStore.init(context, bytesStore);
// 0 segments initially. // 0 segments initially.
assertEquals(0, bytesStore.getSegments().size()); assertEquals(0, bytesStore.getSegments().size());
final String key = "a"; final String key = "a";
@ -533,7 +533,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
final List<KeyValue<Windowed<String>, Long>> expected = new ArrayList<>(); final List<KeyValue<Windowed<String>, Long>> expected = new ArrayList<>();
expected.add(new KeyValue<>(new Windowed<>(key, windows[3]), 100L)); expected.add(new KeyValue<>(new Windowed<>(key, windows[3]), 100L));
final List<KeyValue<Windowed<String>, Long>> results = toList(bytesStore.all()); final List<KeyValue<Windowed<String>, Long>> results = toListAndCloseIterator(bytesStore.all());
assertEquals(expected, results); assertEquals(expected, results);
} }
@ -541,7 +541,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
@MethodSource("getKeySchemas") @MethodSource("getKeySchemas")
public void shouldMatchPositionAfterPut(final SegmentedBytesStore.KeySchema schema) { public void shouldMatchPositionAfterPut(final SegmentedBytesStore.KeySchema schema) {
before(schema); before(schema);
bytesStore.init((StateStoreContext) context, bytesStore); bytesStore.init(context, bytesStore);
final String keyA = "a"; final String keyA = "a";
final String keyB = "b"; final String keyB = "b";
@ -579,7 +579,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
Time.SYSTEM Time.SYSTEM
); );
bytesStore = getBytesStore(); bytesStore = getBytesStore();
bytesStore.init((StateStoreContext) context, bytesStore); bytesStore.init(context, bytesStore);
// 0 segments initially. // 0 segments initially.
assertEquals(0, bytesStore.getSegments().size()); assertEquals(0, bytesStore.getSegments().size());
@ -594,7 +594,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
final List<KeyValue<Windowed<String>, Long>> expected = new ArrayList<>(); final List<KeyValue<Windowed<String>, Long>> expected = new ArrayList<>();
expected.add(new KeyValue<>(new Windowed<>(key, windows[3]), 200L)); expected.add(new KeyValue<>(new Windowed<>(key, windows[3]), 200L));
final List<KeyValue<Windowed<String>, Long>> results = toList(bytesStore.all()); final List<KeyValue<Windowed<String>, Long>> results = toListAndCloseIterator(bytesStore.all());
assertEquals(expected, results); assertEquals(expected, results);
assertThat(bytesStore.getPosition(), Matchers.notNullValue()); assertThat(bytesStore.getPosition(), Matchers.notNullValue());
assertThat(bytesStore.getPosition().getPartitionPositions(""), Matchers.notNullValue()); assertThat(bytesStore.getPosition().getPartitionPositions(""), Matchers.notNullValue());
@ -619,7 +619,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
Time.SYSTEM Time.SYSTEM
); );
bytesStore = getBytesStore(); bytesStore = getBytesStore();
bytesStore.init((StateStoreContext) context, bytesStore); bytesStore.init(context, bytesStore);
// 0 segments initially. // 0 segments initially.
assertEquals(0, bytesStore.getSegments().size()); assertEquals(0, bytesStore.getSegments().size());
@ -634,7 +634,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
final List<KeyValue<Windowed<String>, Long>> expected = new ArrayList<>(); final List<KeyValue<Windowed<String>, Long>> expected = new ArrayList<>();
expected.add(new KeyValue<>(new Windowed<>(key, windows[3]), 200L)); expected.add(new KeyValue<>(new Windowed<>(key, windows[3]), 200L));
final List<KeyValue<Windowed<String>, Long>> results = toList(bytesStore.all()); final List<KeyValue<Windowed<String>, Long>> results = toListAndCloseIterator(bytesStore.all());
assertEquals(expected, results); assertEquals(expected, results);
assertThat(bytesStore.getPosition(), Matchers.notNullValue()); assertThat(bytesStore.getPosition(), Matchers.notNullValue());
assertThat(bytesStore.getPosition().getPartitionPositions("A"), Matchers.notNullValue()); assertThat(bytesStore.getPosition().getPartitionPositions("A"), Matchers.notNullValue());
@ -661,7 +661,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
Time.SYSTEM Time.SYSTEM
); );
bytesStore = getBytesStore(); bytesStore = getBytesStore();
bytesStore.init((StateStoreContext) context, bytesStore); bytesStore.init(context, bytesStore);
// 0 segments initially. // 0 segments initially.
assertEquals(0, bytesStore.getSegments().size()); assertEquals(0, bytesStore.getSegments().size());
@ -675,7 +675,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
* SessionKeySchema, it's 1500. Which changes the actual-from while fetching. In case of SessionKeySchema, the * SessionKeySchema, it's 1500. Which changes the actual-from while fetching. In case of SessionKeySchema, the
* fetch happens from 501 to end while for WindowKeySchema it's from 1 to end. * fetch happens from 501 to end while for WindowKeySchema it's from 1 to end.
*/ */
final List<KeyValue<Windowed<String>, Long>> results = toList(bytesStore.all()); final List<KeyValue<Windowed<String>, Long>> results = toListAndCloseIterator(bytesStore.all());
if (schema instanceof SessionKeySchema) { if (schema instanceof SessionKeySchema) {
assertEquals(Collections.emptyList(), results); assertEquals(Collections.emptyList(), results);
} else { } else {
@ -705,7 +705,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
Time.SYSTEM Time.SYSTEM
); );
bytesStore = getBytesStore(); bytesStore = getBytesStore();
bytesStore.init((StateStoreContext) context, bytesStore); bytesStore.init(context, bytesStore);
bytesStore.restoreAllInternal(getChangelogRecordsWithoutHeaders()); bytesStore.restoreAllInternal(getChangelogRecordsWithoutHeaders());
assertThat(bytesStore.getPosition(), is(Position.emptyPosition())); assertThat(bytesStore.getPosition(), is(Position.emptyPosition()));
} }
@ -816,13 +816,13 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
before(schema); before(schema);
final Properties streamsConfig = StreamsTestUtils.getStreamsConfig(); final Properties streamsConfig = StreamsTestUtils.getStreamsConfig();
final AbstractRocksDBSegmentedBytesStore<S> bytesStore = getBytesStore(); final AbstractRocksDBSegmentedBytesStore<S> bytesStore = getBytesStore();
final InternalMockProcessorContext context = new InternalMockProcessorContext( final InternalMockProcessorContext<?, ?> context = new InternalMockProcessorContext<>(
TestUtils.tempDirectory(), TestUtils.tempDirectory(),
new StreamsConfig(streamsConfig) new StreamsConfig(streamsConfig)
); );
final Time time = Time.SYSTEM; final Time time = Time.SYSTEM;
context.setSystemTimeMs(time.milliseconds()); context.setSystemTimeMs(time.milliseconds());
bytesStore.init((StateStoreContext) context, bytesStore); bytesStore.init(context, bytesStore);
// write a record to advance stream time, with a high enough timestamp // write a record to advance stream time, with a high enough timestamp
// that the subsequent record in windows[0] will already be expired. // that the subsequent record in windows[0] will already be expired.
@ -878,11 +878,14 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
} }
} }
@SuppressWarnings("resource")
private byte[] serializeValue(final long value) { private byte[] serializeValue(final long value) {
return Serdes.Long().serializer().serialize("", value); return new LongSerializer().serialize("", value);
} }
private List<KeyValue<Windowed<String>, Long>> toList(final KeyValueIterator<Bytes, byte[]> iterator) { @SuppressWarnings("resource")
private List<KeyValue<Windowed<String>, Long>> toListAndCloseIterator(final KeyValueIterator<Bytes, byte[]> iterator) {
try (iterator) {
final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>(); final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>();
final StateSerdes<String, Long> stateSerdes = StateSerdes.withBuiltinTypes("dummy", String.class, Long.class); final StateSerdes<String, Long> stateSerdes = StateSerdes.withBuiltinTypes("dummy", String.class, Long.class);
while (iterator.hasNext()) { while (iterator.hasNext()) {
@ -910,4 +913,5 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
} }
return results; return results;
} }
}
} }

View File

@ -23,7 +23,6 @@ import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.query.Position;
import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.Stores;
@ -45,7 +44,7 @@ import static java.util.Arrays.asList;
import static java.util.Objects.requireNonNull; import static java.util.Objects.requireNonNull;
import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.test.StreamsTestUtils.valuesToSet; import static org.apache.kafka.test.StreamsTestUtils.valuesToSetAndCloseIterator;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertFalse;
@ -177,7 +176,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
// expired record // expired record
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
0, 0,
ofEpochMilli(startTime - WINDOW_SIZE), ofEpochMilli(startTime - WINDOW_SIZE),
ofEpochMilli(startTime + WINDOW_SIZE)))); ofEpochMilli(startTime + WINDOW_SIZE))));
@ -193,7 +192,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
if (storeType() == StoreType.RocksDBWindowStore) { if (storeType() == StoreType.RocksDBWindowStore) {
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
1, 1,
ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment - WINDOW_SIZE),
ofEpochMilli(startTime + increment + WINDOW_SIZE)))); ofEpochMilli(startTime + increment + WINDOW_SIZE))));
@ -201,32 +200,32 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
} else { } else {
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("one")), new HashSet<>(Collections.singletonList("one")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
1, 1,
ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment - WINDOW_SIZE),
ofEpochMilli(startTime + increment + WINDOW_SIZE)))); ofEpochMilli(startTime + increment + WINDOW_SIZE))));
} }
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("two")), new HashSet<>(Collections.singletonList("two")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
3, 3,
ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("four")), new HashSet<>(Collections.singletonList("four")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
4, 4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("five")), new HashSet<>(Collections.singletonList("five")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
5, 5,
ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE))));
@ -243,13 +242,13 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
0, 0,
ofEpochMilli(startTime - WINDOW_SIZE), ofEpochMilli(startTime - WINDOW_SIZE),
ofEpochMilli(startTime + WINDOW_SIZE)))); ofEpochMilli(startTime + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
1, 1,
ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment - WINDOW_SIZE),
ofEpochMilli(startTime + increment + WINDOW_SIZE)))); ofEpochMilli(startTime + increment + WINDOW_SIZE))));
@ -266,7 +265,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
assertEquals( assertEquals(
// expired record // expired record
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE))));
@ -274,32 +273,32 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
assertEquals( assertEquals(
// expired record // expired record
new HashSet<>(Collections.singletonList("two")), new HashSet<>(Collections.singletonList("two")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE))));
} }
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
3, 3,
ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("four")), new HashSet<>(Collections.singletonList("four")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
4, 4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("five")), new HashSet<>(Collections.singletonList("five")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
5, 5,
ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("six")), new HashSet<>(Collections.singletonList("six")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
6, 6,
ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE))));
@ -316,50 +315,50 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
0, 0,
ofEpochMilli(startTime - WINDOW_SIZE), ofEpochMilli(startTime - WINDOW_SIZE),
ofEpochMilli(startTime + WINDOW_SIZE)))); ofEpochMilli(startTime + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
1, 1,
ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment - WINDOW_SIZE),
ofEpochMilli(startTime + increment + WINDOW_SIZE)))); ofEpochMilli(startTime + increment + WINDOW_SIZE))));
assertEquals( assertEquals(
// expired record // expired record
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
3, 3,
ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("four")), new HashSet<>(Collections.singletonList("four")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
4, 4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("five")), new HashSet<>(Collections.singletonList("five")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
5, 5,
ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("six")), new HashSet<>(Collections.singletonList("six")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
6, 6,
ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("seven")), new HashSet<>(Collections.singletonList("seven")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
7, 7,
ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE))));
@ -376,25 +375,25 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
0, 0,
ofEpochMilli(startTime - WINDOW_SIZE), ofEpochMilli(startTime - WINDOW_SIZE),
ofEpochMilli(startTime + WINDOW_SIZE)))); ofEpochMilli(startTime + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
1, 1,
ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment - WINDOW_SIZE),
ofEpochMilli(startTime + increment + WINDOW_SIZE)))); ofEpochMilli(startTime + increment + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
3, 3,
ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE))));
@ -402,7 +401,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
assertEquals( assertEquals(
// expired record // expired record
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
4, 4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE))));
@ -410,7 +409,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
assertEquals( assertEquals(
// expired record // expired record
new HashSet<>(Collections.singletonList("four")), new HashSet<>(Collections.singletonList("four")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
4, 4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE))));
@ -418,25 +417,25 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
} }
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("five")), new HashSet<>(Collections.singletonList("five")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
5, 5,
ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("six")), new HashSet<>(Collections.singletonList("six")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
6, 6,
ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("seven")), new HashSet<>(Collections.singletonList("seven")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
7, 7,
ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("eight")), new HashSet<>(Collections.singletonList("eight")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
8, 8,
ofEpochMilli(startTime + increment * 8 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 8 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 8 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 8 + WINDOW_SIZE))));
@ -458,7 +457,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
windowStore.close(); windowStore.close();
windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(),
Serdes.String()); Serdes.String());
windowStore.init((StateStoreContext) context, windowStore); windowStore.init(context, windowStore);
context.setTime(0L); context.setTime(0L);
windowStore.put(0, "v", 0); windowStore.put(0, "v", 0);
@ -480,15 +479,15 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
segmentDirs(baseDir) segmentDirs(baseDir)
); );
WindowStoreIterator iter;
int fetchedCount; int fetchedCount;
iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(SEGMENT_INTERVAL * 4)); try (final WindowStoreIterator<String> iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(SEGMENT_INTERVAL * 4))) {
fetchedCount = 0; fetchedCount = 0;
while (iter.hasNext()) { while (iter.hasNext()) {
iter.next(); iter.next();
fetchedCount++; fetchedCount++;
} }
}
assertEquals(4, fetchedCount); assertEquals(4, fetchedCount);
assertEquals( assertEquals(
@ -498,12 +497,13 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
windowStore.put(0, "v", SEGMENT_INTERVAL * 3); windowStore.put(0, "v", SEGMENT_INTERVAL * 3);
iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(SEGMENT_INTERVAL * 4)); try (final WindowStoreIterator<String> iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(SEGMENT_INTERVAL * 4))) {
fetchedCount = 0; fetchedCount = 0;
while (iter.hasNext()) { while (iter.hasNext()) {
iter.next(); iter.next();
fetchedCount++; fetchedCount++;
} }
}
// 1 extra record is expired in the case of RocksDBWindowStore as // 1 extra record is expired in the case of RocksDBWindowStore as
// actualFrom = observedStreamTime - retentionPeriod + 1. The +1 // actualFrom = observedStreamTime - retentionPeriod + 1. The +1
// isn't present for RocksDbTimeOrderedStoreWith*Index // isn't present for RocksDbTimeOrderedStoreWith*Index
@ -520,12 +520,13 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
windowStore.put(0, "v", SEGMENT_INTERVAL * 5); windowStore.put(0, "v", SEGMENT_INTERVAL * 5);
iter = windowStore.fetch(0, ofEpochMilli(SEGMENT_INTERVAL * 4), ofEpochMilli(SEGMENT_INTERVAL * 10)); try (final WindowStoreIterator<String> iter = windowStore.fetch(0, ofEpochMilli(SEGMENT_INTERVAL * 4), ofEpochMilli(SEGMENT_INTERVAL * 10))) {
fetchedCount = 0; fetchedCount = 0;
while (iter.hasNext()) { while (iter.hasNext()) {
iter.next(); iter.next();
fetchedCount++; fetchedCount++;
} }
}
// the latest record has a timestamp > 60k. So, the +1 in actualFrom calculation in // the latest record has a timestamp > 60k. So, the +1 in actualFrom calculation in
// RocksDbWindowStore shouldn't have an implciation and all stores should return the same fetched counts. // RocksDbWindowStore shouldn't have an implciation and all stores should return the same fetched counts.
@ -552,7 +553,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
windowStore.close(); windowStore.close();
windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, false, Serdes.Integer(), Serdes.String()); windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, false, Serdes.Integer(), Serdes.String());
windowStore.init((StateStoreContext) context, windowStore); windowStore.init(context, windowStore);
// put something in the store to advance its stream time and expire the old segments // put something in the store to advance its stream time and expire the old segments
windowStore.put(1, "v", 6L * SEGMENT_INTERVAL); windowStore.put(1, "v", 6L * SEGMENT_INTERVAL);
@ -568,7 +569,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
assertEquals(expected, actual); assertEquals(expected, actual);
try (final WindowStoreIterator iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(1000000L))) { try (final WindowStoreIterator<String> iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(1000000L))) {
while (iter.hasNext()) { while (iter.hasNext()) {
iter.next(); iter.next();
} }
@ -583,7 +584,6 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
); );
} }
@SuppressWarnings("unchecked")
@Test @Test
public void testRestore() throws Exception { public void testRestore() throws Exception {
final long startTime = SEGMENT_INTERVAL * 2; final long startTime = SEGMENT_INTERVAL * 2;
@ -610,62 +610,62 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
false, false,
Serdes.Integer(), Serdes.Integer(),
Serdes.String()); Serdes.String());
windowStore.init((StateStoreContext) context, windowStore); windowStore.init(context, windowStore);
// For all tests, for WindowStore actualFrom is computed using observedStreamTime - retention + 1. // For all tests, for WindowStore actualFrom is computed using observedStreamTime - retention + 1.
// while for TimeOrderedWindowStores, actualFrom = observedStreamTime - retention // while for TimeOrderedWindowStores, actualFrom = observedStreamTime - retention
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
0, 0,
ofEpochMilli(startTime - WINDOW_SIZE), ofEpochMilli(startTime - WINDOW_SIZE),
ofEpochMilli(startTime + WINDOW_SIZE)))); ofEpochMilli(startTime + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
1, 1,
ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment - WINDOW_SIZE),
ofEpochMilli(startTime + increment + WINDOW_SIZE)))); ofEpochMilli(startTime + increment + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
3, 3,
ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
4, 4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
5, 5,
ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
6, 6,
ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
7, 7,
ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
8, 8,
ofEpochMilli(startTime + increment * 8 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 8 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 8 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 8 + WINDOW_SIZE))));
@ -679,25 +679,25 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
0, 0,
ofEpochMilli(startTime - WINDOW_SIZE), ofEpochMilli(startTime - WINDOW_SIZE),
ofEpochMilli(startTime + WINDOW_SIZE)))); ofEpochMilli(startTime + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
1, 1,
ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment - WINDOW_SIZE),
ofEpochMilli(startTime + increment + WINDOW_SIZE)))); ofEpochMilli(startTime + increment + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
3, 3,
ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE))));
@ -713,14 +713,14 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
if (storeType() == StoreType.RocksDBWindowStore) { if (storeType() == StoreType.RocksDBWindowStore) {
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
4, 4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE))));
} else { } else {
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("four")), new HashSet<>(Collections.singletonList("four")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
4, 4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE))));
@ -728,25 +728,25 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
} }
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("five")), new HashSet<>(Collections.singletonList("five")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
5, 5,
ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("six")), new HashSet<>(Collections.singletonList("six")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
6, 6,
ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("seven")), new HashSet<>(Collections.singletonList("seven")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
7, 7,
ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("eight")), new HashSet<>(Collections.singletonList("eight")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
8, 8,
ofEpochMilli(startTime + increment * 8 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 8 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 8 + WINDOW_SIZE)))); ofEpochMilli(startTime + increment * 8 + WINDOW_SIZE))));
@ -762,11 +762,12 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
); );
} }
@SuppressWarnings({"rawtypes", "unchecked"})
@Test @Test
public void shouldMatchPositionAfterPut() { public void shouldMatchPositionAfterPut() {
final MeteredWindowStore<Integer, String> meteredSessionStore = (MeteredWindowStore<Integer, String>) windowStore; final MeteredWindowStore<Integer, String> meteredSessionStore = (MeteredWindowStore<Integer, String>) windowStore;
final ChangeLoggingWindowBytesStore changeLoggingSessionBytesStore = (ChangeLoggingWindowBytesStore) meteredSessionStore.wrapped(); final ChangeLoggingWindowBytesStore changeLoggingSessionBytesStore = (ChangeLoggingWindowBytesStore) meteredSessionStore.wrapped();
final WrappedStateStore rocksDBWindowStore = (WrappedStateStore) changeLoggingSessionBytesStore.wrapped(); final WrappedStateStore<WrappedStateStore, Integer, String> rocksDBWindowStore = (WrappedStateStore<WrappedStateStore, Integer, String>) changeLoggingSessionBytesStore.wrapped();
context.setRecordContext(new ProcessorRecordContext(0, 1, 0, "", new RecordHeaders())); context.setRecordContext(new ProcessorRecordContext(0, 1, 0, "", new RecordHeaders()));
windowStore.put(0, "0", SEGMENT_INTERVAL); windowStore.put(0, "0", SEGMENT_INTERVAL);

View File

@ -21,8 +21,10 @@ import org.apache.kafka.common.Metric;
import org.apache.kafka.common.MetricName; import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.serialization.IntegerSerializer;
import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.common.utils.LogCaptureAppender;
import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.LogContext;
@ -31,7 +33,6 @@ import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.kstream.internals.SessionWindow; import org.apache.kafka.streams.kstream.internals.SessionWindow;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.query.Position;
@ -93,7 +94,7 @@ public abstract class AbstractSessionBytesStoreTest {
private MockRecordCollector recordCollector; private MockRecordCollector recordCollector;
InternalMockProcessorContext context; InternalMockProcessorContext<?, ?> context;
<K, V> SessionStore<K, V> buildSessionStore(final long retentionPeriod, <K, V> SessionStore<K, V> buildSessionStore(final long retentionPeriod,
final Serde<K> keySerde, final Serde<K> keySerde,
@ -159,7 +160,7 @@ public abstract class AbstractSessionBytesStoreTest {
new MockStreamsMetrics(new Metrics()))); new MockStreamsMetrics(new Metrics())));
context.setTime(1L); context.setTime(1L);
sessionStore.init((StateStoreContext) context, sessionStore); sessionStore.init(context, sessionStore);
} }
@AfterEach @AfterEach
@ -240,6 +241,7 @@ public abstract class AbstractSessionBytesStoreTest {
} }
} }
@SuppressWarnings("resource")
@Test @Test
public void shouldFindSessionsForTimeRange() { public void shouldFindSessionsForTimeRange() {
sessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 5L); sessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 5L);
@ -552,7 +554,7 @@ public abstract class AbstractSessionBytesStoreTest {
public void shouldFetchExactKeys() { public void shouldFetchExactKeys() {
sessionStore.close(); sessionStore.close();
sessionStore = buildSessionStore(0x7a00000000000000L, Serdes.String(), Serdes.Long()); sessionStore = buildSessionStore(0x7a00000000000000L, Serdes.String(), Serdes.Long());
sessionStore.init((StateStoreContext) context, sessionStore); sessionStore.init(context, sessionStore);
sessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L); sessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L);
sessionStore.put(new Windowed<>("aa", new SessionWindow(0, 10)), 2L); sessionStore.put(new Windowed<>("aa", new SessionWindow(0, 10)), 2L);
@ -608,7 +610,7 @@ public abstract class AbstractSessionBytesStoreTest {
public void shouldBackwardFetchExactKeys() { public void shouldBackwardFetchExactKeys() {
sessionStore.close(); sessionStore.close();
sessionStore = buildSessionStore(0x7a00000000000000L, Serdes.String(), Serdes.Long()); sessionStore = buildSessionStore(0x7a00000000000000L, Serdes.String(), Serdes.Long());
sessionStore.init((StateStoreContext) context, sessionStore); sessionStore.init(context, sessionStore);
sessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L); sessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L);
sessionStore.put(new Windowed<>("aa", new SessionWindow(0, 10)), 2L); sessionStore.put(new Windowed<>("aa", new SessionWindow(0, 10)), 2L);
@ -665,7 +667,7 @@ public abstract class AbstractSessionBytesStoreTest {
final SessionStore<Bytes, String> sessionStore = final SessionStore<Bytes, String> sessionStore =
buildSessionStore(RETENTION_PERIOD, Serdes.Bytes(), Serdes.String()); buildSessionStore(RETENTION_PERIOD, Serdes.Bytes(), Serdes.String());
sessionStore.init((StateStoreContext) context, sessionStore); sessionStore.init(context, sessionStore);
final Bytes key1 = Bytes.wrap(new byte[] {0}); final Bytes key1 = Bytes.wrap(new byte[] {0});
final Bytes key2 = Bytes.wrap(new byte[] {0, 0}); final Bytes key2 = Bytes.wrap(new byte[] {0, 0});
@ -704,7 +706,7 @@ public abstract class AbstractSessionBytesStoreTest {
final SessionStore<Bytes, String> sessionStore = final SessionStore<Bytes, String> sessionStore =
buildSessionStore(RETENTION_PERIOD, Serdes.Bytes(), Serdes.String()); buildSessionStore(RETENTION_PERIOD, Serdes.Bytes(), Serdes.String());
sessionStore.init((StateStoreContext) context, sessionStore); sessionStore.init(context, sessionStore);
final Bytes key1 = Bytes.wrap(new byte[] {0}); final Bytes key1 = Bytes.wrap(new byte[] {0});
final Bytes key2 = Bytes.wrap(new byte[] {0, 0}); final Bytes key2 = Bytes.wrap(new byte[] {0, 0});
@ -747,10 +749,11 @@ public abstract class AbstractSessionBytesStoreTest {
sessionStore.put(new Windowed<>("aa", new SessionWindow(10, 20)), 4L); sessionStore.put(new Windowed<>("aa", new SessionWindow(10, 20)), 4L);
try (final KeyValueIterator<Windowed<String>, Long> iterator = sessionStore.findSessions("a", 0L, 20)) { try (final KeyValueIterator<Windowed<String>, Long> iterator = sessionStore.findSessions("a", 0L, 20)) {
assertEquals(new Windowed<>("a", new SessionWindow(0L, 0L)), iterator.peekNextKey());
assertEquals(iterator.peekNextKey(), new Windowed<>("a", new SessionWindow(0L, 0L))); final Windowed<String> k1 = iterator.peekNextKey();
assertEquals(iterator.peekNextKey(), iterator.next().key); assertEquals(iterator.next().key, k1);
assertEquals(iterator.peekNextKey(), iterator.next().key); final Windowed<String> k2 = iterator.peekNextKey();
assertEquals(iterator.next().key, k2);
assertFalse(iterator.hasNext()); assertFalse(iterator.hasNext());
} }
} }
@ -763,15 +766,15 @@ public abstract class AbstractSessionBytesStoreTest {
sessionStore.put(new Windowed<>("aa", new SessionWindow(10, 20)), 4L); sessionStore.put(new Windowed<>("aa", new SessionWindow(10, 20)), 4L);
try (final KeyValueIterator<Windowed<String>, Long> iterator = sessionStore.backwardFindSessions("a", 0L, 20)) { try (final KeyValueIterator<Windowed<String>, Long> iterator = sessionStore.backwardFindSessions("a", 0L, 20)) {
assertEquals(new Windowed<>("a", new SessionWindow(10L, 20L)), iterator.peekNextKey());
assertEquals(iterator.peekNextKey(), new Windowed<>("a", new SessionWindow(10L, 20L))); final Windowed<String> k1 = iterator.peekNextKey();
assertEquals(iterator.peekNextKey(), iterator.next().key); assertEquals(iterator.next().key, k1);
assertEquals(iterator.peekNextKey(), iterator.next().key); final Windowed<String> k2 = iterator.peekNextKey();
assertEquals(iterator.next().key, k2);
assertFalse(iterator.hasNext()); assertFalse(iterator.hasNext());
} }
} }
@SuppressWarnings("unchecked")
@Test @Test
public void shouldRestore() { public void shouldRestore() {
final List<KeyValue<Windowed<String>, Long>> expected = Arrays.asList( final List<KeyValue<Windowed<String>, Long>> expected = Arrays.asList(
@ -842,7 +845,7 @@ public abstract class AbstractSessionBytesStoreTest {
public void shouldMeasureExpiredRecords() { public void shouldMeasureExpiredRecords() {
final Properties streamsConfig = StreamsTestUtils.getStreamsConfig(); final Properties streamsConfig = StreamsTestUtils.getStreamsConfig();
final SessionStore<String, Long> sessionStore = buildSessionStore(RETENTION_PERIOD, Serdes.String(), Serdes.Long()); final SessionStore<String, Long> sessionStore = buildSessionStore(RETENTION_PERIOD, Serdes.String(), Serdes.Long());
final InternalMockProcessorContext context = new InternalMockProcessorContext( final InternalMockProcessorContext<?, ?> context = new InternalMockProcessorContext<>(
TestUtils.tempDirectory(), TestUtils.tempDirectory(),
new StreamsConfig(streamsConfig), new StreamsConfig(streamsConfig),
recordCollector recordCollector
@ -850,7 +853,7 @@ public abstract class AbstractSessionBytesStoreTest {
final Time time = Time.SYSTEM; final Time time = Time.SYSTEM;
context.setTime(1L); context.setTime(1L);
context.setSystemTimeMs(time.milliseconds()); context.setSystemTimeMs(time.milliseconds());
sessionStore.init((StateStoreContext) context, sessionStore); sessionStore.init(context, sessionStore);
// Advance stream time by inserting record with large enough timestamp that records with timestamp 0 are expired // Advance stream time by inserting record with large enough timestamp that records with timestamp 0 are expired
// Note that rocksdb will only expire segments at a time (where segment interval = 60,000 for this retention period) // Note that rocksdb will only expire segments at a time (where segment interval = 60,000 for this retention period)
@ -894,11 +897,13 @@ public abstract class AbstractSessionBytesStoreTest {
sessionStore.remove(new Windowed<>("a", new SessionWindow(0, 1))); sessionStore.remove(new Windowed<>("a", new SessionWindow(0, 1)));
} }
@SuppressWarnings("resource")
@Test @Test
public void shouldThrowNullPointerExceptionOnFindSessionsNullKey() { public void shouldThrowNullPointerExceptionOnFindSessionsNullKey() {
assertThrows(NullPointerException.class, () -> sessionStore.findSessions(null, 1L, 2L)); assertThrows(NullPointerException.class, () -> sessionStore.findSessions(null, 1L, 2L));
} }
@SuppressWarnings("resource")
@Test @Test
public void shouldThrowNullPointerExceptionOnFetchNullKey() { public void shouldThrowNullPointerExceptionOnFetchNullKey() {
assertThrows(NullPointerException.class, () -> sessionStore.fetch(null)); assertThrows(NullPointerException.class, () -> sessionStore.fetch(null));
@ -914,12 +919,11 @@ public abstract class AbstractSessionBytesStoreTest {
assertThrows(NullPointerException.class, () -> sessionStore.put(null, 1L)); assertThrows(NullPointerException.class, () -> sessionStore.put(null, 1L));
} }
@SuppressWarnings("resource")
@Test @Test
public void shouldNotThrowInvalidRangeExceptionWithNegativeFromKey() { public void shouldNotThrowInvalidRangeExceptionWithNegativeFromKey() {
final String keyFrom = Serdes.String().deserializer() final String keyFrom = new StringDeserializer().deserialize("", new IntegerSerializer().serialize("", -1));
.deserialize("", Serdes.Integer().serializer().serialize("", -1)); final String keyTo = new StringDeserializer().deserialize("", new IntegerSerializer().serialize("", 1));
final String keyTo = Serdes.String().deserializer()
.deserialize("", Serdes.Integer().serializer().serialize("", 1));
try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister();
final KeyValueIterator<Windowed<String>, Long> iterator = sessionStore.findSessions(keyFrom, keyTo, 0L, 10L)) { final KeyValueIterator<Windowed<String>, Long> iterator = sessionStore.findSessions(keyFrom, keyTo, 0L, 10L)) {
@ -968,16 +972,12 @@ public abstract class AbstractSessionBytesStoreTest {
@Test @Test
public void shouldMatchPositionAfterPut() { public void shouldMatchPositionAfterPut() {
final MeteredSessionStore<String, Long> meteredSessionStore = (MeteredSessionStore<String, Long>) sessionStore;
final ChangeLoggingSessionBytesStore changeLoggingSessionBytesStore = (ChangeLoggingSessionBytesStore) meteredSessionStore.wrapped();
final SessionStore wrapped = (SessionStore) changeLoggingSessionBytesStore.wrapped();
context.setRecordContext(new ProcessorRecordContext(0, 1, 0, "", new RecordHeaders())); context.setRecordContext(new ProcessorRecordContext(0, 1, 0, "", new RecordHeaders()));
sessionStore.put(new Windowed<String>("a", new SessionWindow(0, 0)), 1L); sessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L);
context.setRecordContext(new ProcessorRecordContext(0, 2, 0, "", new RecordHeaders())); context.setRecordContext(new ProcessorRecordContext(0, 2, 0, "", new RecordHeaders()));
sessionStore.put(new Windowed<String>("aa", new SessionWindow(0, 10)), 2L); sessionStore.put(new Windowed<>("aa", new SessionWindow(0, 10)), 2L);
context.setRecordContext(new ProcessorRecordContext(0, 3, 0, "", new RecordHeaders())); context.setRecordContext(new ProcessorRecordContext(0, 3, 0, "", new RecordHeaders()));
sessionStore.put(new Windowed<String>("a", new SessionWindow(10, 20)), 3L); sessionStore.put(new Windowed<>("a", new SessionWindow(10, 20)), 3L);
final Position expected = Position.fromMap(mkMap(mkEntry("", mkMap(mkEntry(0, 3L))))); final Position expected = Position.fromMap(mkMap(mkEntry("", mkMap(mkEntry(0, 3L)))));
final Position actual = sessionStore.getPosition(); final Position actual = sessionStore.getPosition();

View File

@ -30,7 +30,6 @@ import org.apache.kafka.common.utils.Time;
import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueIterator;
@ -60,9 +59,9 @@ import static java.time.Instant.ofEpochMilli;
import static java.util.Arrays.asList; import static java.util.Arrays.asList;
import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator;
import static org.apache.kafka.test.StreamsTestUtils.toSet; import static org.apache.kafka.test.StreamsTestUtils.toSet;
import static org.apache.kafka.test.StreamsTestUtils.valuesToSet; import static org.apache.kafka.test.StreamsTestUtils.valuesToSetAndCloseIterator;
import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.CoreMatchers.hasItem;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
@ -89,7 +88,7 @@ public abstract class AbstractWindowBytesStoreTest {
final KeyValue<Windowed<Integer>, String> five = windowedPair(5, "five", defaultStartTime + 5); final KeyValue<Windowed<Integer>, String> five = windowedPair(5, "five", defaultStartTime + 5);
WindowStore<Integer, String> windowStore; WindowStore<Integer, String> windowStore;
InternalMockProcessorContext context; InternalMockProcessorContext<?, ?> context;
MockRecordCollector recordCollector; MockRecordCollector recordCollector;
final File baseDir = TestUtils.tempDirectory("test"); final File baseDir = TestUtils.tempDirectory("test");
@ -117,7 +116,7 @@ public abstract class AbstractWindowBytesStoreTest {
new MockStreamsMetrics(new Metrics()))); new MockStreamsMetrics(new Metrics())));
context.setTime(1L); context.setTime(1L);
windowStore.init((StateStoreContext) context, windowStore); windowStore.init(context, windowStore);
} }
@AfterEach @AfterEach
@ -131,12 +130,12 @@ public abstract class AbstractWindowBytesStoreTest {
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("zero")), new HashSet<>(Collections.singletonList("zero")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
0, 0,
ofEpochMilli(defaultStartTime + 0 - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 0 + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + WINDOW_SIZE))));
putSecondBatch(windowStore, defaultStartTime, context); putSecondBatch(windowStore, defaultStartTime);
assertEquals("two+1", windowStore.fetch(2, defaultStartTime + 3L)); assertEquals("two+1", windowStore.fetch(2, defaultStartTime + 3L));
assertEquals("two+2", windowStore.fetch(2, defaultStartTime + 4L)); assertEquals("two+2", windowStore.fetch(2, defaultStartTime + 4L));
@ -147,91 +146,91 @@ public abstract class AbstractWindowBytesStoreTest {
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime - 2L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - 2L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime - 2L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime - 2L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("two")), new HashSet<>(Collections.singletonList("two")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime - 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - 1L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime - 1L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime - 1L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two", "two+1")), new HashSet<>(asList("two", "two+1")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two", "two+1", "two+2")), new HashSet<>(asList("two", "two+1", "two+2")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime + 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 1L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two", "two+1", "two+2", "two+3")), new HashSet<>(asList("two", "two+1", "two+2", "two+3")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime + 2L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 2L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two", "two+1", "two+2", "two+3", "two+4")), new HashSet<>(asList("two", "two+1", "two+2", "two+3", "two+4")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two", "two+1", "two+2", "two+3", "two+4", "two+5")), new HashSet<>(asList("two", "two+1", "two+2", "two+3", "two+4", "two+5")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two", "two+1", "two+2", "two+3", "two+4", "two+5", "two+6")), new HashSet<>(asList("two", "two+1", "two+2", "two+3", "two+4", "two+5", "two+6")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime + 5L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 5L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 5L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 5L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two+1", "two+2", "two+3", "two+4", "two+5", "two+6")), new HashSet<>(asList("two+1", "two+2", "two+3", "two+4", "two+5", "two+6")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime + 6L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 6L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 6L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 6L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two+2", "two+3", "two+4", "two+5", "two+6")), new HashSet<>(asList("two+2", "two+3", "two+4", "two+5", "two+6")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime + 7L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 7L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 7L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 7L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two+3", "two+4", "two+5", "two+6")), new HashSet<>(asList("two+3", "two+4", "two+5", "two+6")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime + 8L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 8L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 8L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 8L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two+4", "two+5", "two+6")), new HashSet<>(asList("two+4", "two+5", "two+6")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime + 9L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 9L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 9L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 9L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two+5", "two+6")), new HashSet<>(asList("two+5", "two+6")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime + 10L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 10L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 10L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 10L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("two+6")), new HashSet<>(Collections.singletonList("two+6")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime + 11L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 11L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 11L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 11L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
2, 2,
ofEpochMilli(defaultStartTime + 12L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 12L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 12L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 12L + WINDOW_SIZE))));
@ -264,14 +263,14 @@ public abstract class AbstractWindowBytesStoreTest {
assertEquals( assertEquals(
asList(zero, one, two, three, four, five), asList(zero, one, two, three, four, five),
toList(windowStore.all()) toListAndCloseIterator(windowStore.all())
); );
} }
@Test @Test
public void shouldGetAllNonDeletedRecords() { public void shouldGetAllNonDeletedRecords() {
// Add some records // Add some records
windowStore.put(0, "zero", defaultStartTime + 0); windowStore.put(0, "zero", defaultStartTime);
windowStore.put(1, "one", defaultStartTime + 1); windowStore.put(1, "one", defaultStartTime + 1);
windowStore.put(2, "two", defaultStartTime + 2); windowStore.put(2, "two", defaultStartTime + 2);
windowStore.put(3, "three", defaultStartTime + 3); windowStore.put(3, "three", defaultStartTime + 3);
@ -284,7 +283,7 @@ public abstract class AbstractWindowBytesStoreTest {
// Only non-deleted records should appear in the all() iterator // Only non-deleted records should appear in the all() iterator
assertEquals( assertEquals(
asList(zero, two, four), asList(zero, two, four),
toList(windowStore.all()) toListAndCloseIterator(windowStore.all())
); );
} }
@ -292,7 +291,7 @@ public abstract class AbstractWindowBytesStoreTest {
public void shouldGetAllReturnTimestampOrderedRecords() { public void shouldGetAllReturnTimestampOrderedRecords() {
// Add some records in different order // Add some records in different order
windowStore.put(4, "four", defaultStartTime + 4); windowStore.put(4, "four", defaultStartTime + 4);
windowStore.put(0, "zero", defaultStartTime + 0); windowStore.put(0, "zero", defaultStartTime);
windowStore.put(2, "two", defaultStartTime + 2); windowStore.put(2, "two", defaultStartTime + 2);
windowStore.put(3, "three", defaultStartTime + 3); windowStore.put(3, "three", defaultStartTime + 3);
windowStore.put(1, "one", defaultStartTime + 1); windowStore.put(1, "one", defaultStartTime + 1);
@ -302,13 +301,13 @@ public abstract class AbstractWindowBytesStoreTest {
assertEquals( assertEquals(
asList(zero, one, two, three, four), asList(zero, one, two, three, four),
toList(windowStore.all()) toListAndCloseIterator(windowStore.all())
); );
} }
@Test @Test
public void shouldEarlyClosedIteratorStillGetAllRecords() { public void shouldEarlyClosedIteratorStillGetAllRecords() {
windowStore.put(0, "zero", defaultStartTime + 0); windowStore.put(0, "zero", defaultStartTime);
windowStore.put(1, "one", defaultStartTime + 1); windowStore.put(1, "one", defaultStartTime + 1);
final KeyValueIterator<Windowed<Integer>, String> it = windowStore.all(); final KeyValueIterator<Windowed<Integer>, String> it = windowStore.all();
@ -318,7 +317,7 @@ public abstract class AbstractWindowBytesStoreTest {
// A new all() iterator after a previous all() iterator was closed should return all elements. // A new all() iterator after a previous all() iterator was closed should return all elements.
assertEquals( assertEquals(
asList(zero, one), asList(zero, one),
toList(windowStore.all()) toListAndCloseIterator(windowStore.all())
); );
} }
@ -328,7 +327,7 @@ public abstract class AbstractWindowBytesStoreTest {
assertEquals( assertEquals(
asList(five, four, three, two, one, zero), asList(five, four, three, two, one, zero),
toList(windowStore.backwardAll()) toListAndCloseIterator(windowStore.backwardAll())
); );
} }
@ -338,15 +337,15 @@ public abstract class AbstractWindowBytesStoreTest {
assertEquals( assertEquals(
asList(one, two, three, four), asList(one, two, three, four),
toList(windowStore.fetchAll(ofEpochMilli(defaultStartTime + 1), ofEpochMilli(defaultStartTime + 4))) toListAndCloseIterator(windowStore.fetchAll(ofEpochMilli(defaultStartTime + 1), ofEpochMilli(defaultStartTime + 4)))
); );
assertEquals( assertEquals(
asList(zero, one, two, three), asList(zero, one, two, three),
toList(windowStore.fetchAll(ofEpochMilli(defaultStartTime), ofEpochMilli(defaultStartTime + 3))) toListAndCloseIterator(windowStore.fetchAll(ofEpochMilli(defaultStartTime), ofEpochMilli(defaultStartTime + 3)))
); );
assertEquals( assertEquals(
asList(one, two, three, four, five), asList(one, two, three, four, five),
toList(windowStore.fetchAll(ofEpochMilli(defaultStartTime + 1), ofEpochMilli(defaultStartTime + 5))) toListAndCloseIterator(windowStore.fetchAll(ofEpochMilli(defaultStartTime + 1), ofEpochMilli(defaultStartTime + 5)))
); );
} }
@ -356,15 +355,15 @@ public abstract class AbstractWindowBytesStoreTest {
assertEquals( assertEquals(
asList(four, three, two, one), asList(four, three, two, one),
toList(windowStore.backwardFetchAll(ofEpochMilli(defaultStartTime + 1), ofEpochMilli(defaultStartTime + 4))) toListAndCloseIterator(windowStore.backwardFetchAll(ofEpochMilli(defaultStartTime + 1), ofEpochMilli(defaultStartTime + 4)))
); );
assertEquals( assertEquals(
asList(three, two, one, zero), asList(three, two, one, zero),
toList(windowStore.backwardFetchAll(ofEpochMilli(defaultStartTime), ofEpochMilli(defaultStartTime + 3))) toListAndCloseIterator(windowStore.backwardFetchAll(ofEpochMilli(defaultStartTime), ofEpochMilli(defaultStartTime + 3)))
); );
assertEquals( assertEquals(
asList(five, four, three, two, one), asList(five, four, three, two, one),
toList(windowStore.backwardFetchAll(ofEpochMilli(defaultStartTime + 1), ofEpochMilli(defaultStartTime + 5))) toListAndCloseIterator(windowStore.backwardFetchAll(ofEpochMilli(defaultStartTime + 1), ofEpochMilli(defaultStartTime + 5)))
); );
} }
@ -374,55 +373,55 @@ public abstract class AbstractWindowBytesStoreTest {
assertEquals( assertEquals(
asList(zero, one), asList(zero, one),
toList(windowStore.fetch( toListAndCloseIterator(windowStore.fetch(
0, 0,
1, 1,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) ofEpochMilli(defaultStartTime + WINDOW_SIZE)))
); );
assertEquals( assertEquals(
Collections.singletonList(one), Collections.singletonList(one),
toList(windowStore.fetch( toListAndCloseIterator(windowStore.fetch(
1, 1,
1, 1,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) ofEpochMilli(defaultStartTime + WINDOW_SIZE)))
); );
assertEquals( assertEquals(
asList(one, two, three), asList(one, two, three),
toList(windowStore.fetch( toListAndCloseIterator(windowStore.fetch(
1, 1,
3, 3,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) ofEpochMilli(defaultStartTime + WINDOW_SIZE)))
); );
assertEquals( assertEquals(
asList(zero, one, two, three), asList(zero, one, two, three),
toList(windowStore.fetch( toListAndCloseIterator(windowStore.fetch(
0, 0,
5, 5,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) ofEpochMilli(defaultStartTime + WINDOW_SIZE)))
); );
assertEquals( assertEquals(
asList(zero, one, two, three, four, five), asList(zero, one, two, three, four, five),
toList(windowStore.fetch( toListAndCloseIterator(windowStore.fetch(
0, 0,
5, 5,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE + 5L))) ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L)))
); );
assertEquals( assertEquals(
asList(two, three, four, five), asList(two, three, four, five),
toList(windowStore.fetch( toListAndCloseIterator(windowStore.fetch(
0, 0,
5, 5,
ofEpochMilli(defaultStartTime + 2L), ofEpochMilli(defaultStartTime + 2L),
ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE + 5L))) ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L)))
); );
assertEquals( assertEquals(
Collections.emptyList(), Collections.emptyList(),
toList(windowStore.fetch( toListAndCloseIterator(windowStore.fetch(
4, 4,
5, 5,
ofEpochMilli(defaultStartTime + 2L), ofEpochMilli(defaultStartTime + 2L),
@ -430,7 +429,7 @@ public abstract class AbstractWindowBytesStoreTest {
); );
assertEquals( assertEquals(
Collections.emptyList(), Collections.emptyList(),
toList(windowStore.fetch( toListAndCloseIterator(windowStore.fetch(
0, 0,
3, 3,
ofEpochMilli(defaultStartTime + 3L), ofEpochMilli(defaultStartTime + 3L),
@ -438,26 +437,26 @@ public abstract class AbstractWindowBytesStoreTest {
); );
assertEquals( assertEquals(
asList(zero, one, two), asList(zero, one, two),
toList(windowStore.fetch( toListAndCloseIterator(windowStore.fetch(
null, null,
2, 2,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + WINDOW_SIZE + 2L))) ofEpochMilli(defaultStartTime + WINDOW_SIZE + 2L)))
); );
assertEquals( assertEquals(
asList(two, three, four, five), asList(two, three, four, five),
toList(windowStore.fetch( toListAndCloseIterator(windowStore.fetch(
2, 2,
null, null,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L))) ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L)))
); );
assertEquals( assertEquals(
asList(zero, one, two, three, four, five), asList(zero, one, two, three, four, five),
toList(windowStore.fetch( toListAndCloseIterator(windowStore.fetch(
null, null,
null, null,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L))) ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L)))
); );
} }
@ -469,55 +468,55 @@ public abstract class AbstractWindowBytesStoreTest {
assertEquals( assertEquals(
asList(one, zero), asList(one, zero),
toList(windowStore.backwardFetch( toListAndCloseIterator(windowStore.backwardFetch(
0, 0,
1, 1,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) ofEpochMilli(defaultStartTime + WINDOW_SIZE)))
); );
assertEquals( assertEquals(
Collections.singletonList(one), Collections.singletonList(one),
toList(windowStore.backwardFetch( toListAndCloseIterator(windowStore.backwardFetch(
1, 1,
1, 1,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) ofEpochMilli(defaultStartTime + WINDOW_SIZE)))
); );
assertEquals( assertEquals(
asList(three, two, one), asList(three, two, one),
toList(windowStore.backwardFetch( toListAndCloseIterator(windowStore.backwardFetch(
1, 1,
3, 3,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) ofEpochMilli(defaultStartTime + WINDOW_SIZE)))
); );
assertEquals( assertEquals(
asList(three, two, one, zero), asList(three, two, one, zero),
toList(windowStore.backwardFetch( toListAndCloseIterator(windowStore.backwardFetch(
0, 0,
5, 5,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) ofEpochMilli(defaultStartTime + WINDOW_SIZE)))
); );
assertEquals( assertEquals(
asList(five, four, three, two, one, zero), asList(five, four, three, two, one, zero),
toList(windowStore.backwardFetch( toListAndCloseIterator(windowStore.backwardFetch(
0, 0,
5, 5,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE + 5L))) ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L)))
); );
assertEquals( assertEquals(
asList(five, four, three, two), asList(five, four, three, two),
toList(windowStore.backwardFetch( toListAndCloseIterator(windowStore.backwardFetch(
0, 0,
5, 5,
ofEpochMilli(defaultStartTime + 2L), ofEpochMilli(defaultStartTime + 2L),
ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE + 5L))) ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L)))
); );
assertEquals( assertEquals(
Collections.emptyList(), Collections.emptyList(),
toList(windowStore.backwardFetch( toListAndCloseIterator(windowStore.backwardFetch(
4, 4,
5, 5,
ofEpochMilli(defaultStartTime + 2L), ofEpochMilli(defaultStartTime + 2L),
@ -525,7 +524,7 @@ public abstract class AbstractWindowBytesStoreTest {
); );
assertEquals( assertEquals(
Collections.emptyList(), Collections.emptyList(),
toList(windowStore.backwardFetch( toListAndCloseIterator(windowStore.backwardFetch(
0, 0,
3, 3,
ofEpochMilli(defaultStartTime + 3L), ofEpochMilli(defaultStartTime + 3L),
@ -533,26 +532,26 @@ public abstract class AbstractWindowBytesStoreTest {
); );
assertEquals( assertEquals(
asList(two, one, zero), asList(two, one, zero),
toList(windowStore.backwardFetch( toListAndCloseIterator(windowStore.backwardFetch(
null, null,
2, 2,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + WINDOW_SIZE + 2L))) ofEpochMilli(defaultStartTime + WINDOW_SIZE + 2L)))
); );
assertEquals( assertEquals(
asList(five, four, three, two), asList(five, four, three, two),
toList(windowStore.backwardFetch( toListAndCloseIterator(windowStore.backwardFetch(
2, 2,
null, null,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L))) ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L)))
); );
assertEquals( assertEquals(
asList(five, four, three, two, one, zero), asList(five, four, three, two, one, zero),
toList(windowStore.backwardFetch( toListAndCloseIterator(windowStore.backwardFetch(
null, null,
null, null,
ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L))) ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L)))
); );
} }
@ -563,70 +562,70 @@ public abstract class AbstractWindowBytesStoreTest {
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("zero")), new HashSet<>(Collections.singletonList("zero")),
valuesToSet(windowStore.fetch(0, ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 0L)))); valuesToSetAndCloseIterator(windowStore.fetch(0, ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("one")), new HashSet<>(Collections.singletonList("one")),
valuesToSet(windowStore.fetch(1, ofEpochMilli(defaultStartTime + 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 1L)))); valuesToSetAndCloseIterator(windowStore.fetch(1, ofEpochMilli(defaultStartTime + 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 1L))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("two")), new HashSet<>(Collections.singletonList("two")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 2L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 2L))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("three")), new HashSet<>(Collections.singletonList("three")),
valuesToSet(windowStore.fetch(3, ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 3L)))); valuesToSetAndCloseIterator(windowStore.fetch(3, ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 3L))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("four")), new HashSet<>(Collections.singletonList("four")),
valuesToSet(windowStore.fetch(4, ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 4L)))); valuesToSetAndCloseIterator(windowStore.fetch(4, ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 4L))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("five")), new HashSet<>(Collections.singletonList("five")),
valuesToSet(windowStore.fetch(5, ofEpochMilli(defaultStartTime + 5L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 5L)))); valuesToSetAndCloseIterator(windowStore.fetch(5, ofEpochMilli(defaultStartTime + 5L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 5L))));
putSecondBatch(windowStore, defaultStartTime, context); putSecondBatch(windowStore, defaultStartTime);
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime - 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - 1L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime - 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - 1L))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 0L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 1L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 1L))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("two")), new HashSet<>(Collections.singletonList("two")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 2L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 2L))));
assertEquals( assertEquals(
new HashSet<>(asList("two", "two+1")), new HashSet<>(asList("two", "two+1")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 3L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 3L))));
assertEquals( assertEquals(
new HashSet<>(asList("two", "two+1", "two+2")), new HashSet<>(asList("two", "two+1", "two+2")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 4L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 4L))));
assertEquals( assertEquals(
new HashSet<>(asList("two", "two+1", "two+2", "two+3")), new HashSet<>(asList("two", "two+1", "two+2", "two+3")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 5L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 5L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 5L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 5L))));
assertEquals( assertEquals(
new HashSet<>(asList("two+1", "two+2", "two+3", "two+4")), new HashSet<>(asList("two+1", "two+2", "two+3", "two+4")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 6L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 6L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 6L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 6L))));
assertEquals( assertEquals(
new HashSet<>(asList("two+2", "two+3", "two+4", "two+5")), new HashSet<>(asList("two+2", "two+3", "two+4", "two+5")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 7L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 7L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 7L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 7L))));
assertEquals( assertEquals(
new HashSet<>(asList("two+3", "two+4", "two+5", "two+6")), new HashSet<>(asList("two+3", "two+4", "two+5", "two+6")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 8L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 8L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 8L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 8L))));
assertEquals( assertEquals(
new HashSet<>(asList("two+4", "two+5", "two+6")), new HashSet<>(asList("two+4", "two+5", "two+6")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 9L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 9L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 9L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 9L))));
assertEquals( assertEquals(
new HashSet<>(asList("two+5", "two+6")), new HashSet<>(asList("two+5", "two+6")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 10L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 10L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 10L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 10L))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("two+6")), new HashSet<>(Collections.singletonList("two+6")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 11L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 11L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 11L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 11L))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 12L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 12L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 12L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 12L))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 13L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 13L)))); valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 13L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 13L))));
// Flush the store and verify all current entries were properly flushed ... // Flush the store and verify all current entries were properly flushed ...
windowStore.flush(); windowStore.flush();
@ -652,90 +651,90 @@ public abstract class AbstractWindowBytesStoreTest {
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("zero")), new HashSet<>(Collections.singletonList("zero")),
valuesToSet(windowStore.fetch(0, ofEpochMilli(defaultStartTime + 0L), valuesToSetAndCloseIterator(windowStore.fetch(0, ofEpochMilli(defaultStartTime),
ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("one")), new HashSet<>(Collections.singletonList("one")),
valuesToSet(windowStore.fetch(1, ofEpochMilli(defaultStartTime + 1L), valuesToSetAndCloseIterator(windowStore.fetch(1, ofEpochMilli(defaultStartTime + 1L),
ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("two")), new HashSet<>(Collections.singletonList("two")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L),
ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch(3, ofEpochMilli(defaultStartTime + 3L), valuesToSetAndCloseIterator(windowStore.fetch(3, ofEpochMilli(defaultStartTime + 3L),
ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("four")), new HashSet<>(Collections.singletonList("four")),
valuesToSet(windowStore.fetch(4, ofEpochMilli(defaultStartTime + 4L), valuesToSetAndCloseIterator(windowStore.fetch(4, ofEpochMilli(defaultStartTime + 4L),
ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("five")), new HashSet<>(Collections.singletonList("five")),
valuesToSet(windowStore.fetch(5, ofEpochMilli(defaultStartTime + 5L), valuesToSetAndCloseIterator(windowStore.fetch(5, ofEpochMilli(defaultStartTime + 5L),
ofEpochMilli(defaultStartTime + 5L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 5L + WINDOW_SIZE))));
putSecondBatch(windowStore, defaultStartTime, context); putSecondBatch(windowStore, defaultStartTime);
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime - 2L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime - 2L),
ofEpochMilli(defaultStartTime - 2L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime - 2L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("two")), new HashSet<>(Collections.singletonList("two")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime - 1L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime - 1L),
ofEpochMilli(defaultStartTime - 1L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime - 1L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two", "two+1")), new HashSet<>(asList("two", "two+1")),
valuesToSet(windowStore valuesToSetAndCloseIterator(windowStore
.fetch(2, ofEpochMilli(defaultStartTime), ofEpochMilli(defaultStartTime + WINDOW_SIZE)))); .fetch(2, ofEpochMilli(defaultStartTime), ofEpochMilli(defaultStartTime + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two", "two+1", "two+2")), new HashSet<>(asList("two", "two+1", "two+2")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 1L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 1L),
ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two", "two+1", "two+2", "two+3")), new HashSet<>(asList("two", "two+1", "two+2", "two+3")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L),
ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two+1", "two+2", "two+3", "two+4")), new HashSet<>(asList("two+1", "two+2", "two+3", "two+4")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 3L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 3L),
ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two+2", "two+3", "two+4", "two+5")), new HashSet<>(asList("two+2", "two+3", "two+4", "two+5")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 4L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 4L),
ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two+3", "two+4", "two+5", "two+6")), new HashSet<>(asList("two+3", "two+4", "two+5", "two+6")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 5L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 5L),
ofEpochMilli(defaultStartTime + 5L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 5L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two+4", "two+5", "two+6")), new HashSet<>(asList("two+4", "two+5", "two+6")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 6L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 6L),
ofEpochMilli(defaultStartTime + 6L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 6L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("two+5", "two+6")), new HashSet<>(asList("two+5", "two+6")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 7L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 7L),
ofEpochMilli(defaultStartTime + 7L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 7L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("two+6")), new HashSet<>(Collections.singletonList("two+6")),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 8L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 8L),
ofEpochMilli(defaultStartTime + 8L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 8L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 9L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 9L),
ofEpochMilli(defaultStartTime + 9L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 9L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 10L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 10L),
ofEpochMilli(defaultStartTime + 10L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 10L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 11L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 11L),
ofEpochMilli(defaultStartTime + 11L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 11L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 12L), valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 12L),
ofEpochMilli(defaultStartTime + 12L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 12L + WINDOW_SIZE))));
// Flush the store and verify all current entries were properly flushed ... // Flush the store and verify all current entries were properly flushed ...
@ -763,13 +762,13 @@ public abstract class AbstractWindowBytesStoreTest {
public void testPutSameKeyTimestamp() { public void testPutSameKeyTimestamp() {
windowStore.close(); windowStore.close();
windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), Serdes.String()); windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), Serdes.String());
windowStore.init((StateStoreContext) context, windowStore); windowStore.init(context, windowStore);
windowStore.put(0, "zero", defaultStartTime); windowStore.put(0, "zero", defaultStartTime);
assertEquals( assertEquals(
new HashSet<>(Collections.singletonList("zero")), new HashSet<>(Collections.singletonList("zero")),
valuesToSet(windowStore.fetch(0, ofEpochMilli(defaultStartTime - WINDOW_SIZE), valuesToSetAndCloseIterator(windowStore.fetch(0, ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + WINDOW_SIZE))));
windowStore.put(0, "zero", defaultStartTime); windowStore.put(0, "zero", defaultStartTime);
@ -778,31 +777,31 @@ public abstract class AbstractWindowBytesStoreTest {
assertEquals( assertEquals(
new HashSet<>(asList("zero", "zero", "zero+", "zero++")), new HashSet<>(asList("zero", "zero", "zero+", "zero++")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
0, 0,
ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("zero", "zero", "zero+", "zero++")), new HashSet<>(asList("zero", "zero", "zero+", "zero++")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
0, 0,
ofEpochMilli(defaultStartTime + 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 1L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("zero", "zero", "zero+", "zero++")), new HashSet<>(asList("zero", "zero", "zero+", "zero++")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
0, 0,
ofEpochMilli(defaultStartTime + 2L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 2L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(asList("zero", "zero", "zero+", "zero++")), new HashSet<>(asList("zero", "zero", "zero+", "zero++")),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
0, 0,
ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE))));
assertEquals( assertEquals(
new HashSet<>(Collections.emptyList()), new HashSet<>(Collections.emptyList()),
valuesToSet(windowStore.fetch( valuesToSetAndCloseIterator(windowStore.fetch(
0, 0,
ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE)))); ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE))));
@ -845,7 +844,7 @@ public abstract class AbstractWindowBytesStoreTest {
Serdes.String(), Serdes.String(),
Serdes.String()); Serdes.String());
windowStore.init((StateStoreContext) context, windowStore); windowStore.init(context, windowStore);
windowStore.put("a", "0001", 0); windowStore.put("a", "0001", 0);
windowStore.put("aa", "0002", 0); windowStore.put("aa", "0002", 0);
@ -855,7 +854,7 @@ public abstract class AbstractWindowBytesStoreTest {
final Set<String> expected = new HashSet<>(asList("0001", "0003", "0005")); final Set<String> expected = new HashSet<>(asList("0001", "0003", "0005"));
assertThat( assertThat(
valuesToSet(windowStore.fetch("a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), valuesToSetAndCloseIterator(windowStore.fetch("a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))),
equalTo(expected) equalTo(expected)
); );
@ -887,13 +886,15 @@ public abstract class AbstractWindowBytesStoreTest {
windowStore.put(1, "one", currentTime); windowStore.put(1, "one", currentTime);
windowStore.put(1, "one v2", currentTime); windowStore.put(1, "one v2", currentTime);
WindowStoreIterator<String> iterator = windowStore.fetch(1, 0, currentTime); try (final WindowStoreIterator<String> iterator = windowStore.fetch(1, 0, currentTime)) {
assertEquals(new KeyValue<>(currentTime, "one v2"), iterator.next()); assertEquals(new KeyValue<>(currentTime, "one v2"), iterator.next());
}
windowStore.put(1, null, currentTime); windowStore.put(1, null, currentTime);
iterator = windowStore.fetch(1, 0, currentTime); try (final WindowStoreIterator<String> iterator = windowStore.fetch(1, 0, currentTime)) {
assertFalse(iterator.hasNext()); assertFalse(iterator.hasNext());
} }
}
@Test @Test
public void shouldReturnNullOnWindowNotFound() { public void shouldReturnNullOnWindowNotFound() {
@ -905,6 +906,7 @@ public abstract class AbstractWindowBytesStoreTest {
assertThrows(NullPointerException.class, () -> windowStore.put(null, "anyValue", 0L)); assertThrows(NullPointerException.class, () -> windowStore.put(null, "anyValue", 0L));
} }
@SuppressWarnings("resource")
@Test @Test
public void shouldThrowNullPointerExceptionOnGetNullKey() { public void shouldThrowNullPointerExceptionOnGetNullKey() {
assertThrows(NullPointerException.class, () -> windowStore.fetch(null, ofEpochMilli(1L), ofEpochMilli(2L))); assertThrows(NullPointerException.class, () -> windowStore.fetch(null, ofEpochMilli(1L), ofEpochMilli(2L)));
@ -917,7 +919,7 @@ public abstract class AbstractWindowBytesStoreTest {
true, true,
Serdes.Bytes(), Serdes.Bytes(),
Serdes.String()); Serdes.String());
windowStore.init((StateStoreContext) context, windowStore); windowStore.init(context, windowStore);
final Bytes key1 = Bytes.wrap(new byte[] {0}); final Bytes key1 = Bytes.wrap(new byte[] {0});
final Bytes key2 = Bytes.wrap(new byte[] {0, 0}); final Bytes key2 = Bytes.wrap(new byte[] {0, 0});
@ -934,17 +936,17 @@ public abstract class AbstractWindowBytesStoreTest {
final Set<String> expectedKey1 = new HashSet<>(asList("1", "4", "7")); final Set<String> expectedKey1 = new HashSet<>(asList("1", "4", "7"));
assertThat( assertThat(
valuesToSet(windowStore.fetch(key1, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), valuesToSetAndCloseIterator(windowStore.fetch(key1, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))),
equalTo(expectedKey1) equalTo(expectedKey1)
); );
final Set<String> expectedKey2 = new HashSet<>(asList("2", "5", "8")); final Set<String> expectedKey2 = new HashSet<>(asList("2", "5", "8"));
assertThat( assertThat(
valuesToSet(windowStore.fetch(key2, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), valuesToSetAndCloseIterator(windowStore.fetch(key2, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))),
equalTo(expectedKey2) equalTo(expectedKey2)
); );
final Set<String> expectedKey3 = new HashSet<>(asList("3", "6", "9")); final Set<String> expectedKey3 = new HashSet<>(asList("3", "6", "9"));
assertThat( assertThat(
valuesToSet(windowStore.fetch(key3, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), valuesToSetAndCloseIterator(windowStore.fetch(key3, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))),
equalTo(expectedKey3) equalTo(expectedKey3)
); );
@ -990,7 +992,7 @@ public abstract class AbstractWindowBytesStoreTest {
final Properties streamsConfig = StreamsTestUtils.getStreamsConfig(); final Properties streamsConfig = StreamsTestUtils.getStreamsConfig();
final WindowStore<Integer, String> windowStore = final WindowStore<Integer, String> windowStore =
buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, false, Serdes.Integer(), Serdes.String()); buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, false, Serdes.Integer(), Serdes.String());
final InternalMockProcessorContext context = new InternalMockProcessorContext( final InternalMockProcessorContext<?, ?> context = new InternalMockProcessorContext<>(
TestUtils.tempDirectory(), TestUtils.tempDirectory(),
new StreamsConfig(streamsConfig), new StreamsConfig(streamsConfig),
recordCollector recordCollector
@ -998,7 +1000,7 @@ public abstract class AbstractWindowBytesStoreTest {
final Time time = Time.SYSTEM; final Time time = Time.SYSTEM;
context.setSystemTimeMs(time.milliseconds()); context.setSystemTimeMs(time.milliseconds());
context.setTime(1L); context.setTime(1L);
windowStore.init((StateStoreContext) context, windowStore); windowStore.init(context, windowStore);
// Advance stream time by inserting record with large enough timestamp that records with timestamp 0 are expired // Advance stream time by inserting record with large enough timestamp that records with timestamp 0 are expired
windowStore.put(1, "initial record", 2 * RETENTION_PERIOD); windowStore.put(1, "initial record", 2 * RETENTION_PERIOD);
@ -1108,7 +1110,7 @@ public abstract class AbstractWindowBytesStoreTest {
public void testFetchDuplicates() { public void testFetchDuplicates() {
windowStore.close(); windowStore.close();
windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), Serdes.String()); windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), Serdes.String());
windowStore.init((StateStoreContext) context, windowStore); windowStore.init(context, windowStore);
long currentTime = 0; long currentTime = 0;
windowStore.put(1, "one", currentTime); windowStore.put(1, "one", currentTime);
@ -1135,7 +1137,7 @@ public abstract class AbstractWindowBytesStoreTest {
private void putFirstBatch(final WindowStore<Integer, String> store, private void putFirstBatch(final WindowStore<Integer, String> store,
@SuppressWarnings("SameParameterValue") final long startTime, @SuppressWarnings("SameParameterValue") final long startTime,
final InternalMockProcessorContext context) { final InternalMockProcessorContext<?, ?> context) {
context.setRecordContext(createRecordContext(startTime)); context.setRecordContext(createRecordContext(startTime));
store.put(0, "zero", startTime); store.put(0, "zero", startTime);
store.put(1, "one", startTime + 1L); store.put(1, "one", startTime + 1L);
@ -1146,8 +1148,7 @@ public abstract class AbstractWindowBytesStoreTest {
} }
private void putSecondBatch(final WindowStore<Integer, String> store, private void putSecondBatch(final WindowStore<Integer, String> store,
@SuppressWarnings("SameParameterValue") final long startTime, @SuppressWarnings("SameParameterValue") final long startTime) {
final InternalMockProcessorContext context) {
store.put(2, "two+1", startTime + 3L); store.put(2, "two+1", startTime + 3L);
store.put(2, "two+2", startTime + 4L); store.put(2, "two+2", startTime + 4L);
store.put(2, "two+3", startTime + 5L); store.put(2, "two+3", startTime + 5L);

View File

@ -60,7 +60,7 @@ import java.util.Random;
import static java.util.Arrays.asList; import static java.util.Arrays.asList;
import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator;
import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList; import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList;
import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue; import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue;
import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.CoreMatchers.hasItem;
@ -401,7 +401,7 @@ public class CachingInMemorySessionStoreTest {
// add one that shouldn't appear in the results // add one that shouldn't appear in the results
cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "5".getBytes()); cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "5".getBytes());
final List<KeyValue<Windowed<Bytes>, byte[]>> results = toList(cachingStore.fetch(keyA)); final List<KeyValue<Windowed<Bytes>, byte[]>> results = toListAndCloseIterator(cachingStore.fetch(keyA));
verifyKeyValueList(expected, results); verifyKeyValueList(expected, results);
} }
@ -420,7 +420,7 @@ public class CachingInMemorySessionStoreTest {
// add one that shouldn't appear in the results // add one that shouldn't appear in the results
cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "5".getBytes()); cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "5".getBytes());
final List<KeyValue<Windowed<Bytes>, byte[]>> results = toList(cachingStore.backwardFetch(keyA)); final List<KeyValue<Windowed<Bytes>, byte[]>> results = toListAndCloseIterator(cachingStore.backwardFetch(keyA));
Collections.reverse(results); Collections.reverse(results);
verifyKeyValueList(expected, results); verifyKeyValueList(expected, results);
} }
@ -439,7 +439,7 @@ public class CachingInMemorySessionStoreTest {
@Test @Test
public void shouldQueryItemsInCacheAndStore() { public void shouldQueryItemsInCacheAndStore() {
final List<KeyValue<Windowed<Bytes>, byte[]>> added = addSessionsUntilOverflow("a"); final List<KeyValue<Windowed<Bytes>, byte[]>> added = addSessionsUntilOverflow("a");
final List<KeyValue<Windowed<Bytes>, byte[]>> actual = toList(cachingStore.findSessions( final List<KeyValue<Windowed<Bytes>, byte[]>> actual = toListAndCloseIterator(cachingStore.findSessions(
Bytes.wrap("a".getBytes(StandardCharsets.UTF_8)), Bytes.wrap("a".getBytes(StandardCharsets.UTF_8)),
0, 0,
added.size() * 10L)); added.size() * 10L));

View File

@ -60,7 +60,7 @@ import java.util.Random;
import static java.util.Arrays.asList; import static java.util.Arrays.asList;
import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator;
import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList; import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList;
import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue; import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue;
import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.CoreMatchers.hasItem;
@ -409,7 +409,7 @@ public class CachingPersistentSessionStoreTest {
// add one that shouldn't appear in the results // add one that shouldn't appear in the results
cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "5".getBytes()); cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "5".getBytes());
final List<KeyValue<Windowed<Bytes>, byte[]>> results = toList(cachingStore.fetch(keyA)); final List<KeyValue<Windowed<Bytes>, byte[]>> results = toListAndCloseIterator(cachingStore.fetch(keyA));
verifyKeyValueList(expected, results); verifyKeyValueList(expected, results);
} }
@ -428,7 +428,7 @@ public class CachingPersistentSessionStoreTest {
// add one that shouldn't appear in the results // add one that shouldn't appear in the results
cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "5".getBytes()); cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "5".getBytes());
final List<KeyValue<Windowed<Bytes>, byte[]>> results = toList(cachingStore.backwardFetch(keyA)); final List<KeyValue<Windowed<Bytes>, byte[]>> results = toListAndCloseIterator(cachingStore.backwardFetch(keyA));
Collections.reverse(results); Collections.reverse(results);
verifyKeyValueList(expected, results); verifyKeyValueList(expected, results);
} }
@ -448,7 +448,7 @@ public class CachingPersistentSessionStoreTest {
@Test @Test
public void shouldQueryItemsInCacheAndStore() { public void shouldQueryItemsInCacheAndStore() {
final List<KeyValue<Windowed<Bytes>, byte[]>> added = addSessionsUntilOverflow("a"); final List<KeyValue<Windowed<Bytes>, byte[]>> added = addSessionsUntilOverflow("a");
final List<KeyValue<Windowed<Bytes>, byte[]>> actual = toList(cachingStore.findSessions( final List<KeyValue<Windowed<Bytes>, byte[]>> actual = toListAndCloseIterator(cachingStore.findSessions(
Bytes.wrap("a".getBytes(StandardCharsets.UTF_8)), Bytes.wrap("a".getBytes(StandardCharsets.UTF_8)),
0, 0,
added.size() * 10L added.size() * 10L

View File

@ -73,7 +73,7 @@ import static java.util.Arrays.asList;
import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize; import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize;
import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator;
import static org.apache.kafka.test.StreamsTestUtils.verifyAllWindowedKeyValues; import static org.apache.kafka.test.StreamsTestUtils.verifyAllWindowedKeyValues;
import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList; import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList;
import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue; import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue;
@ -844,7 +844,7 @@ public class CachingPersistentWindowStoreTest {
KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")) KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005"))
); );
final List<KeyValue<Long, byte[]>> actual = final List<KeyValue<Long, byte[]>> actual =
toList(cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)));
verifyKeyValueList(expected, actual); verifyKeyValueList(expected, actual);
} }
@ -862,7 +862,7 @@ public class CachingPersistentWindowStoreTest {
KeyValue.pair(0L, bytesValue("0001")) KeyValue.pair(0L, bytesValue("0001"))
); );
final List<KeyValue<Long, byte[]>> actual = final List<KeyValue<Long, byte[]>> actual =
toList(cachingStore.backwardFetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)));
verifyKeyValueList(expected, actual); verifyKeyValueList(expected, actual);
} }
@ -880,14 +880,14 @@ public class CachingPersistentWindowStoreTest {
windowedPair("a", "0003", 1), windowedPair("a", "0003", 1),
windowedPair("a", "0005", SEGMENT_INTERVAL) windowedPair("a", "0005", SEGMENT_INTERVAL)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
verifyKeyValueList( verifyKeyValueList(
asList( asList(
windowedPair("aa", "0002", 0), windowedPair("aa", "0002", 0),
windowedPair("aa", "0004", 1)), windowedPair("aa", "0004", 1)),
toList(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
verifyKeyValueList( verifyKeyValueList(
@ -898,7 +898,7 @@ public class CachingPersistentWindowStoreTest {
windowedPair("aa", "0004", 1), windowedPair("aa", "0004", 1),
windowedPair("a", "0005", SEGMENT_INTERVAL) windowedPair("a", "0005", SEGMENT_INTERVAL)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
} }
@ -916,14 +916,14 @@ public class CachingPersistentWindowStoreTest {
windowedPair("a", "0003", 1), windowedPair("a", "0003", 1),
windowedPair("a", "0001", 0) windowedPair("a", "0001", 0)
), ),
toList(cachingStore.backwardFetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
verifyKeyValueList( verifyKeyValueList(
asList( asList(
windowedPair("aa", "0004", 1), windowedPair("aa", "0004", 1),
windowedPair("aa", "0002", 0)), windowedPair("aa", "0002", 0)),
toList(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
verifyKeyValueList( verifyKeyValueList(
@ -934,7 +934,7 @@ public class CachingPersistentWindowStoreTest {
windowedPair("a", "0003", 1), windowedPair("a", "0003", 1),
windowedPair("a", "0001", 0) windowedPair("a", "0001", 0)
), ),
toList(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
} }

View File

@ -43,7 +43,7 @@ import java.util.NoSuchElementException;
import static java.util.Arrays.asList; import static java.util.Arrays.asList;
import static java.util.Collections.singletonList; import static java.util.Collections.singletonList;
import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator;
import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertNull;
@ -276,7 +276,7 @@ public class CompositeReadOnlyKeyValueStoreTest {
stubOneUnderlying.put("b", "b"); stubOneUnderlying.put("b", "b");
stubOneUnderlying.put("c", "c"); stubOneUnderlying.put("c", "c");
final List<KeyValue<String, String>> results = toList(theStore.range("a", "b")); final List<KeyValue<String, String>> results = toListAndCloseIterator(theStore.range("a", "b"));
assertTrue(results.contains(new KeyValue<>("a", "a"))); assertTrue(results.contains(new KeyValue<>("a", "a")));
assertTrue(results.contains(new KeyValue<>("b", "b"))); assertTrue(results.contains(new KeyValue<>("b", "b")));
assertEquals(2, results.size()); assertEquals(2, results.size());
@ -288,7 +288,7 @@ public class CompositeReadOnlyKeyValueStoreTest {
stubOneUnderlying.put("b", "b"); stubOneUnderlying.put("b", "b");
stubOneUnderlying.put("c", "c"); stubOneUnderlying.put("c", "c");
final List<KeyValue<String, String>> results = toList(theStore.reverseRange("a", "b")); final List<KeyValue<String, String>> results = toListAndCloseIterator(theStore.reverseRange("a", "b"));
assertArrayEquals( assertArrayEquals(
asList( asList(
new KeyValue<>("b", "b"), new KeyValue<>("b", "b"),
@ -303,7 +303,7 @@ public class CompositeReadOnlyKeyValueStoreTest {
stubOneUnderlying.put("abcd", "b"); stubOneUnderlying.put("abcd", "b");
stubOneUnderlying.put("abce", "c"); stubOneUnderlying.put("abce", "c");
final List<KeyValue<String, String>> results = toList(theStore.prefixScan("abcd", new StringSerializer())); final List<KeyValue<String, String>> results = toListAndCloseIterator(theStore.prefixScan("abcd", new StringSerializer()));
assertTrue(results.contains(new KeyValue<>("abcd", "b"))); assertTrue(results.contains(new KeyValue<>("abcd", "b")));
assertEquals(1, results.size()); assertEquals(1, results.size());
} }
@ -314,7 +314,7 @@ public class CompositeReadOnlyKeyValueStoreTest {
stubOneUnderlying.put("aa", "b"); stubOneUnderlying.put("aa", "b");
stubOneUnderlying.put("b", "c"); stubOneUnderlying.put("b", "c");
final List<KeyValue<String, String>> results = toList(theStore.prefixScan("a", new StringSerializer())); final List<KeyValue<String, String>> results = toListAndCloseIterator(theStore.prefixScan("a", new StringSerializer()));
assertTrue(results.contains(new KeyValue<>("a", "a"))); assertTrue(results.contains(new KeyValue<>("a", "a")));
assertTrue(results.contains(new KeyValue<>("aa", "b"))); assertTrue(results.contains(new KeyValue<>("aa", "b")));
assertEquals(2, results.size()); assertEquals(2, results.size());
@ -333,7 +333,7 @@ public class CompositeReadOnlyKeyValueStoreTest {
cache.put("d", "d"); cache.put("d", "d");
cache.put("x", "x"); cache.put("x", "x");
final List<KeyValue<String, String>> results = toList(theStore.range("a", "e")); final List<KeyValue<String, String>> results = toListAndCloseIterator(theStore.range("a", "e"));
assertArrayEquals( assertArrayEquals(
asList( asList(
new KeyValue<>("a", "a"), new KeyValue<>("a", "a"),
@ -357,7 +357,7 @@ public class CompositeReadOnlyKeyValueStoreTest {
cache.put("ab", "d"); cache.put("ab", "d");
cache.put("x", "x"); cache.put("x", "x");
final List<KeyValue<String, String>> results = toList(theStore.prefixScan("a", new StringSerializer())); final List<KeyValue<String, String>> results = toListAndCloseIterator(theStore.prefixScan("a", new StringSerializer()));
assertArrayEquals( assertArrayEquals(
asList( asList(
new KeyValue<>("a", "a"), new KeyValue<>("a", "a"),
@ -380,7 +380,7 @@ public class CompositeReadOnlyKeyValueStoreTest {
cache.put("d", "d"); cache.put("d", "d");
cache.put("x", "x"); cache.put("x", "x");
final List<KeyValue<String, String>> results = toList(theStore.reverseRange("a", "e")); final List<KeyValue<String, String>> results = toListAndCloseIterator(theStore.reverseRange("a", "e"));
assertTrue(results.contains(new KeyValue<>("a", "a"))); assertTrue(results.contains(new KeyValue<>("a", "a")));
assertTrue(results.contains(new KeyValue<>("b", "b"))); assertTrue(results.contains(new KeyValue<>("b", "b")));
assertTrue(results.contains(new KeyValue<>("c", "c"))); assertTrue(results.contains(new KeyValue<>("c", "c")));
@ -401,7 +401,7 @@ public class CompositeReadOnlyKeyValueStoreTest {
cache.put("d", "d"); cache.put("d", "d");
cache.put("x", "x"); cache.put("x", "x");
final List<KeyValue<String, String>> results = toList(theStore.all()); final List<KeyValue<String, String>> results = toListAndCloseIterator(theStore.all());
assertTrue(results.contains(new KeyValue<>("a", "a"))); assertTrue(results.contains(new KeyValue<>("a", "a")));
assertTrue(results.contains(new KeyValue<>("b", "b"))); assertTrue(results.contains(new KeyValue<>("b", "b")));
assertTrue(results.contains(new KeyValue<>("c", "c"))); assertTrue(results.contains(new KeyValue<>("c", "c")));
@ -424,7 +424,7 @@ public class CompositeReadOnlyKeyValueStoreTest {
cache.put("d", "d"); cache.put("d", "d");
cache.put("x", "x"); cache.put("x", "x");
final List<KeyValue<String, String>> results = toList(theStore.reverseAll()); final List<KeyValue<String, String>> results = toListAndCloseIterator(theStore.reverseAll());
assertTrue(results.contains(new KeyValue<>("a", "a"))); assertTrue(results.contains(new KeyValue<>("a", "a")));
assertTrue(results.contains(new KeyValue<>("b", "b"))); assertTrue(results.contains(new KeyValue<>("b", "b")));
assertTrue(results.contains(new KeyValue<>("c", "c"))); assertTrue(results.contains(new KeyValue<>("c", "c")));

View File

@ -36,7 +36,7 @@ import java.util.Arrays;
import java.util.List; import java.util.List;
import static java.util.Collections.singletonList; import static java.util.Collections.singletonList;
import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.core.IsEqual.equalTo; import static org.hamcrest.core.IsEqual.equalTo;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
@ -69,7 +69,7 @@ public class CompositeReadOnlySessionStoreTest {
underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L); underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L);
underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(10, 10)), 2L); underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(10, 10)), 2L);
final List<KeyValue<Windowed<String>, Long>> results = toList(sessionStore.fetch("a")); final List<KeyValue<Windowed<String>, Long>> results = toListAndCloseIterator(sessionStore.fetch("a"));
assertEquals(Arrays.asList(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 1L), assertEquals(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(10, 10)), 2L)),
results); results);
@ -93,8 +93,8 @@ public class CompositeReadOnlySessionStoreTest {
underlyingSessionStore.put(keyOne, 0L); underlyingSessionStore.put(keyOne, 0L);
secondUnderlying.put(keyTwo, 10L); secondUnderlying.put(keyTwo, 10L);
final List<KeyValue<Windowed<String>, Long>> keyOneResults = toList(sessionStore.fetch("key-one")); final List<KeyValue<Windowed<String>, Long>> keyOneResults = toListAndCloseIterator(sessionStore.fetch("key-one"));
final List<KeyValue<Windowed<String>, Long>> keyTwoResults = toList(sessionStore.fetch("key-two")); final List<KeyValue<Windowed<String>, Long>> keyTwoResults = toListAndCloseIterator(sessionStore.fetch("key-two"));
assertEquals(singletonList(KeyValue.pair(keyOne, 0L)), keyOneResults); assertEquals(singletonList(KeyValue.pair(keyOne, 0L)), keyOneResults);
assertEquals(singletonList(KeyValue.pair(keyTwo, 10L)), keyTwoResults); assertEquals(singletonList(KeyValue.pair(keyTwo, 10L)), keyTwoResults);
@ -146,7 +146,7 @@ public class CompositeReadOnlySessionStoreTest {
stubProviderTwo.addStore(storeName, secondUnderlying); stubProviderTwo.addStore(storeName, secondUnderlying);
underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 0L); underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 0L);
secondUnderlying.put(new Windowed<>("b", new SessionWindow(0, 0)), 10L); secondUnderlying.put(new Windowed<>("b", new SessionWindow(0, 0)), 10L);
final List<KeyValue<Windowed<String>, Long>> results = StreamsTestUtils.toList(sessionStore.fetch("a", "b")); final List<KeyValue<Windowed<String>, Long>> results = StreamsTestUtils.toListAndCloseIterator(sessionStore.fetch("a", "b"));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 0L), KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 0L),
KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), 10L)))); KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), 10L))));
@ -159,7 +159,7 @@ public class CompositeReadOnlySessionStoreTest {
stubProviderTwo.addStore(storeName, secondUnderlying); stubProviderTwo.addStore(storeName, secondUnderlying);
underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 0L); underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 0L);
secondUnderlying.put(new Windowed<>("b", new SessionWindow(0, 0)), 10L); secondUnderlying.put(new Windowed<>("b", new SessionWindow(0, 0)), 10L);
final List<KeyValue<Windowed<String>, Long>> results = StreamsTestUtils.toList(sessionStore.fetch(null, "b")); final List<KeyValue<Windowed<String>, Long>> results = StreamsTestUtils.toListAndCloseIterator(sessionStore.fetch(null, "b"));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 0L), KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 0L),
KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), 10L)))); KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), 10L))));
@ -172,7 +172,7 @@ public class CompositeReadOnlySessionStoreTest {
stubProviderTwo.addStore(storeName, secondUnderlying); stubProviderTwo.addStore(storeName, secondUnderlying);
underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 0L); underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 0L);
secondUnderlying.put(new Windowed<>("b", new SessionWindow(0, 0)), 10L); secondUnderlying.put(new Windowed<>("b", new SessionWindow(0, 0)), 10L);
final List<KeyValue<Windowed<String>, Long>> results = StreamsTestUtils.toList(sessionStore.fetch("a", null)); final List<KeyValue<Windowed<String>, Long>> results = StreamsTestUtils.toListAndCloseIterator(sessionStore.fetch("a", null));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 0L), KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 0L),
KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), 10L)))); KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), 10L))));
@ -185,7 +185,7 @@ public class CompositeReadOnlySessionStoreTest {
stubProviderTwo.addStore(storeName, secondUnderlying); stubProviderTwo.addStore(storeName, secondUnderlying);
underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 0L); underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 0L);
secondUnderlying.put(new Windowed<>("b", new SessionWindow(0, 0)), 10L); secondUnderlying.put(new Windowed<>("b", new SessionWindow(0, 0)), 10L);
final List<KeyValue<Windowed<String>, Long>> results = StreamsTestUtils.toList(sessionStore.fetch(null, null)); final List<KeyValue<Windowed<String>, Long>> results = StreamsTestUtils.toListAndCloseIterator(sessionStore.fetch(null, null));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 0L), KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 0L),
KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), 10L)))); KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), 10L))));

View File

@ -90,7 +90,7 @@ public class CompositeReadOnlyWindowStoreTest {
assertEquals( assertEquals(
asList(new KeyValue<>(0L, "my-value"), new KeyValue<>(10L, "my-later-value")), asList(new KeyValue<>(0L, "my-value"), new KeyValue<>(10L, "my-later-value")),
StreamsTestUtils.toList(windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L))) StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)))
); );
} }
@ -102,7 +102,7 @@ public class CompositeReadOnlyWindowStoreTest {
assertEquals( assertEquals(
asList(new KeyValue<>(10L, "my-later-value"), new KeyValue<>(0L, "my-value")), asList(new KeyValue<>(10L, "my-later-value"), new KeyValue<>(0L, "my-value")),
StreamsTestUtils.toList(windowStore.backwardFetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L))) StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)))
); );
} }
@ -132,9 +132,9 @@ public class CompositeReadOnlyWindowStoreTest {
secondUnderlying.put("key-two", "value-two", 10L); secondUnderlying.put("key-two", "value-two", 10L);
final List<KeyValue<Long, String>> keyOneResults = final List<KeyValue<Long, String>> keyOneResults =
StreamsTestUtils.toList(windowStore.fetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L)));
final List<KeyValue<Long, String>> keyTwoResults = final List<KeyValue<Long, String>> keyTwoResults =
StreamsTestUtils.toList(windowStore.fetch("key-two", ofEpochMilli(10L), ofEpochMilli(11L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("key-two", ofEpochMilli(10L), ofEpochMilli(11L)));
assertEquals(Collections.singletonList(KeyValue.pair(0L, "value-one")), keyOneResults); assertEquals(Collections.singletonList(KeyValue.pair(0L, "value-one")), keyOneResults);
assertEquals(Collections.singletonList(KeyValue.pair(10L, "value-two")), keyTwoResults); assertEquals(Collections.singletonList(KeyValue.pair(10L, "value-two")), keyTwoResults);
@ -150,9 +150,9 @@ public class CompositeReadOnlyWindowStoreTest {
secondUnderlying.put("key-two", "value-two", 10L); secondUnderlying.put("key-two", "value-two", 10L);
final List<KeyValue<Long, String>> keyOneResults = final List<KeyValue<Long, String>> keyOneResults =
StreamsTestUtils.toList(windowStore.backwardFetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L))); StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L)));
final List<KeyValue<Long, String>> keyTwoResults = final List<KeyValue<Long, String>> keyTwoResults =
StreamsTestUtils.toList(windowStore.backwardFetch("key-two", ofEpochMilli(10L), ofEpochMilli(11L))); StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("key-two", ofEpochMilli(10L), ofEpochMilli(11L)));
assertEquals(Collections.singletonList(KeyValue.pair(0L, "value-one")), keyOneResults); assertEquals(Collections.singletonList(KeyValue.pair(0L, "value-one")), keyOneResults);
assertEquals(Collections.singletonList(KeyValue.pair(10L, "value-two")), keyTwoResults); assertEquals(Collections.singletonList(KeyValue.pair(10L, "value-two")), keyTwoResults);
@ -164,7 +164,7 @@ public class CompositeReadOnlyWindowStoreTest {
underlyingWindowStore.put("some-key", "my-value", 1L); underlyingWindowStore.put("some-key", "my-value", 1L);
final List<KeyValue<Long, String>> results = final List<KeyValue<Long, String>> results =
StreamsTestUtils.toList(windowStore.fetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L)));
assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results); assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results);
} }
@ -174,7 +174,7 @@ public class CompositeReadOnlyWindowStoreTest {
underlyingWindowStore.put("some-key", "my-value", 1L); underlyingWindowStore.put("some-key", "my-value", 1L);
final List<KeyValue<Long, String>> results = final List<KeyValue<Long, String>> results =
StreamsTestUtils.toList(windowStore.backwardFetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L))); StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L)));
assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results); assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results);
} }
@ -348,7 +348,7 @@ public class CompositeReadOnlyWindowStoreTest {
underlyingWindowStore.put("a", "a", 0L); underlyingWindowStore.put("a", "a", 0L);
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
final List<KeyValue<Windowed<String>, String>> results = final List<KeyValue<Windowed<String>, String>> results =
StreamsTestUtils.toList(windowStore.fetch("a", "b", ofEpochMilli(0), ofEpochMilli(10))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("a", "b", ofEpochMilli(0), ofEpochMilli(10)));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), 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<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
@ -362,7 +362,7 @@ public class CompositeReadOnlyWindowStoreTest {
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
secondUnderlying.put("c", "c", 10L); secondUnderlying.put("c", "c", 10L);
final List<KeyValue<Windowed<String>, String>> results = final List<KeyValue<Windowed<String>, String>> results =
StreamsTestUtils.toList(windowStore.fetch("b", null, ofEpochMilli(0), ofEpochMilli(10))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("b", null, ofEpochMilli(0), ofEpochMilli(10)));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"),
KeyValue.pair(new Windowed<>("c", new TimeWindow(10, 10 + WINDOW_SIZE)), "c")))); KeyValue.pair(new Windowed<>("c", new TimeWindow(10, 10 + WINDOW_SIZE)), "c"))));
@ -376,7 +376,7 @@ public class CompositeReadOnlyWindowStoreTest {
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
secondUnderlying.put("c", "c", 10L); secondUnderlying.put("c", "c", 10L);
final List<KeyValue<Windowed<String>, String>> results = final List<KeyValue<Windowed<String>, String>> results =
StreamsTestUtils.toList(windowStore.fetch(null, "b", ofEpochMilli(0), ofEpochMilli(10))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch(null, "b", ofEpochMilli(0), ofEpochMilli(10)));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), 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<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
@ -390,7 +390,7 @@ public class CompositeReadOnlyWindowStoreTest {
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
secondUnderlying.put("c", "c", 10L); secondUnderlying.put("c", "c", 10L);
final List<KeyValue<Windowed<String>, String>> results = final List<KeyValue<Windowed<String>, String>> results =
StreamsTestUtils.toList(windowStore.fetch(null, null, ofEpochMilli(0), ofEpochMilli(10))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetch(null, null, ofEpochMilli(0), ofEpochMilli(10)));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), 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<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"),
@ -405,7 +405,7 @@ public class CompositeReadOnlyWindowStoreTest {
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
secondUnderlying.put("c", "c", 10L); secondUnderlying.put("c", "c", 10L);
final List<KeyValue<Windowed<String>, String>> results = final List<KeyValue<Windowed<String>, String>> results =
StreamsTestUtils.toList(windowStore.backwardFetch("b", null, ofEpochMilli(0), ofEpochMilli(10))); StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("b", null, ofEpochMilli(0), ofEpochMilli(10)));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("c", new TimeWindow(10, 10 + WINDOW_SIZE)), "c"), KeyValue.pair(new Windowed<>("c", new TimeWindow(10, 10 + WINDOW_SIZE)), "c"),
KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
@ -419,7 +419,7 @@ public class CompositeReadOnlyWindowStoreTest {
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
secondUnderlying.put("c", "c", 10L); secondUnderlying.put("c", "c", 10L);
final List<KeyValue<Windowed<String>, String>> results = final List<KeyValue<Windowed<String>, String>> results =
StreamsTestUtils.toList(windowStore.backwardFetch(null, "b", ofEpochMilli(0), ofEpochMilli(10))); StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch(null, "b", ofEpochMilli(0), ofEpochMilli(10)));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), 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<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")
@ -434,7 +434,7 @@ public class CompositeReadOnlyWindowStoreTest {
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
secondUnderlying.put("c", "c", 10L); secondUnderlying.put("c", "c", 10L);
final List<KeyValue<Windowed<String>, String>> results = final List<KeyValue<Windowed<String>, String>> results =
StreamsTestUtils.toList(windowStore.backwardFetch(null, null, ofEpochMilli(0), ofEpochMilli(10))); StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch(null, null, ofEpochMilli(0), ofEpochMilli(10)));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"),
KeyValue.pair(new Windowed<>("c", new TimeWindow(10, 10 + WINDOW_SIZE)), "c"), KeyValue.pair(new Windowed<>("c", new TimeWindow(10, 10 + WINDOW_SIZE)), "c"),
@ -448,7 +448,7 @@ public class CompositeReadOnlyWindowStoreTest {
underlyingWindowStore.put("a", "a", 0L); underlyingWindowStore.put("a", "a", 0L);
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
final List<KeyValue<Windowed<String>, String>> results = final List<KeyValue<Windowed<String>, String>> results =
StreamsTestUtils.toList(windowStore.backwardFetch("a", "b", ofEpochMilli(0), ofEpochMilli(10))); StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("a", "b", ofEpochMilli(0), ofEpochMilli(10)));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), 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<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
@ -473,7 +473,7 @@ public class CompositeReadOnlyWindowStoreTest {
stubProviderTwo.addStore(storeName, secondUnderlying); stubProviderTwo.addStore(storeName, secondUnderlying);
underlyingWindowStore.put("a", "a", 0L); underlyingWindowStore.put("a", "a", 0L);
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.all()); final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toListAndCloseIterator(windowStore.all());
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), 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<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
@ -486,7 +486,7 @@ public class CompositeReadOnlyWindowStoreTest {
stubProviderTwo.addStore(storeName, secondUnderlying); stubProviderTwo.addStore(storeName, secondUnderlying);
underlyingWindowStore.put("a", "a", 0L); underlyingWindowStore.put("a", "a", 0L);
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.backwardAll()); final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toListAndCloseIterator(windowStore.backwardAll());
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), 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<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
@ -500,7 +500,7 @@ public class CompositeReadOnlyWindowStoreTest {
underlyingWindowStore.put("a", "a", 0L); underlyingWindowStore.put("a", "a", 0L);
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
final List<KeyValue<Windowed<String>, String>> results = final List<KeyValue<Windowed<String>, String>> results =
StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(0), ofEpochMilli(10))); StreamsTestUtils.toListAndCloseIterator(windowStore.fetchAll(ofEpochMilli(0), ofEpochMilli(10)));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), 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<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
@ -514,7 +514,7 @@ public class CompositeReadOnlyWindowStoreTest {
underlyingWindowStore.put("a", "a", 0L); underlyingWindowStore.put("a", "a", 0L);
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
final List<KeyValue<Windowed<String>, String>> results = final List<KeyValue<Windowed<String>, String>> results =
StreamsTestUtils.toList(windowStore.backwardFetchAll(ofEpochMilli(0), ofEpochMilli(10))); StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetchAll(ofEpochMilli(0), ofEpochMilli(10)));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), 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<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));

View File

@ -30,7 +30,7 @@ import java.util.Collections;
import java.util.List; import java.util.List;
import static java.util.Arrays.asList; import static java.util.Arrays.asList;
import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator;
import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertFalse;
@ -91,7 +91,7 @@ public class FilteredCacheIteratorTest {
@Test @Test
public void shouldAllowEntryMatchingHasNextCondition() { public void shouldAllowEntryMatchingHasNextCondition() {
final List<KeyValue<Bytes, LRUCacheEntry>> keyValues = toList(allIterator); final List<KeyValue<Bytes, LRUCacheEntry>> keyValues = toListAndCloseIterator(allIterator);
assertThat(keyValues, equalTo(entries)); assertThat(keyValues, equalTo(entries));
} }
@ -122,7 +122,7 @@ public class FilteredCacheIteratorTest {
@Test @Test
public void shouldFilterEntriesNotMatchingHasNextCondition() { public void shouldFilterEntriesNotMatchingHasNextCondition() {
final List<KeyValue<Bytes, LRUCacheEntry>> keyValues = toList(firstEntryIterator); final List<KeyValue<Bytes, LRUCacheEntry>> keyValues = toListAndCloseIterator(firstEntryIterator);
assertThat(keyValues, equalTo(Collections.singletonList(firstEntry))); assertThat(keyValues, equalTo(Collections.singletonList(firstEntry)));
} }

View File

@ -52,7 +52,11 @@ public class InMemorySessionStoreTest extends AbstractSessionBytesStoreTest {
assertFalse(iterator.hasNext()); assertFalse(iterator.hasNext());
iterator.close(); iterator.close();
assertFalse(sessionStore.findSessions("a", "b", 0L, 20L).hasNext());
try (final KeyValueIterator<Windowed<String>, Long> it =
sessionStore.findSessions("a", "b", 0L, 20L)) {
assertFalse(it.hasNext());
}
} }
} }

View File

@ -40,7 +40,7 @@ import java.io.File;
import java.util.Collections; import java.util.Collections;
import static java.util.Arrays.asList; import static java.util.Arrays.asList;
import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
@ -103,7 +103,7 @@ public class ListValueStoreTest {
assertEquals( assertEquals(
asList(zero, zeroAgain, one, two), asList(zero, zeroAgain, one, two),
toList(listStore.all()) toListAndCloseIterator(listStore.all())
); );
} }
@ -130,7 +130,7 @@ public class ListValueStoreTest {
assertEquals( assertEquals(
asList(zero, two, four), asList(zero, two, four),
toList(listStore.all()) toListAndCloseIterator(listStore.all())
); );
} }
@ -158,7 +158,7 @@ public class ListValueStoreTest {
assertEquals( assertEquals(
asList(zero, one, two1, two2, three, four), asList(zero, one, two1, two2, three, four),
toList(listStore.all()) toListAndCloseIterator(listStore.all())
); );
} }
@ -185,7 +185,7 @@ public class ListValueStoreTest {
it.close(); it.close();
// A new all() iterator after a previous all() iterator was closed should not return deleted records. // A new all() iterator after a previous all() iterator was closed should not return deleted records.
assertEquals(Collections.singletonList(one), toList(listStore.all())); assertEquals(Collections.singletonList(one), toListAndCloseIterator(listStore.all()));
} }
@ParameterizedTest @ParameterizedTest

View File

@ -72,7 +72,7 @@ import static java.util.Arrays.asList;
import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize; import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize;
import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator;
import static org.apache.kafka.test.StreamsTestUtils.verifyAllWindowedKeyValues; import static org.apache.kafka.test.StreamsTestUtils.verifyAllWindowedKeyValues;
import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList; import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList;
import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue; import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue;
@ -945,7 +945,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest {
windowedPair("a", "0001", 1), windowedPair("a", "0001", 1),
windowedPair("aa", "0002", 0) windowedPair("aa", "0002", 0)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0),
ofEpochMilli(Long.MAX_VALUE))) ofEpochMilli(Long.MAX_VALUE)))
); );
} else { } else {
@ -954,7 +954,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest {
windowedPair("aa", "0002", 0), windowedPair("aa", "0002", 0),
windowedPair("a", "0001", 1) windowedPair("a", "0001", 1)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0),
ofEpochMilli(Long.MAX_VALUE))) ofEpochMilli(Long.MAX_VALUE)))
); );
} }
@ -976,7 +976,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest {
KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")) KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005"))
); );
final List<KeyValue<Long, byte[]>> actual = final List<KeyValue<Long, byte[]>> actual =
toList(cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)));
verifyKeyValueList(expected, actual); verifyKeyValueList(expected, actual);
} }
@ -996,7 +996,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest {
KeyValue.pair(0L, bytesValue("0001")) KeyValue.pair(0L, bytesValue("0001"))
); );
final List<KeyValue<Long, byte[]>> actual = final List<KeyValue<Long, byte[]>> actual =
toList(cachingStore.backwardFetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)));
verifyKeyValueList(expected, actual); verifyKeyValueList(expected, actual);
} }
@ -1016,14 +1016,14 @@ public class TimeOrderedCachingPersistentWindowStoreTest {
windowedPair("a", "0003", 1), windowedPair("a", "0003", 1),
windowedPair("a", "0005", SEGMENT_INTERVAL) windowedPair("a", "0005", SEGMENT_INTERVAL)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
verifyKeyValueList( verifyKeyValueList(
asList( asList(
windowedPair("aa", "0002", 0), windowedPair("aa", "0002", 0),
windowedPair("aa", "0004", 1)), windowedPair("aa", "0004", 1)),
toList(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
if (hasIndex) { if (hasIndex) {
@ -1035,7 +1035,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest {
windowedPair("aa", "0004", 1), windowedPair("aa", "0004", 1),
windowedPair("a", "0005", SEGMENT_INTERVAL) windowedPair("a", "0005", SEGMENT_INTERVAL)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0),
ofEpochMilli(Long.MAX_VALUE))) ofEpochMilli(Long.MAX_VALUE)))
); );
} else { } else {
@ -1047,7 +1047,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest {
windowedPair("aa", "0004", 1), windowedPair("aa", "0004", 1),
windowedPair("a", "0005", SEGMENT_INTERVAL) windowedPair("a", "0005", SEGMENT_INTERVAL)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0),
ofEpochMilli(Long.MAX_VALUE))) ofEpochMilli(Long.MAX_VALUE)))
); );
} }
@ -1069,14 +1069,14 @@ public class TimeOrderedCachingPersistentWindowStoreTest {
windowedPair("a", "0003", 1), windowedPair("a", "0003", 1),
windowedPair("a", "0001", 0) windowedPair("a", "0001", 0)
), ),
toList(cachingStore.backwardFetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
verifyKeyValueList( verifyKeyValueList(
asList( asList(
windowedPair("aa", "0004", 1), windowedPair("aa", "0004", 1),
windowedPair("aa", "0002", 0)), windowedPair("aa", "0002", 0)),
toList(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
if (!hasIndex) { if (!hasIndex) {
@ -1089,7 +1089,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest {
windowedPair("aa", "0002", 0), windowedPair("aa", "0002", 0),
windowedPair("a", "0001", 0) windowedPair("a", "0001", 0)
), ),
toList(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0),
ofEpochMilli(Long.MAX_VALUE))) ofEpochMilli(Long.MAX_VALUE)))
); );
} else { } else {
@ -1102,7 +1102,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest {
windowedPair("a", "0003", 1), windowedPair("a", "0003", 1),
windowedPair("a", "0001", 0) windowedPair("a", "0001", 0)
), ),
toList(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0),
ofEpochMilli(Long.MAX_VALUE))) ofEpochMilli(Long.MAX_VALUE)))
); );
} }

View File

@ -71,7 +71,7 @@ import static java.util.Arrays.asList;
import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize; import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize;
import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator;
import static org.apache.kafka.test.StreamsTestUtils.verifyAllWindowedKeyValues; import static org.apache.kafka.test.StreamsTestUtils.verifyAllWindowedKeyValues;
import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList; import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList;
import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue; import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue;
@ -951,7 +951,7 @@ public class TimeOrderedWindowStoreTest {
windowedPair("a", "0001", 1), windowedPair("a", "0001", 1),
windowedPair("aa", "0002", 0) windowedPair("aa", "0002", 0)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0),
ofEpochMilli(Long.MAX_VALUE))) ofEpochMilli(Long.MAX_VALUE)))
); );
} else { } else {
@ -960,7 +960,7 @@ public class TimeOrderedWindowStoreTest {
windowedPair("aa", "0002", 0), windowedPair("aa", "0002", 0),
windowedPair("a", "0001", 1) windowedPair("a", "0001", 1)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0),
ofEpochMilli(Long.MAX_VALUE))) ofEpochMilli(Long.MAX_VALUE)))
); );
} }
@ -982,7 +982,7 @@ public class TimeOrderedWindowStoreTest {
KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")) KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005"))
); );
final List<KeyValue<Long, byte[]>> actual = final List<KeyValue<Long, byte[]>> actual =
toList(cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)));
verifyKeyValueList(expected, actual); verifyKeyValueList(expected, actual);
} }
@ -1002,7 +1002,7 @@ public class TimeOrderedWindowStoreTest {
KeyValue.pair(0L, bytesValue("0001")) KeyValue.pair(0L, bytesValue("0001"))
); );
final List<KeyValue<Long, byte[]>> actual = final List<KeyValue<Long, byte[]>> actual =
toList(cachingStore.backwardFetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)));
verifyKeyValueList(expected, actual); verifyKeyValueList(expected, actual);
} }
@ -1022,14 +1022,14 @@ public class TimeOrderedWindowStoreTest {
windowedPair("a", "0003", 1), windowedPair("a", "0003", 1),
windowedPair("a", "0005", SEGMENT_INTERVAL) windowedPair("a", "0005", SEGMENT_INTERVAL)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
verifyKeyValueList( verifyKeyValueList(
asList( asList(
windowedPair("aa", "0002", 0), windowedPair("aa", "0002", 0),
windowedPair("aa", "0004", 1)), windowedPair("aa", "0004", 1)),
toList(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
if (hasIndex) { if (hasIndex) {
@ -1041,7 +1041,7 @@ public class TimeOrderedWindowStoreTest {
windowedPair("aa", "0004", 1), windowedPair("aa", "0004", 1),
windowedPair("a", "0005", SEGMENT_INTERVAL) windowedPair("a", "0005", SEGMENT_INTERVAL)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0),
ofEpochMilli(Long.MAX_VALUE))) ofEpochMilli(Long.MAX_VALUE)))
); );
} else { } else {
@ -1053,7 +1053,7 @@ public class TimeOrderedWindowStoreTest {
windowedPair("aa", "0004", 1), windowedPair("aa", "0004", 1),
windowedPair("a", "0005", SEGMENT_INTERVAL) windowedPair("a", "0005", SEGMENT_INTERVAL)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0),
ofEpochMilli(Long.MAX_VALUE))) ofEpochMilli(Long.MAX_VALUE)))
); );
} }
@ -1075,14 +1075,14 @@ public class TimeOrderedWindowStoreTest {
windowedPair("a", "0003", 1), windowedPair("a", "0003", 1),
windowedPair("a", "0001", 0) windowedPair("a", "0001", 0)
), ),
toList(cachingStore.backwardFetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
verifyKeyValueList( verifyKeyValueList(
asList( asList(
windowedPair("aa", "0004", 1), windowedPair("aa", "0004", 1),
windowedPair("aa", "0002", 0)), windowedPair("aa", "0002", 0)),
toList(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
if (!hasIndex) { if (!hasIndex) {
@ -1095,7 +1095,7 @@ public class TimeOrderedWindowStoreTest {
windowedPair("aa", "0002", 0), windowedPair("aa", "0002", 0),
windowedPair("a", "0001", 0) windowedPair("a", "0001", 0)
), ),
toList(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0),
ofEpochMilli(Long.MAX_VALUE))) ofEpochMilli(Long.MAX_VALUE)))
); );
} else { } else {
@ -1108,7 +1108,7 @@ public class TimeOrderedWindowStoreTest {
windowedPair("a", "0003", 1), windowedPair("a", "0003", 1),
windowedPair("a", "0001", 0) windowedPair("a", "0001", 0)
), ),
toList(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0),
ofEpochMilli(Long.MAX_VALUE))) ofEpochMilli(Long.MAX_VALUE)))
); );
} }

View File

@ -126,7 +126,7 @@ public class WindowKeySchemaTest {
private final Window window = new TimeWindow(startTime, endTime); private final Window window = new TimeWindow(startTime, endTime);
private final Windowed<String> windowedKey = new Windowed<>(key, window); private final Windowed<String> windowedKey = new Windowed<>(key, window);
private KeySchema keySchema; private KeySchema keySchema;
private final Serde<Windowed<String>> keySerde = new WindowedSerdes.TimeWindowedSerde<>(serde, Long.MAX_VALUE); private final Serde<Windowed<String>> keySerde = new WindowedSerdes.TimeWindowedSerde<>(serde, endTime - startTime);
private final StateSerdes<String, byte[]> stateSerdes = new StateSerdes<>("dummy", serde, Serdes.ByteArray()); private final StateSerdes<String, byte[]> stateSerdes = new StateSerdes<>("dummy", serde, Serdes.ByteArray());
public SchemaType schemaType; public SchemaType schemaType;
@ -401,7 +401,7 @@ public class WindowKeySchemaTest {
final byte[] bytes = keySerde.serializer().serialize(topic, windowedKey); final byte[] bytes = keySerde.serializer().serialize(topic, windowedKey);
final Windowed<String> result = keySerde.deserializer().deserialize(topic, bytes); final Windowed<String> result = keySerde.deserializer().deserialize(topic, bytes);
// TODO: fix this part as last bits of KAFKA-4468 // TODO: fix this part as last bits of KAFKA-4468
assertEquals(new Windowed<>(key, new TimeWindow(startTime, Long.MAX_VALUE)), result); assertEquals(new Windowed<>(key, new TimeWindow(startTime, endTime)), result);
} }
@EnumSource(SchemaType.class) @EnumSource(SchemaType.class)

View File

@ -35,8 +35,6 @@ import org.apache.kafka.streams.state.KeyValueIterator;
import org.mockito.quality.Strictness; import org.mockito.quality.Strictness;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
@ -116,21 +114,17 @@ public final class StreamsTestUtils {
return getStreamsConfig(UUID.randomUUID().toString()); return getStreamsConfig(UUID.randomUUID().toString());
} }
public static <K, V> List<KeyValue<K, V>> toList(final Iterator<KeyValue<K, V>> iterator) { public static <K, V> List<KeyValue<K, V>> toListAndCloseIterator(final KeyValueIterator<K, V> iterator) {
try (iterator) {
final List<KeyValue<K, V>> results = new ArrayList<>(); final List<KeyValue<K, V>> results = new ArrayList<>();
while (iterator.hasNext()) { while (iterator.hasNext()) {
results.add(iterator.next()); results.add(iterator.next());
} }
if (iterator instanceof Closeable) {
try {
((Closeable) iterator).close();
} catch (IOException e) { /* do nothing */ }
}
return results; return results;
} }
}
public static <K, V> Set<KeyValue<K, V>> toSet(final Iterator<KeyValue<K, V>> iterator) { public static <K, V> Set<KeyValue<K, V>> toSet(final Iterator<KeyValue<K, V>> iterator) {
final Set<KeyValue<K, V>> results = new LinkedHashSet<>(); final Set<KeyValue<K, V>> results = new LinkedHashSet<>();
@ -141,7 +135,7 @@ public final class StreamsTestUtils {
return results; return results;
} }
public static <K, V> Set<V> valuesToSet(final Iterator<KeyValue<K, V>> iterator) { public static <K, V> Set<V> valuesToSet(final KeyValueIterator<K, V> iterator) {
final Set<V> results = new HashSet<>(); final Set<V> results = new HashSet<>();
while (iterator.hasNext()) { while (iterator.hasNext()) {
@ -150,6 +144,12 @@ public final class StreamsTestUtils {
return results; return results;
} }
public static <K, V> Set<V> valuesToSetAndCloseIterator(final KeyValueIterator<K, V> iterator) {
try (iterator) {
return valuesToSet(iterator);
}
}
public static <K> void verifyKeyValueList(final List<KeyValue<K, byte[]>> expected, final List<KeyValue<K, byte[]>> actual) { public static <K> void verifyKeyValueList(final List<KeyValue<K, byte[]>> expected, final List<KeyValue<K, byte[]>> actual) {
assertThat(actual.size(), equalTo(expected.size())); assertThat(actual.size(), equalTo(expected.size()));
for (int i = 0; i < actual.size(); i++) { for (int i = 0; i < actual.size(); i++) {
@ -254,8 +254,8 @@ public final class StreamsTestUtils {
} }
/** /**
* Used to keep tests simple, and ignore calls from {@link org.apache.kafka.streams.internals.ApiUtils#checkSupplier(Supplier)} )}. * Used to keep tests simple, and ignore calls from {@link org.apache.kafka.streams.internals.ApiUtils#checkSupplier(Supplier)}.
* @return true if the stack context is within a {@link org.apache.kafka.streams.internals.ApiUtils#checkSupplier(Supplier)} )} call * @return true if the stack context is within a {@link org.apache.kafka.streams.internals.ApiUtils#checkSupplier(Supplier)} call
*/ */
public static boolean isCheckSupplierCall() { public static boolean isCheckSupplierCall() {
return Arrays.stream(Thread.currentThread().getStackTrace()) return Arrays.stream(Thread.currentThread().getStackTrace())