diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java index 17a6e38683b..6dbb9b8d1db 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java @@ -227,7 +227,7 @@ public class SessionWindowedKStreamImplTest { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { processData(driver); final SessionStore store = driver.getSessionStore("count-store"); - final List, Long>> data = StreamsTestUtils.toList(store.fetch("1", "2")); + final List, Long>> data = StreamsTestUtils.toListAndCloseIterator(store.fetch("1", "2")); if (!emitFinal) { assertThat( data, @@ -255,7 +255,7 @@ public class SessionWindowedKStreamImplTest { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { processData(driver); final SessionStore sessionStore = driver.getSessionStore("reduced"); - final List, String>> data = StreamsTestUtils.toList(sessionStore.fetch("1", "2")); + final List, String>> data = StreamsTestUtils.toListAndCloseIterator(sessionStore.fetch("1", "2")); if (!emitFinal) { assertThat( @@ -288,7 +288,7 @@ public class SessionWindowedKStreamImplTest { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { processData(driver); final SessionStore sessionStore = driver.getSessionStore("aggregated"); - final List, String>> data = StreamsTestUtils.toList(sessionStore.fetch("1", "2")); + final List, String>> data = StreamsTestUtils.toListAndCloseIterator(sessionStore.fetch("1", "2")); if (!emitFinal) { assertThat( data, diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java index 2928c04b1c5..e9c52c33831 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java @@ -208,7 +208,7 @@ public class SlidingWindowedKStreamImplTest { { final WindowStore windowStore = driver.getWindowStore("count-store"); final List, 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( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), 1L), @@ -223,7 +223,7 @@ public class SlidingWindowedKStreamImplTest { final WindowStore> windowStore = driver.getTimestampedWindowStore("count-store"); final List, ValueAndTimestamp>> 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( 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)), @@ -248,7 +248,7 @@ public class SlidingWindowedKStreamImplTest { { final WindowStore windowStore = driver.getWindowStore("reduced"); final List, 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( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), "1"), KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), "1+2"), @@ -262,7 +262,7 @@ public class SlidingWindowedKStreamImplTest { final WindowStore> windowStore = driver.getTimestampedWindowStore("reduced"); final List, ValueAndTimestamp>> 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( 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)), @@ -289,7 +289,7 @@ public class SlidingWindowedKStreamImplTest { { final WindowStore windowStore = driver.getWindowStore("aggregated"); final List, 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( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), "0+1"), KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), "0+1+2"), @@ -303,7 +303,7 @@ public class SlidingWindowedKStreamImplTest { final WindowStore> windowStore = driver.getTimestampedWindowStore("aggregated"); final List, ValueAndTimestamp>> 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( 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)), @@ -410,7 +410,7 @@ public class SlidingWindowedKStreamImplTest { { final WindowStore windowStore = driver.getWindowStore("aggregated"); final List, 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( KeyValue.pair(new Windowed<>("1", new TimeWindow(900, 1000)), "0+4"), KeyValue.pair(new Windowed<>("1", new TimeWindow(1900, 2000)), "0+5")))); @@ -419,7 +419,7 @@ public class SlidingWindowedKStreamImplTest { final WindowStore> windowStore = driver.getTimestampedWindowStore("aggregated"); final List, ValueAndTimestamp>> 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( 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))))); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java index 3758ad7b9bc..c526b09f9c6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java @@ -239,7 +239,7 @@ public class TimeWindowedKStreamImplTest { { final WindowStore windowStore = driver.getWindowStore("count-store"); final List, Long>> data = - StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); if (withCache) { // with cache returns all records (expired from underneath as well) as part of @@ -266,7 +266,7 @@ public class TimeWindowedKStreamImplTest { final WindowStore> windowStore = driver.getTimestampedWindowStore("count-store"); final List, ValueAndTimestamp>> 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. if (withCache) { @@ -305,7 +305,7 @@ public class TimeWindowedKStreamImplTest { { final WindowStore windowStore = driver.getWindowStore("reduced"); final List, String>> data = - StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); if (withCache) { // with cache returns all records (expired from underneath as well) as part of @@ -325,7 +325,7 @@ public class TimeWindowedKStreamImplTest { { final WindowStore> windowStore = driver.getTimestampedWindowStore("reduced"); final List, ValueAndTimestamp>> 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. if (withCache) { @@ -358,7 +358,7 @@ public class TimeWindowedKStreamImplTest { { final WindowStore windowStore = driver.getWindowStore("aggregated"); final List, String>> data = - StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); if (withCache) { // with cache returns all records (expired from underneath as well) as part of @@ -379,7 +379,7 @@ public class TimeWindowedKStreamImplTest { { final WindowStore> windowStore = driver.getTimestampedWindowStore("aggregated"); final List, ValueAndTimestamp>> data = - StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); if (withCache) { assertThat(data, equalTo(asList( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), ValueAndTimestamp.make("0+1+2", 15L)), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStoreTest.java index f7578112da8..0f8ce890b19 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStoreTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.RecordBatch; 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.utils.Bytes; import org.apache.kafka.common.utils.LogContext; @@ -37,11 +38,9 @@ import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.SessionWindow; import org.apache.kafka.streams.kstream.internals.TimeWindow; -import org.apache.kafka.streams.processor.StateStoreContext; import org.apache.kafka.streams.processor.internals.ChangelogRecordDeserializationHelper; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; -import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.state.KeyValueIterator; @@ -96,8 +95,8 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { private static final String METRICS_SCOPE = "metrics-scope"; - private final long windowSizeForTimeWindow = 500; - private InternalMockProcessorContext context; + private long windowSizeForTimeWindow = 500; + private InternalMockProcessorContext context; private AbstractDualSchemaRocksDBSegmentedBytesStore bytesStore; private File stateDir; private final Window[] windows = new Window[4]; @@ -159,7 +158,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { new MockRecordCollector(), new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())) ); - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); } @AfterEach @@ -204,7 +203,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { default: throw new IllegalStateException("Unknown SchemaType: " + schemaType()); } - }; + } AbstractSegments newSegments() { return new KeyValueSegments(storeName, METRICS_SCOPE, retention, segmentInterval); @@ -244,54 +243,43 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { bytesStore.put(serializeKey(new Windowed<>(keyB, windows[2])), serializeValue(100)); bytesStore.put(serializeKey(new Windowed<>(keyC, windows[3])), serializeValue(200)); - try (final KeyValueIterator values = bytesStore.fetch( - Bytes.wrap(keyA.getBytes()), 0, windows[2].start())) { - // For all tests, actualFrom is computed using observedStreamTime - retention + 1. - // so actualFrom = 60000(observedStreamTime) - 1000(retention) + 1 = 59001 - // all records expired as actual from is 59001 and to is 1000 - final List, Long>> expected = Collections.emptyList(); + // For all tests, actualFrom is computed using observedStreamTime - retention + 1. + // so actualFrom = 60000(observedStreamTime) - 1000(retention) + 1 = 59001 + // all records expired as actual from is 59001 and to is 1000 + assertEquals( + Collections.emptyList(), + toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(keyA.getBytes()), 0, windows[2].start())) + ); - assertEquals(expected, toList(values)); - } + // all records expired as actual from is 59001 and to is 1000 + assertEquals( + Collections.emptyList(), + toListAndCloseIterator(bytesStore.fetch( + Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), 0, windows[2].start()) + ) + ); - try (final KeyValueIterator values = bytesStore.fetch( - Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), 0, windows[2].start())) { + // all records expired as actual from is 59001 and to is 1000 + assertEquals( + Collections.emptyList(), + toListAndCloseIterator( + bytesStore.fetch(null, Bytes.wrap(keyB.getBytes()), 0, windows[2].start()) + ) + ); - // all records expired as actual from is 59001 and to is 1000 - final List, Long>> expected = Collections.emptyList(); + // key B is expired as actual from is 59001 + assertEquals( + Collections.singletonList(KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L)), + toListAndCloseIterator( + bytesStore.fetch(Bytes.wrap(keyB.getBytes()), null, 0, windows[3].start()) + ) + ); - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.fetch( - null, Bytes.wrap(keyB.getBytes()), 0, windows[2].start())) { - - // all records expired as actual from is 59001 and to is 1000 - final List, Long>> expected = Collections.emptyList(); - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.fetch( - Bytes.wrap(keyB.getBytes()), null, 0, windows[3].start())) { - - // key B is expired as actual from is 59001 - final List, Long>> expected = Collections.singletonList( - KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L) - ); - - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.fetch( - null, null, 0, windows[3].start())) { - - // keys A and B expired as actual from is 59001 - final List, Long>> expected = Collections.singletonList( - KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L) - ); - - assertEquals(expected, toList(values)); - } + // keys A and B expired as actual from is 59001 + assertEquals( + Collections.singletonList(KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L)), + toListAndCloseIterator(bytesStore.fetch(null, null, 0, windows[3].start())) + ); } @Test @@ -305,55 +293,46 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { bytesStore.put(serializeKey(new Windowed<>(keyB, windows[2])), serializeValue(100)); bytesStore.put(serializeKey(new Windowed<>(keyC, windows[3])), serializeValue(200)); - try (final KeyValueIterator values = bytesStore.backwardFetch( - Bytes.wrap(keyA.getBytes()), 0, windows[2].start())) { + // For all tests, actualFrom is computed using observedStreamTime - retention + 1. + // so actualFrom = 60000(observedStreamTime) - 1000(retention) + 1 = 59001 + // all records expired as actual from is 59001 and to = 1000 + assertEquals( + Collections.emptyList(), + toListAndCloseIterator(bytesStore.backwardFetch(Bytes.wrap(keyA.getBytes()), 0, windows[2].start())) + ); - // For all tests, actualFrom is computed using observedStreamTime - retention + 1. - // so actualFrom = 60000(observedStreamTime) - 1000(retention) + 1 = 59001 - // all records expired as actual from is 59001 and to = 1000 - final List, Long>> expected = Collections.emptyList(); + // all records expired as actual from is 59001 and to = 1000 + assertEquals( + Collections.emptyList(), + toListAndCloseIterator(bytesStore.backwardFetch( + Bytes.wrap(keyA.getBytes()), + Bytes.wrap(keyB.getBytes()), + 0, + windows[2].start() + )) + ); - assertEquals(expected, toList(values)); - } + // all records expired as actual from is 59001 and to = 1000 + assertEquals( + Collections.emptyList(), + toListAndCloseIterator( + bytesStore.backwardFetch(null, Bytes.wrap(keyB.getBytes()), 0, windows[2].start()) + ) + ); - try (final KeyValueIterator values = bytesStore.backwardFetch( - Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), 0, windows[2].start())) { + // only 1 record left as actual from is 59001 and to = 60,000 + assertEquals( + Collections.singletonList(KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L)), + toListAndCloseIterator( + bytesStore.backwardFetch(Bytes.wrap(keyB.getBytes()), null, 0, windows[3].start()) + ) + ); - // all records expired as actual from is 59001 and to = 1000 - final List, Long>> expected = Collections.emptyList(); - - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.backwardFetch( - null, Bytes.wrap(keyB.getBytes()), 0, windows[2].start())) { - - // all records expired as actual from is 59001 and to = 1000 - final List, Long>> expected = Collections.emptyList(); - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.backwardFetch( - Bytes.wrap(keyB.getBytes()), null, 0, windows[3].start())) { - - // only 1 record left as actual from is 59001 and to = 60,000 - final List, Long>> expected = Collections.singletonList( - KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L) - ); - - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.backwardFetch( - null, null, 0, windows[3].start())) { - - // only 1 record left as actual from is 59001 and to = 60,000 - final List, Long>> expected = Collections.singletonList( - KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L) - ); - - assertEquals(expected, toList(values)); - } + // only 1 record left as actual from is 59001 and to = 60,000 + assertEquals( + Collections.singletonList(KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L)), + toListAndCloseIterator(bytesStore.backwardFetch(null, null, 0, windows[3].start())) + ); } @Test @@ -376,52 +355,41 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { bytesStore.put(serializedKeyBEnd, serializeValue(150)); // Can fetch start/end edge for single key - try (final KeyValueIterator values = bytesStore.fetch( - Bytes.wrap(keyA.getBytes()), startEdgeTime, endEdgeTime)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L), KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(bytesStore.fetch( + Bytes.wrap(keyA.getBytes()), startEdgeTime, endEdgeTime)) + ); // Can fetch start/end edge for single key - try (final KeyValueIterator values = bytesStore.fetch( - Bytes.wrap(keyB.getBytes()), startEdgeTime, endEdgeTime)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyB, startEdgeWindow), 100L), KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(keyB.getBytes()), startEdgeTime, endEdgeTime)) + ); // Can fetch from 0 to max for single key - try (final KeyValueIterator values = bytesStore.fetch( - Bytes.wrap(keyA.getBytes()), 0, Long.MAX_VALUE)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L), KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(keyA.getBytes()), 0, Long.MAX_VALUE)) + ); // Can fetch from 0 to max for single key - try (final KeyValueIterator values = bytesStore.fetch( - Bytes.wrap(keyB.getBytes()), 0, Long.MAX_VALUE)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyB, startEdgeWindow), 100L), KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(keyB.getBytes()), 0, Long.MAX_VALUE)) + ); } @Test @@ -443,9 +411,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { bytesStore.put(serializedKeyBStart, serializeValue(100)); bytesStore.put(serializedKeyBEnd, serializeValue(150)); // Can fetch from start/end for key range - try (final KeyValueIterator values = bytesStore.fetch( - Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), startEdgeTime, endEdgeTime)) { - + { final List, Long>> expected = getIndexSchema() == null ? asList( KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L), KeyValue.pair(new Windowed<>(keyB, startEdgeWindow), 100L), @@ -457,13 +423,20 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { KeyValue.pair(new Windowed<>(keyB, startEdgeWindow), 100L), KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L) ); - assertEquals(expected, toList(values)); + + assertEquals( + expected, + toListAndCloseIterator(bytesStore.fetch( + Bytes.wrap(keyA.getBytes()), + Bytes.wrap(keyB.getBytes()), + startEdgeTime, + endEdgeTime + )) + ); } // Can fetch from 0 to max for key range - try (final KeyValueIterator values = bytesStore.fetch( - Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), 0L, Long.MAX_VALUE)) { - + { final List, Long>> expected = getIndexSchema() == null ? asList( KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L), KeyValue.pair(new Windowed<>(keyB, startEdgeWindow), 100L), @@ -475,50 +448,34 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { KeyValue.pair(new Windowed<>(keyB, startEdgeWindow), 100L), KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L) ); - assertEquals(expected, toList(values)); + + assertEquals( + expected, + toListAndCloseIterator(bytesStore.fetch( + Bytes.wrap(keyA.getBytes()), + Bytes.wrap(keyB.getBytes()), + 0L, + Long.MAX_VALUE + )) + ); } // KeyB should be ignored and KeyA should be included even in storage - try (final KeyValueIterator values = bytesStore.fetch( - null, Bytes.wrap(keyA.getBytes()), startEdgeTime, endEdgeTime - 1L)) { + assertEquals( + Collections.singletonList(KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L)), + toListAndCloseIterator( + bytesStore.fetch(null, Bytes.wrap(keyA.getBytes()), startEdgeTime, endEdgeTime - 1L) + ) + ); - final List, Long>> expected = Collections.singletonList( - KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L) - ); - - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.fetch( - Bytes.wrap(keyB.getBytes()), null, startEdgeTime + 1, endEdgeTime)) { - - final List, Long>> expected = Collections.singletonList( - KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L) - ); - - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.fetch( - null, null, 0, Long.MAX_VALUE)) { - - final List, Long>> expected = getIndexSchema() == null ? asList( - KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L), - KeyValue.pair(new Windowed<>(keyB, startEdgeWindow), 100L), - KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), - KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L) - ) : asList( - KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L), - KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), - KeyValue.pair(new Windowed<>(keyB, startEdgeWindow), 100L), - KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L) - ); - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.fetch( - null, null, startEdgeTime, endEdgeTime)) { + assertEquals( + Collections.singletonList(KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L)), + toListAndCloseIterator( + bytesStore.fetch(Bytes.wrap(keyB.getBytes()), null, startEdgeTime + 1, endEdgeTime) + ) + ); + { final List, Long>> expected = getIndexSchema() == null ? asList( KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L), KeyValue.pair(new Windowed<>(keyB, startEdgeWindow), 100L), @@ -531,7 +488,29 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L) ); - assertEquals(expected, toList(values)); + assertEquals( + expected, + toListAndCloseIterator(bytesStore.fetch(null, null, 0, Long.MAX_VALUE)) + ); + } + + { + final List, Long>> expected = getIndexSchema() == null ? asList( + KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L), + KeyValue.pair(new Windowed<>(keyB, startEdgeWindow), 100L), + KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), + KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L) + ) : asList( + KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L), + KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), + KeyValue.pair(new Windowed<>(keyB, startEdgeWindow), 100L), + KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L) + ); + + assertEquals( + expected, + toListAndCloseIterator(bytesStore.fetch(null, null, startEdgeTime, endEdgeTime)) + ); } } @@ -555,52 +534,40 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { bytesStore.put(serializedKeyBEnd, serializeValue(150)); // Can fetch start/end edge for single key - try (final KeyValueIterator values = bytesStore.backwardFetch( - Bytes.wrap(keyA.getBytes()), startEdgeTime, endEdgeTime)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(bytesStore.backwardFetch(Bytes.wrap(keyA.getBytes()), startEdgeTime, endEdgeTime)) + ); // Can fetch start/end edge for single key - try (final KeyValueIterator values = bytesStore.backwardFetch( - Bytes.wrap(keyB.getBytes()), startEdgeTime, endEdgeTime)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L), KeyValue.pair(new Windowed<>(keyB, startEdgeWindow), 100L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(bytesStore.backwardFetch(Bytes.wrap(keyB.getBytes()), startEdgeTime, endEdgeTime)) + ); // Can fetch from 0 to max for single key - try (final KeyValueIterator values = bytesStore.backwardFetch( - Bytes.wrap(keyA.getBytes()), 0, Long.MAX_VALUE)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(bytesStore.backwardFetch(Bytes.wrap(keyA.getBytes()), 0, Long.MAX_VALUE)) + ); // Can fetch from 0 to max for single key - try (final KeyValueIterator values = bytesStore.backwardFetch( - Bytes.wrap(keyB.getBytes()), 0, Long.MAX_VALUE)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L), KeyValue.pair(new Windowed<>(keyB, startEdgeWindow), 100L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(bytesStore.backwardFetch(Bytes.wrap(keyB.getBytes()), 0, Long.MAX_VALUE)) + ); } @Test @@ -623,9 +590,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { bytesStore.put(serializedKeyBEnd, serializeValue(150)); // Can fetch from start/end for key range - try (final KeyValueIterator values = bytesStore.backwardFetch( - Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), startEdgeTime, endEdgeTime)) { - + { final List, Long>> expected = getIndexSchema() == null ? asList( KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L), KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), @@ -637,13 +602,20 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L) ); - assertEquals(expected, toList(values)); + + assertEquals( + expected, + toListAndCloseIterator(bytesStore.backwardFetch( + Bytes.wrap(keyA.getBytes()), + Bytes.wrap(keyB.getBytes()), + startEdgeTime, + endEdgeTime + )) + ); } // Can fetch from 0 to max for key range - try (final KeyValueIterator values = bytesStore.backwardFetch( - Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), 0L, Long.MAX_VALUE)) { - + { final List, Long>> expected = getIndexSchema() == null ? asList( KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L), KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), @@ -655,33 +627,32 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L) ); - assertEquals(expected, toList(values)); + + assertEquals( + expected, + toListAndCloseIterator(bytesStore.backwardFetch( + Bytes.wrap(keyA.getBytes()), + Bytes.wrap(keyB.getBytes()), + 0L, + Long.MAX_VALUE + )) + ); } // KeyB should be ignored and KeyA should be included even in storage - try (final KeyValueIterator values = bytesStore.backwardFetch( - null, Bytes.wrap(keyA.getBytes()), startEdgeTime, endEdgeTime - 1L)) { + assertEquals( + Collections.singletonList(KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L)), + toListAndCloseIterator( + bytesStore.backwardFetch(null, Bytes.wrap(keyA.getBytes()), startEdgeTime, endEdgeTime - 1L)) + ); - final List, Long>> expected = Collections.singletonList( - KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L) - ); - - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.backwardFetch( - Bytes.wrap(keyB.getBytes()), null, startEdgeTime + 1, endEdgeTime)) { - - final List, Long>> expected = Collections.singletonList( - KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L) - ); - - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.backwardFetch( - null, null, 0, Long.MAX_VALUE)) { + assertEquals( + Collections.singletonList(KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L)), + toListAndCloseIterator( + bytesStore.backwardFetch(Bytes.wrap(keyB.getBytes()), null, startEdgeTime + 1, endEdgeTime)) + ); + { final List, Long>> expected = getIndexSchema() == null ? asList( KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L), KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), @@ -693,12 +664,14 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L) ); - assertEquals(expected, toList(values)); + + assertEquals( + expected, + toListAndCloseIterator(bytesStore.backwardFetch(null, null, 0, Long.MAX_VALUE)) + ); } - try (final KeyValueIterator values = bytesStore.backwardFetch( - null, null, startEdgeTime, endEdgeTime)) { - + { final List, Long>> expected = getIndexSchema() == null ? asList( KeyValue.pair(new Windowed<>(keyB, endEdgeWindow), 150L), KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), @@ -710,7 +683,11 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { KeyValue.pair(new Windowed<>(keyA, endEdgeWindow), 50L), KeyValue.pair(new Windowed<>(keyA, startEdgeWindow), 10L) ); - assertEquals(expected, toList(values)); + + assertEquals( + expected, + toListAndCloseIterator(bytesStore.backwardFetch(null, null, startEdgeTime, endEdgeTime)) + ); } } @@ -724,6 +701,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { final String keyB = "aa"; final String keyC = "aaa"; + windowSizeForTimeWindow = 1L; final Window maxWindow = new TimeWindow(Long.MAX_VALUE - 1, Long.MAX_VALUE); final Bytes serializedKeyA = serializeKey(new Windowed<>(keyA, maxWindow), false, Integer.MAX_VALUE); final Bytes serializedKeyB = serializeKey(new Windowed<>(keyB, maxWindow), false, Integer.MAX_VALUE); @@ -737,63 +715,49 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { bytesStore.put(serializedKeyB, serializeValue(50)); bytesStore.put(serializedKeyC, serializeValue(100)); - try (final KeyValueIterator values = bytesStore.fetch( - Bytes.wrap(keyA.getBytes()), 0, Long.MAX_VALUE)) { + assertEquals( + Collections.singletonList(KeyValue.pair(new Windowed<>(keyA, maxWindow), 10L)), + toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(keyA.getBytes()), 0, Long.MAX_VALUE)) + ); - final List, Long>> expected = Collections.singletonList( - KeyValue.pair(new Windowed<>(keyA, maxWindow), 10L) - ); - - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.fetch( - Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), 0, Long.MAX_VALUE)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyB, maxWindow), 50L), KeyValue.pair(new Windowed<>(keyA, maxWindow), 10L) - ); + ), + toListAndCloseIterator( + bytesStore.fetch(Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), 0, Long.MAX_VALUE) + ) + ); - assertEquals(expected, toList(values)); - } - - // KeyC should be ignored and KeyA should be included even in storage, KeyC is before KeyB - // and KeyA is after KeyB - try (final KeyValueIterator values = bytesStore.fetch( - null, Bytes.wrap(keyB.getBytes()), 0, Long.MAX_VALUE)) { - - final List, Long>> expected = asList( + // KeyC should be ignored and KeyA should be included even in storage, KeyC is before KeyB and KeyA is after KeyB + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyB, maxWindow), 50L), KeyValue.pair(new Windowed<>(keyA, maxWindow), 10L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(bytesStore.fetch(null, Bytes.wrap(keyB.getBytes()), 0, Long.MAX_VALUE)) + ); // KeyC should be included even in storage KeyC is before KeyB - try (final KeyValueIterator values = bytesStore.fetch( - Bytes.wrap(keyB.getBytes()), null, 0, Long.MAX_VALUE)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyC, maxWindow), 100L), KeyValue.pair(new Windowed<>(keyB, maxWindow), 50L) - ); + ), + toListAndCloseIterator( + bytesStore.fetch(Bytes.wrap(keyB.getBytes()), null, 0, Long.MAX_VALUE) + ) + ); - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.fetch( - null, null, 0, Long.MAX_VALUE)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyC, maxWindow), 100L), KeyValue.pair(new Windowed<>(keyB, maxWindow), 50L), KeyValue.pair(new Windowed<>(keyA, maxWindow), 10L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(bytesStore.fetch(null, null, 0, Long.MAX_VALUE)) + ); } @Test @@ -807,6 +771,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { final String keyB = "aa"; final String keyC = "aaa"; + windowSizeForTimeWindow = 1L; final Window maxWindow = new TimeWindow(Long.MAX_VALUE - 1, Long.MAX_VALUE); final Bytes serializedKeyA = serializeKey(new Windowed<>(keyA, maxWindow), false, Integer.MAX_VALUE); final Bytes serializedKeyB = serializeKey(new Windowed<>(keyB, maxWindow), false, Integer.MAX_VALUE); @@ -820,62 +785,48 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { bytesStore.put(serializedKeyB, serializeValue(50)); bytesStore.put(serializedKeyC, serializeValue(100)); - try (final KeyValueIterator values = bytesStore.backwardFetch( - Bytes.wrap(keyA.getBytes()), 0, Long.MAX_VALUE)) { + assertEquals( + Collections.singletonList(KeyValue.pair(new Windowed<>(keyA, maxWindow), 10L)), + toListAndCloseIterator(bytesStore.backwardFetch(Bytes.wrap(keyA.getBytes()), 0, Long.MAX_VALUE)) + ); - final List, Long>> expected = Collections.singletonList( - KeyValue.pair(new Windowed<>(keyA, maxWindow), 10L) - ); - - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.backwardFetch( - Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), 0, Long.MAX_VALUE)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyA, maxWindow), 10L), KeyValue.pair(new Windowed<>(keyB, maxWindow), 50L) - ); + ), + toListAndCloseIterator( + bytesStore.backwardFetch(Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), 0, Long.MAX_VALUE) + ) + ); - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.backwardFetch( - null, Bytes.wrap(keyB.getBytes()), 0, Long.MAX_VALUE)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyA, maxWindow), 10L), KeyValue.pair(new Windowed<>(keyB, maxWindow), 50L) - ); + ), + toListAndCloseIterator(bytesStore.backwardFetch(null, Bytes.wrap(keyB.getBytes()), 0, Long.MAX_VALUE)) + ); - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.backwardFetch( - Bytes.wrap(keyB.getBytes()), null, 0, Long.MAX_VALUE)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyB, maxWindow), 50L), KeyValue.pair(new Windowed<>(keyC, maxWindow), 100L) - ); + ), + toListAndCloseIterator(bytesStore.backwardFetch(Bytes.wrap(keyB.getBytes()), null, 0, Long.MAX_VALUE)) + ); - assertEquals(expected, toList(values)); - } - - try (final KeyValueIterator values = bytesStore.backwardFetch( - null, null, 0, Long.MAX_VALUE)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyA, maxWindow), 10L), KeyValue.pair(new Windowed<>(keyB, maxWindow), 50L), KeyValue.pair(new Windowed<>(keyC, maxWindow), 100L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(bytesStore.backwardFetch(null, null, 0, Long.MAX_VALUE)) + ); } + @SuppressWarnings("resource") @Test public void shouldFetchSessionForSingleKey() { // Only for TimeFirstSessionKeySchema schema @@ -949,74 +900,59 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { // Fetch point - try (final KeyValueIterator values = ((RocksDBTimeOrderedSessionSegmentedBytesStore) bytesStore).fetchSessions(100L, 100L)) { - - final List, Long>> expected = Collections.singletonList( - KeyValue.pair(new Windowed<>(keyA, sessionWindows[0]), 10L) - ); - - assertEquals(expected, toList(values)); - } + assertEquals( + Collections.singletonList(KeyValue.pair(new Windowed<>(keyA, sessionWindows[0]), 10L)), + toListAndCloseIterator(((RocksDBTimeOrderedSessionSegmentedBytesStore) bytesStore).fetchSessions(100L, 100L)) + ); // Fetch partial boundary - try (final KeyValueIterator values = ((RocksDBTimeOrderedSessionSegmentedBytesStore) bytesStore).fetchSessions(100L, 200L)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyA, sessionWindows[0]), 10L), KeyValue.pair(new Windowed<>(keyB, sessionWindows[1]), 100L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(((RocksDBTimeOrderedSessionSegmentedBytesStore) bytesStore).fetchSessions(100L, 200L)) + ); // Fetch partial - try (final KeyValueIterator values = ((RocksDBTimeOrderedSessionSegmentedBytesStore) bytesStore).fetchSessions(99L, 201L)) { - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyA, sessionWindows[0]), 10L), KeyValue.pair(new Windowed<>(keyB, sessionWindows[1]), 100L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(((RocksDBTimeOrderedSessionSegmentedBytesStore) bytesStore).fetchSessions(99L, 201L)) + ); // Fetch partial try (final KeyValueIterator values = ((RocksDBTimeOrderedSessionSegmentedBytesStore) bytesStore).fetchSessions(101L, 199L)) { - assertTrue(toList(values).isEmpty()); + assertTrue(toListAndCloseIterator(values).isEmpty()); } // Fetch all boundary - try (final KeyValueIterator values = ((RocksDBTimeOrderedSessionSegmentedBytesStore) bytesStore).fetchSessions(100L, 300L)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyA, sessionWindows[0]), 10L), KeyValue.pair(new Windowed<>(keyB, sessionWindows[1]), 100L), KeyValue.pair(new Windowed<>(keyC, sessionWindows[2]), 200L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(((RocksDBTimeOrderedSessionSegmentedBytesStore) bytesStore).fetchSessions(100L, 300L)) + ); // Fetch all - try (final KeyValueIterator values = ((RocksDBTimeOrderedSessionSegmentedBytesStore) bytesStore).fetchSessions(99L, 301L)) { - - final List, Long>> expected = asList( + assertEquals( + asList( KeyValue.pair(new Windowed<>(keyA, sessionWindows[0]), 10L), KeyValue.pair(new Windowed<>(keyB, sessionWindows[1]), 100L), KeyValue.pair(new Windowed<>(keyC, sessionWindows[2]), 200L) - ); - - assertEquals(expected, toList(values)); - } + ), + toListAndCloseIterator(((RocksDBTimeOrderedSessionSegmentedBytesStore) bytesStore).fetchSessions(99L, 301L)) + ); // Fetch all - try (final KeyValueIterator values = ((RocksDBTimeOrderedSessionSegmentedBytesStore) bytesStore).fetchSessions(101L, 299L)) { - - final List, Long>> expected = Collections.singletonList( - KeyValue.pair(new Windowed<>(keyB, sessionWindows[1]), 100L) - ); - - assertEquals(expected, toList(values)); - } + assertEquals( + Collections.singletonList(KeyValue.pair(new Windowed<>(keyB, sessionWindows[1]), 100L)), + toListAndCloseIterator(((RocksDBTimeOrderedSessionSegmentedBytesStore) bytesStore).fetchSessions(101L, 299L)) + ); } @Test @@ -1042,32 +978,31 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { final Bytes serializedKey2 = serializeKey(new Windowed<>(keyB, windows[2])); bytesStore.put(serializedKey2, serializeValue(20L)); - try (final KeyValueIterator results = bytesStore.fetch( - Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), 1, 2000)) { + final List, Long>> expected; - final List, Long>> expected; - - // actual from: observedStreamTime - retention + 1 - if (getBaseSchema() instanceof TimeFirstWindowKeySchema) { - // For windowkeyschema, actual from is 1 - // observed stream time = 1000. Retention Period = 1000. - // actual from = (1000 - 1000 + 1) - // and search happens in the range 1-2000 - expected = asList( - KeyValue.pair(new Windowed<>(keyA, windows[0]), 10L), - KeyValue.pair(new Windowed<>(keyB, windows[2]), 20L) - ); - } else { - // For session key schema, actual from is 501 - // observed stream time = 1500. Retention Period = 1000. - // actual from = (1500 - 1000 + 1) - // and search happens in the range 501-2000 - expected = Collections.singletonList(KeyValue.pair(new Windowed<>(keyB, windows[2]), 20L)); - } - - assertEquals(expected, toList(results)); + // actual from: observedStreamTime - retention + 1 + if (getBaseSchema() instanceof TimeFirstWindowKeySchema) { + // For windowkeyschema, actual from is 1 + // observed stream time = 1000. Retention Period = 1000. + // actual from = (1000 - 1000 + 1) + // and search happens in the range 1-2000 + expected = asList( + KeyValue.pair(new Windowed<>(keyA, windows[0]), 10L), + KeyValue.pair(new Windowed<>(keyB, windows[2]), 20L) + ); + } else { + // For session key schema, actual from is 501 + // observed stream time = 1500. Retention Period = 1000. + // actual from = (1500 - 1000 + 1) + // and search happens in the range 501-2000 + expected = Collections.singletonList(KeyValue.pair(new Windowed<>(keyB, windows[2]), 20L)); } + assertEquals( + expected, + toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(keyA.getBytes()), Bytes.wrap(keyB.getBytes()), 1, 2000)) + ); + // Dangling index should be deleted. value = bytesStore.getIndex(serializedKey1); assertThat(value, is(nullValue())); @@ -1082,28 +1017,28 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { bytesStore.put(serializeKey(new Windowed<>(key, windows[2])), serializeValue(100)); // actual from: observedStreamTime - retention + 1 // retention = 1000 - try (final KeyValueIterator results = bytesStore.fetch(Bytes.wrap(key.getBytes()), 1, 999)) { + final List, Long>> expected; - final List, Long>> expected; - - // actual from: observedStreamTime - retention + 1 - if (getBaseSchema() instanceof TimeFirstWindowKeySchema) { - // For windowkeyschema, actual from is 1 - // observed stream time = 1000. actual from = (1000 - 1000 + 1) - // and search happens in the range 1-2000 - expected = asList( - KeyValue.pair(new Windowed<>(key, windows[0]), 10L), - KeyValue.pair(new Windowed<>(key, windows[1]), 50L) - ); - } else { - // For session key schema, actual from is 501 - // observed stream time = 1500. actual from = (1500 - 1000 + 1) - // and search happens in the range 501-2000 deeming first record as expired. - expected = Collections.singletonList(KeyValue.pair(new Windowed<>(key, windows[1]), 50L)); - } - - assertEquals(expected, toList(results)); + // actual from: observedStreamTime - retention + 1 + if (getBaseSchema() instanceof TimeFirstWindowKeySchema) { + // For windowkeyschema, actual from is 1 + // observed stream time = 1000. actual from = (1000 - 1000 + 1) + // and search happens in the range 1-2000 + expected = asList( + KeyValue.pair(new Windowed<>(key, windows[0]), 10L), + KeyValue.pair(new Windowed<>(key, windows[1]), 50L) + ); + } else { + // For session key schema, actual from is 501 + // observed stream time = 1500. actual from = (1500 - 1000 + 1) + // and search happens in the range 501-2000 deeming first record as expired. + expected = Collections.singletonList(KeyValue.pair(new Windowed<>(key, windows[1]), 50L)); } + + assertEquals( + expected, + toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(key.getBytes()), 1, 999)) + ); } @Test @@ -1138,7 +1073,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { bytesStore.put(serializeKey(new Windowed<>(key, windows[3])), serializeValue(1000)); assertEquals(Set.of(segments.segmentName(0), segments.segmentName(1)), segmentDirs()); - final List, Long>> results = toList(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0, 1500)); + final List, Long>> results = toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0, 1500)); // For all tests, actualFrom is computed using observedStreamTime - retention + 1. // so actualFrom = 60000(observedStreamTime) - 1000(retention) + 1 = 59001 @@ -1148,7 +1083,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { results ); - final List, Long>> results1 = toList(bytesStore.fetch(Bytes.wrap(key.getBytes()), 59000, 60000)); + final List, Long>> results1 = toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(key.getBytes()), 59000, 60000)); // only non expired record as actual from is 59001 assertEquals( @@ -1180,7 +1115,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { segmentDirs() ); - final List, Long>> results = toList(bytesStore.all()); + final List, Long>> results = toListAndCloseIterator(bytesStore.all()); // actualFrom is computed using observedStreamTime - retention + 1. // so actualFrom = 60000(observedStreamTime) - 1000(retention) + 1 = 59001 // only one record returned as actual from is 59001 @@ -1215,7 +1150,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { // For all tests, actualFrom is computed using observedStreamTime - retention + 1. // so actualFrom = 60000(observedStreamTime) - 1000(retention) + 1 = 59001 // key A expired as actual from is 59,001 - final List, Long>> results = toList(bytesStore.backwardAll()); + final List, Long>> results = toListAndCloseIterator(bytesStore.backwardAll()); assertEquals( Collections.singletonList( KeyValue.pair(new Windowed<>(keyB, windows[3]), 100L) @@ -1244,7 +1179,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { segmentDirs() ); - final List, Long>> results = toList(bytesStore.fetchAll(0L, 60_000L)); + final List, Long>> results = toListAndCloseIterator(bytesStore.fetchAll(0L, 60_000L)); // For all tests, actualFrom is computed using observedStreamTime - retention + 1. // so actualFrom = 60000(observedStreamTime) - 1000(retention) + 1 = 59001 // only 1 record fetched as actual from is 59001 @@ -1279,8 +1214,8 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { bytesStore = getBytesStore(); - bytesStore.init((StateStoreContext) context, bytesStore); - final List, Long>> results = toList(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0L, 60_000L)); + bytesStore.init(context, bytesStore); + final List, Long>> results = toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0L, 60_000L)); assertThat( results, equalTo( @@ -1311,8 +1246,8 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { bytesStore = getBytesStore(); - bytesStore.init((StateStoreContext) context, bytesStore); - final List, Long>> results = toList(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0L, 60_000L)); + bytesStore.init(context, bytesStore); + final List, Long>> results = toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0L, 60_000L)); assertThat( results, equalTo( @@ -1332,7 +1267,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { // 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.close(); - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); bytesStore.put(serializeKey(new Windowed<>(key, windows[1])), serializeValue(100)); } @@ -1354,17 +1289,17 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { @Test public void shouldRestoreToByteStoreForActiveTask() { - shouldRestoreToByteStore(TaskType.ACTIVE); + shouldRestoreToByteStore(); } @Test public void shouldRestoreToByteStoreForStandbyTask() { context.transitionToStandby(null); - shouldRestoreToByteStore(TaskType.STANDBY); + shouldRestoreToByteStore(); } - private void shouldRestoreToByteStore(final TaskType taskType) { - bytesStore.init((StateStoreContext) context, bytesStore); + private void shouldRestoreToByteStore() { + bytesStore.init(context, bytesStore); // 0 segments initially. assertEquals(0, bytesStore.getSegments().size()); final String key = "a"; @@ -1380,13 +1315,13 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { expected.add(new KeyValue<>(new Windowed<>(key, windows[3]), 100L)); // after restoration, only 1 record should be returned as actual from is 59001 and the prior record is expired. - final List, Long>> results = toList(bytesStore.all()); + final List, Long>> results = toListAndCloseIterator(bytesStore.all()); assertEquals(expected, results); } @Test public void shouldMatchPositionAfterPut() { - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); final String keyA = "a"; final String keyB = "b"; @@ -1422,7 +1357,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { Time.SYSTEM ); bytesStore = getBytesStore(); - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); // 0 segments initially. assertEquals(0, bytesStore.getSegments().size()); @@ -1435,7 +1370,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { expected.add(new KeyValue<>(new Windowed<>(key, windows[3]), 200L)); // after restoration, only non expired segments should be returned which is one as actual from is 59001 - final List, Long>> results = toList(bytesStore.all()); + final List, Long>> results = toListAndCloseIterator(bytesStore.all()); assertEquals(expected, results); assertThat(bytesStore.getPosition(), Matchers.notNullValue()); assertThat(bytesStore.getPosition().getPartitionPositions(""), Matchers.notNullValue()); @@ -1458,7 +1393,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { Time.SYSTEM ); bytesStore = getBytesStore(); - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); // 0 segments initially. assertEquals(0, bytesStore.getSegments().size()); @@ -1472,7 +1407,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { final List, Long>> expected = new ArrayList<>(); expected.add(new KeyValue<>(new Windowed<>(key, windows[3]), 200L)); - final List, Long>> results = toList(bytesStore.all()); + final List, Long>> results = toListAndCloseIterator(bytesStore.all()); assertEquals(expected, results); assertThat(bytesStore.getPosition(), Matchers.notNullValue()); assertThat(bytesStore.getPosition().getPartitionPositions("A"), Matchers.notNullValue()); @@ -1497,7 +1432,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { Time.SYSTEM ); bytesStore = getBytesStore(); - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); // 0 segments initially. assertEquals(0, bytesStore.getSegments().size()); @@ -1516,7 +1451,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { expected.add(new KeyValue<>(new Windowed<>(key, windows[0]), 50L)); } - final List, Long>> results = toList(bytesStore.all()); + final List, Long>> results = toListAndCloseIterator(bytesStore.all()); assertEquals(expected, results); assertThat(bytesStore.getPosition(), Matchers.notNullValue()); assertThat(bytesStore.getPosition().getPartitionPositions("A"), hasEntry(0, 2L)); @@ -1538,7 +1473,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { Time.SYSTEM ); bytesStore = getBytesStore(); - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); bytesStore.restoreAllInternal(getChangelogRecordsWithoutHeaders()); assertThat(bytesStore.getPosition(), is(Position.emptyPosition())); } @@ -1649,13 +1584,13 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { public void shouldMeasureExpiredRecords() { final Properties streamsConfig = StreamsTestUtils.getStreamsConfig(); final AbstractDualSchemaRocksDBSegmentedBytesStore bytesStore = getBytesStore(); - final InternalMockProcessorContext context = new InternalMockProcessorContext( + final InternalMockProcessorContext context = new InternalMockProcessorContext<>( TestUtils.tempDirectory(), new StreamsConfig(streamsConfig) ); final Time time = Time.SYSTEM; 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 // that the subsequent record in windows[0] will already be expired. @@ -1736,36 +1671,40 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest { } } + @SuppressWarnings("resource") private byte[] serializeValue(final long value) { - return Serdes.Long().serializer().serialize("", value); + return new LongSerializer().serialize("", value); } - private List, Long>> toList(final KeyValueIterator iterator) { - final List, Long>> results = new ArrayList<>(); - final StateSerdes stateSerdes = StateSerdes.withBuiltinTypes("dummy", String.class, Long.class); - while (iterator.hasNext()) { - final KeyValue next = iterator.next(); - if (getBaseSchema() instanceof TimeFirstWindowKeySchema) { - final KeyValue, Long> deserialized = KeyValue.pair( - TimeFirstWindowKeySchema.fromStoreKey( - next.key.get(), - windowSizeForTimeWindow, - stateSerdes.keyDeserializer(), - stateSerdes.topic() - ), - stateSerdes.valueDeserializer().deserialize("dummy", next.value) - ); - results.add(deserialized); - } else if (getBaseSchema() instanceof TimeFirstSessionKeySchema) { - final KeyValue, Long> deserialized = KeyValue.pair( - TimeFirstSessionKeySchema.from(next.key.get(), stateSerdes.keyDeserializer(), "dummy"), - stateSerdes.valueDeserializer().deserialize("dummy", next.value) - ); - results.add(deserialized); - } else { - throw new IllegalStateException("Unrecognized serde schema"); + @SuppressWarnings("resource") + private List, Long>> toListAndCloseIterator(final KeyValueIterator iterator) { + try (iterator) { + final List, Long>> results = new ArrayList<>(); + final StateSerdes stateSerdes = StateSerdes.withBuiltinTypes("dummy", String.class, Long.class); + while (iterator.hasNext()) { + final KeyValue next = iterator.next(); + if (getBaseSchema() instanceof TimeFirstWindowKeySchema) { + final KeyValue, Long> deserialized = KeyValue.pair( + TimeFirstWindowKeySchema.fromStoreKey( + next.key.get(), + windowSizeForTimeWindow, + stateSerdes.keyDeserializer(), + stateSerdes.topic() + ), + stateSerdes.valueDeserializer().deserialize("dummy", next.value) + ); + results.add(deserialized); + } else if (getBaseSchema() instanceof TimeFirstSessionKeySchema) { + final KeyValue, Long> deserialized = KeyValue.pair( + TimeFirstSessionKeySchema.from(next.key.get(), stateSerdes.keyDeserializer(), "dummy"), + stateSerdes.valueDeserializer().deserialize("dummy", next.value) + ); + results.add(deserialized); + } else { + throw new IllegalStateException("Unrecognized serde schema"); + } } + return results; } - return results; } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java index 1e8c96f6f52..1175add263f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java @@ -38,11 +38,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; +import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator; import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.MatcherAssert.assertThat; 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.fail; -@SuppressWarnings("unchecked") public abstract class AbstractKeyValueStoreTest { protected abstract KeyValueStore createKeyValueStore(final StateStoreContext context); - protected InternalMockProcessorContext context; + protected InternalMockProcessorContext context; protected KeyValueStore store; protected KeyValueStoreTestDriver driver; @BeforeEach public void before() { driver = KeyValueStoreTestDriver.create(Integer.class, String.class); - context = (InternalMockProcessorContext) driver.context(); + context = (InternalMockProcessorContext) driver.context(); context.setTime(10); store = createKeyValueStore(context); } @@ -75,16 +74,17 @@ public abstract class AbstractKeyValueStoreTest { driver.clear(); } - private static Map getContents(final KeyValueIterator iter) { - final HashMap result = new HashMap<>(); - while (iter.hasNext()) { - final KeyValue entry = iter.next(); - result.put(entry.key, entry.value); + private static Map getContentsAndCloseIterator(final KeyValueIterator iter) { + try (iter) { + final HashMap result = new HashMap<>(); + while (iter.hasNext()) { + final KeyValue entry = iter.next(); + result.put(entry.key, entry.value); + } + return result; } - return result; } - @SuppressWarnings("unchecked") @Test public void shouldNotIncludeDeletedFromRangeResult() { store.close(); @@ -113,7 +113,7 @@ public abstract class AbstractKeyValueStoreTest { // should not include deleted records in iterator final Map expectedContents = Collections.singletonMap(2, "two"); - assertEquals(expectedContents, getContents(store.all())); + assertEquals(expectedContents, getContentsAndCloseIterator(store.all())); } @Test @@ -142,7 +142,7 @@ public abstract class AbstractKeyValueStoreTest { // should not include deleted records in iterator final Map expectedContents = Collections.singletonMap(2, "two"); - assertEquals(expectedContents, getContents(store.all())); + assertEquals(expectedContents, getContentsAndCloseIterator(store.all())); } @Test @@ -184,13 +184,13 @@ public abstract class AbstractKeyValueStoreTest { expectedContents.put(4, "four"); // Check range iteration ... - assertEquals(expectedContents, getContents(store.range(2, 4))); - assertEquals(expectedContents, getContents(store.range(2, 6))); + assertEquals(expectedContents, getContentsAndCloseIterator(store.range(2, 4))); + assertEquals(expectedContents, getContentsAndCloseIterator(store.range(2, 6))); // Check all iteration ... expectedContents.put(0, "zero"); expectedContents.put(1, "one"); - assertEquals(expectedContents, getContents(store.all())); + assertEquals(expectedContents, getContentsAndCloseIterator(store.all())); } @Test @@ -232,13 +232,13 @@ public abstract class AbstractKeyValueStoreTest { expectedContents.put(4, "four"); // Check range iteration ... - assertEquals(expectedContents, getContents(store.reverseRange(2, 4))); - assertEquals(expectedContents, getContents(store.reverseRange(2, 6))); + assertEquals(expectedContents, getContentsAndCloseIterator(store.reverseRange(2, 4))); + assertEquals(expectedContents, getContentsAndCloseIterator(store.reverseRange(2, 6))); // Check all iteration ... expectedContents.put(0, "zero"); expectedContents.put(1, "one"); - assertEquals(expectedContents, getContents(store.reverseAll())); + assertEquals(expectedContents, getContentsAndCloseIterator(store.reverseAll())); } @Test @@ -498,14 +498,10 @@ public abstract class AbstractKeyValueStoreTest { store.putAll(entries); - final List> allReturned = new ArrayList<>(); + final List> allReturned = toListAndCloseIterator(store.all()); final List> expectedReturned = Arrays.asList(KeyValue.pair(1, "one"), KeyValue.pair(2, "two")); - final Iterator> iterator = store.all(); - while (iterator.hasNext()) { - allReturned.add(iterator.next()); - } assertThat(allReturned, equalTo(expectedReturned)); } @@ -517,14 +513,10 @@ public abstract class AbstractKeyValueStoreTest { store.putAll(entries); - final List> allReturned = new ArrayList<>(); + final List> allReturned = toListAndCloseIterator(store.reverseAll()); final List> expectedReturned = Arrays.asList(KeyValue.pair(2, "two"), KeyValue.pair(1, "one")); - final Iterator> iterator = store.reverseAll(); - while (iterator.hasNext()) { - allReturned.add(iterator.next()); - } assertThat(allReturned, equalTo(expectedReturned)); } @@ -545,10 +537,10 @@ public abstract class AbstractKeyValueStoreTest { store.putAll(entries); - final Iterator> iterator = store.range(2, 2); - - assertEquals(iterator.next().value, store.get(2)); - assertFalse(iterator.hasNext()); + try (final KeyValueIterator iterator = store.range(2, 2)) { + assertEquals(iterator.next().value, store.get(2)); + assertFalse(iterator.hasNext()); + } } @Test @@ -560,10 +552,10 @@ public abstract class AbstractKeyValueStoreTest { store.putAll(entries); - final Iterator> iterator = store.reverseRange(2, 2); - - assertEquals(iterator.next().value, store.get(2)); - assertFalse(iterator.hasNext()); + try (final KeyValueIterator iterator = store.reverseRange(2, 2)) { + assertEquals(iterator.next().value, store.get(2)); + assertFalse(iterator.hasNext()); + } } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java index c659fa08417..7a78716530a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.RecordBatch; 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.utils.Bytes; 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.Windowed; 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.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; @@ -88,7 +88,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public abstract class AbstractRocksDBSegmentedBytesStoreTest { private final long windowSizeForTimeWindow = 500; - private InternalMockProcessorContext context; + private InternalMockProcessorContext context; private AbstractRocksDBSegmentedBytesStore bytesStore; private File stateDir; private final Window[] windows = new Window[4]; @@ -140,7 +140,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest new MockRecordCollector(), new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())) ); - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); } @AfterEach @@ -168,21 +168,21 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest Bytes.wrap(keyA.getBytes()), 0, windows[2].start())) { // All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1) // for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000 - assertEquals(Collections.emptyList(), toList(values)); + assertEquals(Collections.emptyList(), toListAndCloseIterator(values)); } try (final KeyValueIterator values = bytesStore.fetch( 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) // for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000 - assertEquals(Collections.emptyList(), toList(values)); + assertEquals(Collections.emptyList(), toListAndCloseIterator(values)); } try (final KeyValueIterator values = bytesStore.fetch( null, Bytes.wrap(keyB.getBytes()), 0, windows[2].start())) { // All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1) // for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000 - assertEquals(Collections.emptyList(), toList(values)); + assertEquals(Collections.emptyList(), toListAndCloseIterator(values)); } try (final KeyValueIterator values = bytesStore.fetch( @@ -193,7 +193,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L) ); - assertEquals(expected, toList(values)); + assertEquals(expected, toListAndCloseIterator(values)); } try (final KeyValueIterator values = bytesStore.fetch( @@ -204,7 +204,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L) ); - assertEquals(expected, toList(values)); + assertEquals(expected, toListAndCloseIterator(values)); } } @@ -225,7 +225,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest // All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1) // for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000 - assertEquals(Collections.emptyList(), toList(values)); + assertEquals(Collections.emptyList(), toListAndCloseIterator(values)); } try (final KeyValueIterator values = bytesStore.backwardFetch( @@ -233,7 +233,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest // All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1) // for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000 - assertEquals(Collections.emptyList(), toList(values)); + assertEquals(Collections.emptyList(), toListAndCloseIterator(values)); } try (final KeyValueIterator values = bytesStore.backwardFetch( @@ -241,7 +241,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest // All Records expired as observed stream time = 60000 implying actual-from = 59001 (60000 - 1000 + 1) // for WindowKeySchema, to = 60000 while for SessionKeySchema, to = 30000 - assertEquals(Collections.emptyList(), toList(values)); + assertEquals(Collections.emptyList(), toListAndCloseIterator(values)); } try (final KeyValueIterator values = bytesStore.backwardFetch( @@ -252,7 +252,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L) ); - assertEquals(expected, toList(values)); + assertEquals(expected, toListAndCloseIterator(values)); } try (final KeyValueIterator values = bytesStore.backwardFetch( @@ -263,7 +263,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest KeyValue.pair(new Windowed<>(keyC, windows[3]), 200L) ); - assertEquals(expected, toList(values)); + assertEquals(expected, toListAndCloseIterator(values)); } } @@ -289,7 +289,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest 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 bytesStore.put(serializeKey(new Windowed<>(key, windows[3])), serializeValue(1000)); assertEquals(Set.of(segments.segmentName(0), segments.segmentName(1)), segmentDirs()); - final List, Long>> results = toList(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0, 1500)); + final List, 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. */ @@ -355,7 +355,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest /* * Only 1 record returned. observed stream time = 60000, actual from = 59001 (60000 - 1000 + 1) and to = Long.MAX. */ - final List, Long>> results = toList(bytesStore.all()); + final List, Long>> results = toListAndCloseIterator(bytesStore.all()); assertEquals( Collections.singletonList( KeyValue.pair(new Windowed<>(key, windows[3]), 100L) @@ -388,7 +388,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest /* * Only 1 record returned. observed stream time = 60000, actual from = 59001 (60000 - 1000 + 1) and to = 60,000. */ - final List, Long>> results = toList(bytesStore.fetchAll(0L, 60_000L)); + final List, Long>> results = toListAndCloseIterator(bytesStore.fetchAll(0L, 60_000L)); assertEquals( Collections.singletonList( KeyValue.pair(new Windowed<>(key, windows[3]), 100L) @@ -422,8 +422,8 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest bytesStore = getBytesStore(); - bytesStore.init((StateStoreContext) context, bytesStore); - final List, Long>> results = toList(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0L, 60_000L)); + bytesStore.init(context, bytesStore); + final List, Long>> results = toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0L, 60_000L)); assertThat( results, equalTo( @@ -456,8 +456,8 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest bytesStore = getBytesStore(); - bytesStore.init((StateStoreContext) context, bytesStore); - final List, Long>> results = toList(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0L, 60_000L)); + bytesStore.init(context, bytesStore); + final List, Long>> results = toListAndCloseIterator(bytesStore.fetch(Bytes.wrap(key.getBytes()), 0L, 60_000L)); assertThat( results, equalTo( @@ -479,7 +479,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest // 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.close(); - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); bytesStore.put(serializeKey(new Windowed<>(key, windows[1])), serializeValue(100)); } @@ -515,7 +515,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest } private void shouldRestoreToByteStore() { - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); // 0 segments initially. assertEquals(0, bytesStore.getSegments().size()); final String key = "a"; @@ -533,7 +533,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest final List, Long>> expected = new ArrayList<>(); expected.add(new KeyValue<>(new Windowed<>(key, windows[3]), 100L)); - final List, Long>> results = toList(bytesStore.all()); + final List, Long>> results = toListAndCloseIterator(bytesStore.all()); assertEquals(expected, results); } @@ -541,7 +541,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest @MethodSource("getKeySchemas") public void shouldMatchPositionAfterPut(final SegmentedBytesStore.KeySchema schema) { before(schema); - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); final String keyA = "a"; final String keyB = "b"; @@ -579,7 +579,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest Time.SYSTEM ); bytesStore = getBytesStore(); - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); // 0 segments initially. assertEquals(0, bytesStore.getSegments().size()); @@ -594,7 +594,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest final List, Long>> expected = new ArrayList<>(); expected.add(new KeyValue<>(new Windowed<>(key, windows[3]), 200L)); - final List, Long>> results = toList(bytesStore.all()); + final List, Long>> results = toListAndCloseIterator(bytesStore.all()); assertEquals(expected, results); assertThat(bytesStore.getPosition(), Matchers.notNullValue()); assertThat(bytesStore.getPosition().getPartitionPositions(""), Matchers.notNullValue()); @@ -619,7 +619,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest Time.SYSTEM ); bytesStore = getBytesStore(); - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); // 0 segments initially. assertEquals(0, bytesStore.getSegments().size()); @@ -634,7 +634,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest final List, Long>> expected = new ArrayList<>(); expected.add(new KeyValue<>(new Windowed<>(key, windows[3]), 200L)); - final List, Long>> results = toList(bytesStore.all()); + final List, Long>> results = toListAndCloseIterator(bytesStore.all()); assertEquals(expected, results); assertThat(bytesStore.getPosition(), Matchers.notNullValue()); assertThat(bytesStore.getPosition().getPartitionPositions("A"), Matchers.notNullValue()); @@ -661,7 +661,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest Time.SYSTEM ); bytesStore = getBytesStore(); - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); // 0 segments initially. assertEquals(0, bytesStore.getSegments().size()); @@ -675,7 +675,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest * 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. */ - final List, Long>> results = toList(bytesStore.all()); + final List, Long>> results = toListAndCloseIterator(bytesStore.all()); if (schema instanceof SessionKeySchema) { assertEquals(Collections.emptyList(), results); } else { @@ -705,7 +705,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest Time.SYSTEM ); bytesStore = getBytesStore(); - bytesStore.init((StateStoreContext) context, bytesStore); + bytesStore.init(context, bytesStore); bytesStore.restoreAllInternal(getChangelogRecordsWithoutHeaders()); assertThat(bytesStore.getPosition(), is(Position.emptyPosition())); } @@ -816,13 +816,13 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest before(schema); final Properties streamsConfig = StreamsTestUtils.getStreamsConfig(); final AbstractRocksDBSegmentedBytesStore bytesStore = getBytesStore(); - final InternalMockProcessorContext context = new InternalMockProcessorContext( + final InternalMockProcessorContext context = new InternalMockProcessorContext<>( TestUtils.tempDirectory(), new StreamsConfig(streamsConfig) ); final Time time = Time.SYSTEM; 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 // that the subsequent record in windows[0] will already be expired. @@ -878,36 +878,40 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest } } + @SuppressWarnings("resource") private byte[] serializeValue(final long value) { - return Serdes.Long().serializer().serialize("", value); + return new LongSerializer().serialize("", value); } - private List, Long>> toList(final KeyValueIterator iterator) { - final List, Long>> results = new ArrayList<>(); - final StateSerdes stateSerdes = StateSerdes.withBuiltinTypes("dummy", String.class, Long.class); - while (iterator.hasNext()) { - final KeyValue next = iterator.next(); - if (schema instanceof WindowKeySchema) { - final KeyValue, Long> deserialized = KeyValue.pair( - WindowKeySchema.fromStoreKey( - next.key.get(), - windowSizeForTimeWindow, - stateSerdes.keyDeserializer(), - stateSerdes.topic() - ), - stateSerdes.valueDeserializer().deserialize("dummy", next.value) - ); - results.add(deserialized); - } else if (schema instanceof SessionKeySchema) { - final KeyValue, Long> deserialized = KeyValue.pair( - SessionKeySchema.from(next.key.get(), stateSerdes.keyDeserializer(), "dummy"), - stateSerdes.valueDeserializer().deserialize("dummy", next.value) - ); - results.add(deserialized); - } else { - throw new IllegalStateException("Unrecognized serde schema"); + @SuppressWarnings("resource") + private List, Long>> toListAndCloseIterator(final KeyValueIterator iterator) { + try (iterator) { + final List, Long>> results = new ArrayList<>(); + final StateSerdes stateSerdes = StateSerdes.withBuiltinTypes("dummy", String.class, Long.class); + while (iterator.hasNext()) { + final KeyValue next = iterator.next(); + if (schema instanceof WindowKeySchema) { + final KeyValue, Long> deserialized = KeyValue.pair( + WindowKeySchema.fromStoreKey( + next.key.get(), + windowSizeForTimeWindow, + stateSerdes.keyDeserializer(), + stateSerdes.topic() + ), + stateSerdes.valueDeserializer().deserialize("dummy", next.value) + ); + results.add(deserialized); + } else if (schema instanceof SessionKeySchema) { + final KeyValue, Long> deserialized = KeyValue.pair( + SessionKeySchema.from(next.key.get(), stateSerdes.keyDeserializer(), "dummy"), + stateSerdes.valueDeserializer().deserialize("dummy", next.value) + ); + results.add(deserialized); + } else { + throw new IllegalStateException("Unrecognized serde schema"); + } } + return results; } - return results; } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBWindowStoreTest.java index 700694fc6da..7760d32e431 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBWindowStoreTest.java @@ -23,7 +23,6 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Utils; 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.query.Position; 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 org.apache.kafka.common.utils.Utils.mkEntry; 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.assertFalse; @@ -177,7 +176,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes // expired record assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 0, ofEpochMilli(startTime - WINDOW_SIZE), ofEpochMilli(startTime + WINDOW_SIZE)))); @@ -193,7 +192,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes if (storeType() == StoreType.RocksDBWindowStore) { assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 1, ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment + WINDOW_SIZE)))); @@ -201,32 +200,32 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes } else { assertEquals( new HashSet<>(Collections.singletonList("one")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 1, ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment + WINDOW_SIZE)))); } assertEquals( new HashSet<>(Collections.singletonList("two")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 3, ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("four")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 4, ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("five")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 5, ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE)))); @@ -243,13 +242,13 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 0, ofEpochMilli(startTime - WINDOW_SIZE), ofEpochMilli(startTime + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 1, ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment + WINDOW_SIZE)))); @@ -266,7 +265,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes assertEquals( // expired record new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); @@ -274,32 +273,32 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes assertEquals( // expired record new HashSet<>(Collections.singletonList("two")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); } assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 3, ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("four")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 4, ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("five")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 5, ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("six")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 6, ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE)))); @@ -316,50 +315,50 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 0, ofEpochMilli(startTime - WINDOW_SIZE), ofEpochMilli(startTime + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 1, ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment + WINDOW_SIZE)))); assertEquals( // expired record new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 3, ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("four")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 4, ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("five")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 5, ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("six")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 6, ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("seven")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 7, ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE)))); @@ -376,25 +375,25 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 0, ofEpochMilli(startTime - WINDOW_SIZE), ofEpochMilli(startTime + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 1, ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 3, ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE)))); @@ -402,7 +401,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes assertEquals( // expired record new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 4, ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); @@ -410,7 +409,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes assertEquals( // expired record new HashSet<>(Collections.singletonList("four")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 4, ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); @@ -418,25 +417,25 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes } assertEquals( new HashSet<>(Collections.singletonList("five")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 5, ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("six")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 6, ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("seven")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 7, ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("eight")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 8, 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 = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), Serdes.String()); - windowStore.init((StateStoreContext) context, windowStore); + windowStore.init(context, windowStore); context.setTime(0L); windowStore.put(0, "v", 0); @@ -480,14 +479,14 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes segmentDirs(baseDir) ); - WindowStoreIterator iter; int fetchedCount; - iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(SEGMENT_INTERVAL * 4)); - fetchedCount = 0; - while (iter.hasNext()) { - iter.next(); - fetchedCount++; + try (final WindowStoreIterator iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(SEGMENT_INTERVAL * 4))) { + fetchedCount = 0; + while (iter.hasNext()) { + iter.next(); + fetchedCount++; + } } assertEquals(4, fetchedCount); @@ -498,11 +497,12 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes windowStore.put(0, "v", SEGMENT_INTERVAL * 3); - iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(SEGMENT_INTERVAL * 4)); - fetchedCount = 0; - while (iter.hasNext()) { - iter.next(); - fetchedCount++; + try (final WindowStoreIterator iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(SEGMENT_INTERVAL * 4))) { + fetchedCount = 0; + while (iter.hasNext()) { + iter.next(); + fetchedCount++; + } } // 1 extra record is expired in the case of RocksDBWindowStore as // actualFrom = observedStreamTime - retentionPeriod + 1. The +1 @@ -520,11 +520,12 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes windowStore.put(0, "v", SEGMENT_INTERVAL * 5); - iter = windowStore.fetch(0, ofEpochMilli(SEGMENT_INTERVAL * 4), ofEpochMilli(SEGMENT_INTERVAL * 10)); - fetchedCount = 0; - while (iter.hasNext()) { - iter.next(); - fetchedCount++; + try (final WindowStoreIterator iter = windowStore.fetch(0, ofEpochMilli(SEGMENT_INTERVAL * 4), ofEpochMilli(SEGMENT_INTERVAL * 10))) { + fetchedCount = 0; + while (iter.hasNext()) { + iter.next(); + fetchedCount++; + } } // the latest record has a timestamp > 60k. So, the +1 in actualFrom calculation in @@ -552,7 +553,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes windowStore.close(); 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 windowStore.put(1, "v", 6L * SEGMENT_INTERVAL); @@ -568,7 +569,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes assertEquals(expected, actual); - try (final WindowStoreIterator iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(1000000L))) { + try (final WindowStoreIterator iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(1000000L))) { while (iter.hasNext()) { iter.next(); } @@ -583,7 +584,6 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes ); } - @SuppressWarnings("unchecked") @Test public void testRestore() throws Exception { final long startTime = SEGMENT_INTERVAL * 2; @@ -610,62 +610,62 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes false, Serdes.Integer(), Serdes.String()); - windowStore.init((StateStoreContext) context, windowStore); + windowStore.init(context, windowStore); // For all tests, for WindowStore actualFrom is computed using observedStreamTime - retention + 1. // while for TimeOrderedWindowStores, actualFrom = observedStreamTime - retention assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 0, ofEpochMilli(startTime - WINDOW_SIZE), ofEpochMilli(startTime + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 1, ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 3, ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 4, ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 5, ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 6, ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 7, ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 8, ofEpochMilli(startTime + increment * 8 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 8 + WINDOW_SIZE)))); @@ -679,25 +679,25 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 0, ofEpochMilli(startTime - WINDOW_SIZE), ofEpochMilli(startTime + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 1, ofEpochMilli(startTime + increment - WINDOW_SIZE), ofEpochMilli(startTime + increment + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 3, 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) { assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 4, ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); } else { assertEquals( new HashSet<>(Collections.singletonList("four")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 4, ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE)))); @@ -728,25 +728,25 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes } assertEquals( new HashSet<>(Collections.singletonList("five")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 5, ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("six")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 6, ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("seven")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 7, ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("eight")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 8, 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 public void shouldMatchPositionAfterPut() { final MeteredWindowStore meteredSessionStore = (MeteredWindowStore) windowStore; final ChangeLoggingWindowBytesStore changeLoggingSessionBytesStore = (ChangeLoggingWindowBytesStore) meteredSessionStore.wrapped(); - final WrappedStateStore rocksDBWindowStore = (WrappedStateStore) changeLoggingSessionBytesStore.wrapped(); + final WrappedStateStore rocksDBWindowStore = (WrappedStateStore) changeLoggingSessionBytesStore.wrapped(); context.setRecordContext(new ProcessorRecordContext(0, 1, 0, "", new RecordHeaders())); windowStore.put(0, "0", SEGMENT_INTERVAL); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java index 161f790c142..744f3ebc864 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java @@ -21,8 +21,10 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.header.internals.RecordHeaders; 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.Serdes; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.LogCaptureAppender; 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.kstream.Windowed; 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.ProcessorRecordContext; import org.apache.kafka.streams.query.Position; @@ -93,7 +94,7 @@ public abstract class AbstractSessionBytesStoreTest { private MockRecordCollector recordCollector; - InternalMockProcessorContext context; + InternalMockProcessorContext context; SessionStore buildSessionStore(final long retentionPeriod, final Serde keySerde, @@ -159,7 +160,7 @@ public abstract class AbstractSessionBytesStoreTest { new MockStreamsMetrics(new Metrics()))); context.setTime(1L); - sessionStore.init((StateStoreContext) context, sessionStore); + sessionStore.init(context, sessionStore); } @AfterEach @@ -240,6 +241,7 @@ public abstract class AbstractSessionBytesStoreTest { } } + @SuppressWarnings("resource") @Test public void shouldFindSessionsForTimeRange() { sessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 5L); @@ -552,7 +554,7 @@ public abstract class AbstractSessionBytesStoreTest { public void shouldFetchExactKeys() { sessionStore.close(); 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<>("aa", new SessionWindow(0, 10)), 2L); @@ -608,7 +610,7 @@ public abstract class AbstractSessionBytesStoreTest { public void shouldBackwardFetchExactKeys() { sessionStore.close(); 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<>("aa", new SessionWindow(0, 10)), 2L); @@ -665,7 +667,7 @@ public abstract class AbstractSessionBytesStoreTest { final SessionStore sessionStore = 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 key2 = Bytes.wrap(new byte[] {0, 0}); @@ -704,7 +706,7 @@ public abstract class AbstractSessionBytesStoreTest { final SessionStore sessionStore = 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 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); try (final KeyValueIterator, Long> iterator = sessionStore.findSessions("a", 0L, 20)) { - - assertEquals(iterator.peekNextKey(), new Windowed<>("a", new SessionWindow(0L, 0L))); - assertEquals(iterator.peekNextKey(), iterator.next().key); - assertEquals(iterator.peekNextKey(), iterator.next().key); + assertEquals(new Windowed<>("a", new SessionWindow(0L, 0L)), iterator.peekNextKey()); + final Windowed k1 = iterator.peekNextKey(); + assertEquals(iterator.next().key, k1); + final Windowed k2 = iterator.peekNextKey(); + assertEquals(iterator.next().key, k2); assertFalse(iterator.hasNext()); } } @@ -763,15 +766,15 @@ public abstract class AbstractSessionBytesStoreTest { sessionStore.put(new Windowed<>("aa", new SessionWindow(10, 20)), 4L); try (final KeyValueIterator, Long> iterator = sessionStore.backwardFindSessions("a", 0L, 20)) { - - assertEquals(iterator.peekNextKey(), new Windowed<>("a", new SessionWindow(10L, 20L))); - assertEquals(iterator.peekNextKey(), iterator.next().key); - assertEquals(iterator.peekNextKey(), iterator.next().key); + assertEquals(new Windowed<>("a", new SessionWindow(10L, 20L)), iterator.peekNextKey()); + final Windowed k1 = iterator.peekNextKey(); + assertEquals(iterator.next().key, k1); + final Windowed k2 = iterator.peekNextKey(); + assertEquals(iterator.next().key, k2); assertFalse(iterator.hasNext()); } } - @SuppressWarnings("unchecked") @Test public void shouldRestore() { final List, Long>> expected = Arrays.asList( @@ -842,7 +845,7 @@ public abstract class AbstractSessionBytesStoreTest { public void shouldMeasureExpiredRecords() { final Properties streamsConfig = StreamsTestUtils.getStreamsConfig(); final SessionStore sessionStore = buildSessionStore(RETENTION_PERIOD, Serdes.String(), Serdes.Long()); - final InternalMockProcessorContext context = new InternalMockProcessorContext( + final InternalMockProcessorContext context = new InternalMockProcessorContext<>( TestUtils.tempDirectory(), new StreamsConfig(streamsConfig), recordCollector @@ -850,7 +853,7 @@ public abstract class AbstractSessionBytesStoreTest { final Time time = Time.SYSTEM; context.setTime(1L); 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 // 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))); } + @SuppressWarnings("resource") @Test public void shouldThrowNullPointerExceptionOnFindSessionsNullKey() { assertThrows(NullPointerException.class, () -> sessionStore.findSessions(null, 1L, 2L)); } + @SuppressWarnings("resource") @Test public void shouldThrowNullPointerExceptionOnFetchNullKey() { assertThrows(NullPointerException.class, () -> sessionStore.fetch(null)); @@ -914,12 +919,11 @@ public abstract class AbstractSessionBytesStoreTest { assertThrows(NullPointerException.class, () -> sessionStore.put(null, 1L)); } + @SuppressWarnings("resource") @Test public void shouldNotThrowInvalidRangeExceptionWithNegativeFromKey() { - final String keyFrom = Serdes.String().deserializer() - .deserialize("", Serdes.Integer().serializer().serialize("", -1)); - final String keyTo = Serdes.String().deserializer() - .deserialize("", Serdes.Integer().serializer().serialize("", 1)); + final String keyFrom = new StringDeserializer().deserialize("", new IntegerSerializer().serialize("", -1)); + final String keyTo = new StringDeserializer().deserialize("", new IntegerSerializer().serialize("", 1)); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(); final KeyValueIterator, Long> iterator = sessionStore.findSessions(keyFrom, keyTo, 0L, 10L)) { @@ -968,16 +972,12 @@ public abstract class AbstractSessionBytesStoreTest { @Test public void shouldMatchPositionAfterPut() { - final MeteredSessionStore meteredSessionStore = (MeteredSessionStore) sessionStore; - final ChangeLoggingSessionBytesStore changeLoggingSessionBytesStore = (ChangeLoggingSessionBytesStore) meteredSessionStore.wrapped(); - final SessionStore wrapped = (SessionStore) changeLoggingSessionBytesStore.wrapped(); - context.setRecordContext(new ProcessorRecordContext(0, 1, 0, "", new RecordHeaders())); - sessionStore.put(new Windowed("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())); - sessionStore.put(new Windowed("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())); - sessionStore.put(new Windowed("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 actual = sessionStore.getPosition(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java index 732527c2c3a..8d2e7e61abd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java @@ -30,7 +30,6 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; 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.ProcessorRecordContext; 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 org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.apache.kafka.test.StreamsTestUtils.toList; +import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator; 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.hasItem; import static org.hamcrest.MatcherAssert.assertThat; @@ -89,7 +88,7 @@ public abstract class AbstractWindowBytesStoreTest { final KeyValue, String> five = windowedPair(5, "five", defaultStartTime + 5); WindowStore windowStore; - InternalMockProcessorContext context; + InternalMockProcessorContext context; MockRecordCollector recordCollector; final File baseDir = TestUtils.tempDirectory("test"); @@ -117,7 +116,7 @@ public abstract class AbstractWindowBytesStoreTest { new MockStreamsMetrics(new Metrics()))); context.setTime(1L); - windowStore.init((StateStoreContext) context, windowStore); + windowStore.init(context, windowStore); } @AfterEach @@ -131,12 +130,12 @@ public abstract class AbstractWindowBytesStoreTest { assertEquals( new HashSet<>(Collections.singletonList("zero")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 0, - ofEpochMilli(defaultStartTime + 0 - WINDOW_SIZE), - ofEpochMilli(defaultStartTime + 0 + WINDOW_SIZE)))); + ofEpochMilli(defaultStartTime - WINDOW_SIZE), + ofEpochMilli(defaultStartTime + WINDOW_SIZE)))); - putSecondBatch(windowStore, defaultStartTime, context); + putSecondBatch(windowStore, defaultStartTime); assertEquals("two+1", windowStore.fetch(2, defaultStartTime + 3L)); assertEquals("two+2", windowStore.fetch(2, defaultStartTime + 4L)); @@ -147,91 +146,91 @@ public abstract class AbstractWindowBytesStoreTest { assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime - 2L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - 2L + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("two")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime - 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - 1L + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("two", "two+1")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("two", "two+1", "two+2")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime + 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("two", "two+1", "two+2", "two+3")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime + 2L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("two", "two+1", "two+2", "two+3", "two+4")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("two", "two+1", "two+2", "two+3", "two+4", "two+5")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("two", "two+1", "two+2", "two+3", "two+4", "two+5", "two+6")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime + 5L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 5L + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("two+1", "two+2", "two+3", "two+4", "two+5", "two+6")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime + 6L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 6L + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("two+2", "two+3", "two+4", "two+5", "two+6")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime + 7L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 7L + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("two+3", "two+4", "two+5", "two+6")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime + 8L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 8L + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("two+4", "two+5", "two+6")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime + 9L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 9L + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("two+5", "two+6")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime + 10L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 10L + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("two+6")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime + 11L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 11L + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 2, ofEpochMilli(defaultStartTime + 12L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 12L + WINDOW_SIZE)))); @@ -264,14 +263,14 @@ public abstract class AbstractWindowBytesStoreTest { assertEquals( asList(zero, one, two, three, four, five), - toList(windowStore.all()) + toListAndCloseIterator(windowStore.all()) ); } @Test public void shouldGetAllNonDeletedRecords() { // Add some records - windowStore.put(0, "zero", defaultStartTime + 0); + windowStore.put(0, "zero", defaultStartTime); windowStore.put(1, "one", defaultStartTime + 1); windowStore.put(2, "two", defaultStartTime + 2); windowStore.put(3, "three", defaultStartTime + 3); @@ -284,7 +283,7 @@ public abstract class AbstractWindowBytesStoreTest { // Only non-deleted records should appear in the all() iterator assertEquals( asList(zero, two, four), - toList(windowStore.all()) + toListAndCloseIterator(windowStore.all()) ); } @@ -292,7 +291,7 @@ public abstract class AbstractWindowBytesStoreTest { public void shouldGetAllReturnTimestampOrderedRecords() { // Add some records in different order 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(3, "three", defaultStartTime + 3); windowStore.put(1, "one", defaultStartTime + 1); @@ -302,13 +301,13 @@ public abstract class AbstractWindowBytesStoreTest { assertEquals( asList(zero, one, two, three, four), - toList(windowStore.all()) + toListAndCloseIterator(windowStore.all()) ); } @Test public void shouldEarlyClosedIteratorStillGetAllRecords() { - windowStore.put(0, "zero", defaultStartTime + 0); + windowStore.put(0, "zero", defaultStartTime); windowStore.put(1, "one", defaultStartTime + 1); final KeyValueIterator, 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. assertEquals( asList(zero, one), - toList(windowStore.all()) + toListAndCloseIterator(windowStore.all()) ); } @@ -328,7 +327,7 @@ public abstract class AbstractWindowBytesStoreTest { assertEquals( asList(five, four, three, two, one, zero), - toList(windowStore.backwardAll()) + toListAndCloseIterator(windowStore.backwardAll()) ); } @@ -338,15 +337,15 @@ public abstract class AbstractWindowBytesStoreTest { assertEquals( asList(one, two, three, four), - toList(windowStore.fetchAll(ofEpochMilli(defaultStartTime + 1), ofEpochMilli(defaultStartTime + 4))) + toListAndCloseIterator(windowStore.fetchAll(ofEpochMilli(defaultStartTime + 1), ofEpochMilli(defaultStartTime + 4))) ); assertEquals( asList(zero, one, two, three), - toList(windowStore.fetchAll(ofEpochMilli(defaultStartTime), ofEpochMilli(defaultStartTime + 3))) + toListAndCloseIterator(windowStore.fetchAll(ofEpochMilli(defaultStartTime), ofEpochMilli(defaultStartTime + 3))) ); assertEquals( 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( asList(four, three, two, one), - toList(windowStore.backwardFetchAll(ofEpochMilli(defaultStartTime + 1), ofEpochMilli(defaultStartTime + 4))) + toListAndCloseIterator(windowStore.backwardFetchAll(ofEpochMilli(defaultStartTime + 1), ofEpochMilli(defaultStartTime + 4))) ); assertEquals( asList(three, two, one, zero), - toList(windowStore.backwardFetchAll(ofEpochMilli(defaultStartTime), ofEpochMilli(defaultStartTime + 3))) + toListAndCloseIterator(windowStore.backwardFetchAll(ofEpochMilli(defaultStartTime), ofEpochMilli(defaultStartTime + 3))) ); assertEquals( 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( asList(zero, one), - toList(windowStore.fetch( + toListAndCloseIterator(windowStore.fetch( 0, 1, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), - ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) + ofEpochMilli(defaultStartTime - WINDOW_SIZE), + ofEpochMilli(defaultStartTime + WINDOW_SIZE))) ); assertEquals( Collections.singletonList(one), - toList(windowStore.fetch( + toListAndCloseIterator(windowStore.fetch( 1, 1, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), - ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) + ofEpochMilli(defaultStartTime - WINDOW_SIZE), + ofEpochMilli(defaultStartTime + WINDOW_SIZE))) ); assertEquals( asList(one, two, three), - toList(windowStore.fetch( + toListAndCloseIterator(windowStore.fetch( 1, 3, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), - ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) + ofEpochMilli(defaultStartTime - WINDOW_SIZE), + ofEpochMilli(defaultStartTime + WINDOW_SIZE))) ); assertEquals( asList(zero, one, two, three), - toList(windowStore.fetch( + toListAndCloseIterator(windowStore.fetch( 0, 5, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), - ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) + ofEpochMilli(defaultStartTime - WINDOW_SIZE), + ofEpochMilli(defaultStartTime + WINDOW_SIZE))) ); assertEquals( asList(zero, one, two, three, four, five), - toList(windowStore.fetch( + toListAndCloseIterator(windowStore.fetch( 0, 5, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), - ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE + 5L))) + ofEpochMilli(defaultStartTime - WINDOW_SIZE), + ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L))) ); assertEquals( asList(two, three, four, five), - toList(windowStore.fetch( + toListAndCloseIterator(windowStore.fetch( 0, 5, ofEpochMilli(defaultStartTime + 2L), - ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE + 5L))) + ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L))) ); assertEquals( Collections.emptyList(), - toList(windowStore.fetch( + toListAndCloseIterator(windowStore.fetch( 4, 5, ofEpochMilli(defaultStartTime + 2L), @@ -430,7 +429,7 @@ public abstract class AbstractWindowBytesStoreTest { ); assertEquals( Collections.emptyList(), - toList(windowStore.fetch( + toListAndCloseIterator(windowStore.fetch( 0, 3, ofEpochMilli(defaultStartTime + 3L), @@ -438,26 +437,26 @@ public abstract class AbstractWindowBytesStoreTest { ); assertEquals( asList(zero, one, two), - toList(windowStore.fetch( + toListAndCloseIterator(windowStore.fetch( null, 2, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), + ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime + WINDOW_SIZE + 2L))) ); assertEquals( asList(two, three, four, five), - toList(windowStore.fetch( + toListAndCloseIterator(windowStore.fetch( 2, null, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), + ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L))) ); assertEquals( asList(zero, one, two, three, four, five), - toList(windowStore.fetch( + toListAndCloseIterator(windowStore.fetch( null, null, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), + ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L))) ); } @@ -469,55 +468,55 @@ public abstract class AbstractWindowBytesStoreTest { assertEquals( asList(one, zero), - toList(windowStore.backwardFetch( + toListAndCloseIterator(windowStore.backwardFetch( 0, 1, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), - ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) + ofEpochMilli(defaultStartTime - WINDOW_SIZE), + ofEpochMilli(defaultStartTime + WINDOW_SIZE))) ); assertEquals( Collections.singletonList(one), - toList(windowStore.backwardFetch( + toListAndCloseIterator(windowStore.backwardFetch( 1, 1, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), - ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) + ofEpochMilli(defaultStartTime - WINDOW_SIZE), + ofEpochMilli(defaultStartTime + WINDOW_SIZE))) ); assertEquals( asList(three, two, one), - toList(windowStore.backwardFetch( + toListAndCloseIterator(windowStore.backwardFetch( 1, 3, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), - ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) + ofEpochMilli(defaultStartTime - WINDOW_SIZE), + ofEpochMilli(defaultStartTime + WINDOW_SIZE))) ); assertEquals( asList(three, two, one, zero), - toList(windowStore.backwardFetch( + toListAndCloseIterator(windowStore.backwardFetch( 0, 5, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), - ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE))) + ofEpochMilli(defaultStartTime - WINDOW_SIZE), + ofEpochMilli(defaultStartTime + WINDOW_SIZE))) ); assertEquals( asList(five, four, three, two, one, zero), - toList(windowStore.backwardFetch( + toListAndCloseIterator(windowStore.backwardFetch( 0, 5, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), - ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE + 5L))) + ofEpochMilli(defaultStartTime - WINDOW_SIZE), + ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L))) ); assertEquals( asList(five, four, three, two), - toList(windowStore.backwardFetch( + toListAndCloseIterator(windowStore.backwardFetch( 0, 5, ofEpochMilli(defaultStartTime + 2L), - ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE + 5L))) + ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L))) ); assertEquals( Collections.emptyList(), - toList(windowStore.backwardFetch( + toListAndCloseIterator(windowStore.backwardFetch( 4, 5, ofEpochMilli(defaultStartTime + 2L), @@ -525,7 +524,7 @@ public abstract class AbstractWindowBytesStoreTest { ); assertEquals( Collections.emptyList(), - toList(windowStore.backwardFetch( + toListAndCloseIterator(windowStore.backwardFetch( 0, 3, ofEpochMilli(defaultStartTime + 3L), @@ -533,26 +532,26 @@ public abstract class AbstractWindowBytesStoreTest { ); assertEquals( asList(two, one, zero), - toList(windowStore.backwardFetch( + toListAndCloseIterator(windowStore.backwardFetch( null, 2, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), + ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime + WINDOW_SIZE + 2L))) ); assertEquals( asList(five, four, three, two), - toList(windowStore.backwardFetch( + toListAndCloseIterator(windowStore.backwardFetch( 2, null, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), + ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L))) ); assertEquals( asList(five, four, three, two, one, zero), - toList(windowStore.backwardFetch( + toListAndCloseIterator(windowStore.backwardFetch( null, null, - ofEpochMilli(defaultStartTime + 0L - WINDOW_SIZE), + ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime + WINDOW_SIZE + 5L))) ); } @@ -563,70 +562,70 @@ public abstract class AbstractWindowBytesStoreTest { assertEquals( 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( 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( 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( 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( 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( 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( 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( 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( 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( 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( 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( 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( 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( 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( 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( 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( 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( 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( 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( 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( 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 ... windowStore.flush(); @@ -652,90 +651,90 @@ public abstract class AbstractWindowBytesStoreTest { assertEquals( new HashSet<>(Collections.singletonList("zero")), - valuesToSet(windowStore.fetch(0, ofEpochMilli(defaultStartTime + 0L), - ofEpochMilli(defaultStartTime + 0L + WINDOW_SIZE)))); + valuesToSetAndCloseIterator(windowStore.fetch(0, ofEpochMilli(defaultStartTime), + ofEpochMilli(defaultStartTime + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("one")), - valuesToSet(windowStore.fetch(1, ofEpochMilli(defaultStartTime + 1L), + valuesToSetAndCloseIterator(windowStore.fetch(1, ofEpochMilli(defaultStartTime + 1L), ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("two")), - valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L), + valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L), ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch(3, ofEpochMilli(defaultStartTime + 3L), + valuesToSetAndCloseIterator(windowStore.fetch(3, ofEpochMilli(defaultStartTime + 3L), ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("four")), - valuesToSet(windowStore.fetch(4, ofEpochMilli(defaultStartTime + 4L), + valuesToSetAndCloseIterator(windowStore.fetch(4, ofEpochMilli(defaultStartTime + 4L), ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("five")), - valuesToSet(windowStore.fetch(5, ofEpochMilli(defaultStartTime + 5L), + valuesToSetAndCloseIterator(windowStore.fetch(5, ofEpochMilli(defaultStartTime + 5L), ofEpochMilli(defaultStartTime + 5L + WINDOW_SIZE)))); - putSecondBatch(windowStore, defaultStartTime, context); + putSecondBatch(windowStore, defaultStartTime); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime - 2L), + valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime - 2L), ofEpochMilli(defaultStartTime - 2L + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.singletonList("two")), - valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime - 1L), + valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime - 1L), ofEpochMilli(defaultStartTime - 1L + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("two", "two+1")), - valuesToSet(windowStore + valuesToSetAndCloseIterator(windowStore .fetch(2, ofEpochMilli(defaultStartTime), ofEpochMilli(defaultStartTime + WINDOW_SIZE)))); assertEquals( 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)))); assertEquals( 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)))); assertEquals( 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)))); assertEquals( 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)))); assertEquals( 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)))); assertEquals( 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)))); assertEquals( 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)))); assertEquals( 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)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 9L), + valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 9L), ofEpochMilli(defaultStartTime + 9L + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 10L), + valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 10L), ofEpochMilli(defaultStartTime + 10L + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 11L), + valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 11L), ofEpochMilli(defaultStartTime + 11L + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 12L), + valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 12L), ofEpochMilli(defaultStartTime + 12L + WINDOW_SIZE)))); // Flush the store and verify all current entries were properly flushed ... @@ -763,13 +762,13 @@ public abstract class AbstractWindowBytesStoreTest { public void testPutSameKeyTimestamp() { windowStore.close(); 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); assertEquals( 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)))); windowStore.put(0, "zero", defaultStartTime); @@ -778,31 +777,31 @@ public abstract class AbstractWindowBytesStoreTest { assertEquals( new HashSet<>(asList("zero", "zero", "zero+", "zero++")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 0, ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("zero", "zero", "zero+", "zero++")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 0, ofEpochMilli(defaultStartTime + 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("zero", "zero", "zero+", "zero++")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 0, ofEpochMilli(defaultStartTime + 2L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE)))); assertEquals( new HashSet<>(asList("zero", "zero", "zero+", "zero++")), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 0, ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE)))); assertEquals( new HashSet<>(Collections.emptyList()), - valuesToSet(windowStore.fetch( + valuesToSetAndCloseIterator(windowStore.fetch( 0, ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE)))); @@ -845,7 +844,7 @@ public abstract class AbstractWindowBytesStoreTest { Serdes.String(), Serdes.String()); - windowStore.init((StateStoreContext) context, windowStore); + windowStore.init(context, windowStore); windowStore.put("a", "0001", 0); windowStore.put("aa", "0002", 0); @@ -855,7 +854,7 @@ public abstract class AbstractWindowBytesStoreTest { final Set expected = new HashSet<>(asList("0001", "0003", "0005")); assertThat( - valuesToSet(windowStore.fetch("a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), + valuesToSetAndCloseIterator(windowStore.fetch("a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expected) ); @@ -887,12 +886,14 @@ public abstract class AbstractWindowBytesStoreTest { windowStore.put(1, "one", currentTime); windowStore.put(1, "one v2", currentTime); - WindowStoreIterator iterator = windowStore.fetch(1, 0, currentTime); - assertEquals(new KeyValue<>(currentTime, "one v2"), iterator.next()); + try (final WindowStoreIterator iterator = windowStore.fetch(1, 0, currentTime)) { + assertEquals(new KeyValue<>(currentTime, "one v2"), iterator.next()); + } windowStore.put(1, null, currentTime); - iterator = windowStore.fetch(1, 0, currentTime); - assertFalse(iterator.hasNext()); + try (final WindowStoreIterator iterator = windowStore.fetch(1, 0, currentTime)) { + assertFalse(iterator.hasNext()); + } } @Test @@ -905,6 +906,7 @@ public abstract class AbstractWindowBytesStoreTest { assertThrows(NullPointerException.class, () -> windowStore.put(null, "anyValue", 0L)); } + @SuppressWarnings("resource") @Test public void shouldThrowNullPointerExceptionOnGetNullKey() { assertThrows(NullPointerException.class, () -> windowStore.fetch(null, ofEpochMilli(1L), ofEpochMilli(2L))); @@ -917,7 +919,7 @@ public abstract class AbstractWindowBytesStoreTest { true, Serdes.Bytes(), Serdes.String()); - windowStore.init((StateStoreContext) context, windowStore); + windowStore.init(context, windowStore); final Bytes key1 = Bytes.wrap(new byte[] {0}); final Bytes key2 = Bytes.wrap(new byte[] {0, 0}); @@ -934,17 +936,17 @@ public abstract class AbstractWindowBytesStoreTest { final Set expectedKey1 = new HashSet<>(asList("1", "4", "7")); assertThat( - valuesToSet(windowStore.fetch(key1, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), + valuesToSetAndCloseIterator(windowStore.fetch(key1, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey1) ); final Set expectedKey2 = new HashSet<>(asList("2", "5", "8")); assertThat( - valuesToSet(windowStore.fetch(key2, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), + valuesToSetAndCloseIterator(windowStore.fetch(key2, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey2) ); final Set expectedKey3 = new HashSet<>(asList("3", "6", "9")); assertThat( - valuesToSet(windowStore.fetch(key3, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), + valuesToSetAndCloseIterator(windowStore.fetch(key3, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey3) ); @@ -990,7 +992,7 @@ public abstract class AbstractWindowBytesStoreTest { final Properties streamsConfig = StreamsTestUtils.getStreamsConfig(); final WindowStore windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, false, Serdes.Integer(), Serdes.String()); - final InternalMockProcessorContext context = new InternalMockProcessorContext( + final InternalMockProcessorContext context = new InternalMockProcessorContext<>( TestUtils.tempDirectory(), new StreamsConfig(streamsConfig), recordCollector @@ -998,7 +1000,7 @@ public abstract class AbstractWindowBytesStoreTest { final Time time = Time.SYSTEM; context.setSystemTimeMs(time.milliseconds()); 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 windowStore.put(1, "initial record", 2 * RETENTION_PERIOD); @@ -1108,7 +1110,7 @@ public abstract class AbstractWindowBytesStoreTest { public void testFetchDuplicates() { windowStore.close(); windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), Serdes.String()); - windowStore.init((StateStoreContext) context, windowStore); + windowStore.init(context, windowStore); long currentTime = 0; windowStore.put(1, "one", currentTime); @@ -1135,7 +1137,7 @@ public abstract class AbstractWindowBytesStoreTest { private void putFirstBatch(final WindowStore store, @SuppressWarnings("SameParameterValue") final long startTime, - final InternalMockProcessorContext context) { + final InternalMockProcessorContext context) { context.setRecordContext(createRecordContext(startTime)); store.put(0, "zero", startTime); store.put(1, "one", startTime + 1L); @@ -1146,8 +1148,7 @@ public abstract class AbstractWindowBytesStoreTest { } private void putSecondBatch(final WindowStore store, - @SuppressWarnings("SameParameterValue") final long startTime, - final InternalMockProcessorContext context) { + @SuppressWarnings("SameParameterValue") final long startTime) { store.put(2, "two+1", startTime + 3L); store.put(2, "two+2", startTime + 4L); store.put(2, "two+3", startTime + 5L); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemorySessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemorySessionStoreTest.java index fcf63e97863..d92bbfbdac3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemorySessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemorySessionStoreTest.java @@ -60,7 +60,7 @@ import java.util.Random; import static java.util.Arrays.asList; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.apache.kafka.test.StreamsTestUtils.toList; +import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator; import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList; import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue; import static org.hamcrest.CoreMatchers.hasItem; @@ -401,7 +401,7 @@ public class CachingInMemorySessionStoreTest { // add one that shouldn't appear in the results cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "5".getBytes()); - final List, byte[]>> results = toList(cachingStore.fetch(keyA)); + final List, byte[]>> results = toListAndCloseIterator(cachingStore.fetch(keyA)); verifyKeyValueList(expected, results); } @@ -420,7 +420,7 @@ public class CachingInMemorySessionStoreTest { // add one that shouldn't appear in the results cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "5".getBytes()); - final List, byte[]>> results = toList(cachingStore.backwardFetch(keyA)); + final List, byte[]>> results = toListAndCloseIterator(cachingStore.backwardFetch(keyA)); Collections.reverse(results); verifyKeyValueList(expected, results); } @@ -439,7 +439,7 @@ public class CachingInMemorySessionStoreTest { @Test public void shouldQueryItemsInCacheAndStore() { final List, byte[]>> added = addSessionsUntilOverflow("a"); - final List, byte[]>> actual = toList(cachingStore.findSessions( + final List, byte[]>> actual = toListAndCloseIterator(cachingStore.findSessions( Bytes.wrap("a".getBytes(StandardCharsets.UTF_8)), 0, added.size() * 10L)); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java index b267c622440..bfe47e9d4d6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java @@ -60,7 +60,7 @@ import java.util.Random; import static java.util.Arrays.asList; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.apache.kafka.test.StreamsTestUtils.toList; +import static org.apache.kafka.test.StreamsTestUtils.toListAndCloseIterator; import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList; import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue; import static org.hamcrest.CoreMatchers.hasItem; @@ -409,7 +409,7 @@ public class CachingPersistentSessionStoreTest { // add one that shouldn't appear in the results cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "5".getBytes()); - final List, byte[]>> results = toList(cachingStore.fetch(keyA)); + final List, byte[]>> results = toListAndCloseIterator(cachingStore.fetch(keyA)); verifyKeyValueList(expected, results); } @@ -428,7 +428,7 @@ public class CachingPersistentSessionStoreTest { // add one that shouldn't appear in the results cachingStore.put(new Windowed<>(keyAA, new SessionWindow(0, 0)), "5".getBytes()); - final List, byte[]>> results = toList(cachingStore.backwardFetch(keyA)); + final List, byte[]>> results = toListAndCloseIterator(cachingStore.backwardFetch(keyA)); Collections.reverse(results); verifyKeyValueList(expected, results); } @@ -448,7 +448,7 @@ public class CachingPersistentSessionStoreTest { @Test public void shouldQueryItemsInCacheAndStore() { final List, byte[]>> added = addSessionsUntilOverflow("a"); - final List, byte[]>> actual = toList(cachingStore.findSessions( + final List, byte[]>> actual = toListAndCloseIterator(cachingStore.findSessions( Bytes.wrap("a".getBytes(StandardCharsets.UTF_8)), 0, added.size() * 10L diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java index 5735e2f7f4e..4363e25c0da 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java @@ -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.mkMap; 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.verifyKeyValueList; import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue; @@ -844,7 +844,7 @@ public class CachingPersistentWindowStoreTest { KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")) ); final List> 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); } @@ -862,7 +862,7 @@ public class CachingPersistentWindowStoreTest { KeyValue.pair(0L, bytesValue("0001")) ); final List> 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); } @@ -880,14 +880,14 @@ public class CachingPersistentWindowStoreTest { windowedPair("a", "0003", 1), 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( asList( windowedPair("aa", "0002", 0), 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( @@ -898,7 +898,7 @@ public class CachingPersistentWindowStoreTest { windowedPair("aa", "0004", 1), 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", "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( asList( windowedPair("aa", "0004", 1), 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( @@ -934,7 +934,7 @@ public class CachingPersistentWindowStoreTest { windowedPair("a", "0003", 1), 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))) ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java index 3af926a3aa4..5480ab7ab92 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java @@ -43,7 +43,7 @@ import java.util.NoSuchElementException; import static java.util.Arrays.asList; 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.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; @@ -276,7 +276,7 @@ public class CompositeReadOnlyKeyValueStoreTest { stubOneUnderlying.put("b", "b"); stubOneUnderlying.put("c", "c"); - final List> results = toList(theStore.range("a", "b")); + final List> results = toListAndCloseIterator(theStore.range("a", "b")); assertTrue(results.contains(new KeyValue<>("a", "a"))); assertTrue(results.contains(new KeyValue<>("b", "b"))); assertEquals(2, results.size()); @@ -288,7 +288,7 @@ public class CompositeReadOnlyKeyValueStoreTest { stubOneUnderlying.put("b", "b"); stubOneUnderlying.put("c", "c"); - final List> results = toList(theStore.reverseRange("a", "b")); + final List> results = toListAndCloseIterator(theStore.reverseRange("a", "b")); assertArrayEquals( asList( new KeyValue<>("b", "b"), @@ -303,7 +303,7 @@ public class CompositeReadOnlyKeyValueStoreTest { stubOneUnderlying.put("abcd", "b"); stubOneUnderlying.put("abce", "c"); - final List> results = toList(theStore.prefixScan("abcd", new StringSerializer())); + final List> results = toListAndCloseIterator(theStore.prefixScan("abcd", new StringSerializer())); assertTrue(results.contains(new KeyValue<>("abcd", "b"))); assertEquals(1, results.size()); } @@ -314,7 +314,7 @@ public class CompositeReadOnlyKeyValueStoreTest { stubOneUnderlying.put("aa", "b"); stubOneUnderlying.put("b", "c"); - final List> results = toList(theStore.prefixScan("a", new StringSerializer())); + final List> results = toListAndCloseIterator(theStore.prefixScan("a", new StringSerializer())); assertTrue(results.contains(new KeyValue<>("a", "a"))); assertTrue(results.contains(new KeyValue<>("aa", "b"))); assertEquals(2, results.size()); @@ -333,7 +333,7 @@ public class CompositeReadOnlyKeyValueStoreTest { cache.put("d", "d"); cache.put("x", "x"); - final List> results = toList(theStore.range("a", "e")); + final List> results = toListAndCloseIterator(theStore.range("a", "e")); assertArrayEquals( asList( new KeyValue<>("a", "a"), @@ -357,7 +357,7 @@ public class CompositeReadOnlyKeyValueStoreTest { cache.put("ab", "d"); cache.put("x", "x"); - final List> results = toList(theStore.prefixScan("a", new StringSerializer())); + final List> results = toListAndCloseIterator(theStore.prefixScan("a", new StringSerializer())); assertArrayEquals( asList( new KeyValue<>("a", "a"), @@ -380,7 +380,7 @@ public class CompositeReadOnlyKeyValueStoreTest { cache.put("d", "d"); cache.put("x", "x"); - final List> results = toList(theStore.reverseRange("a", "e")); + final List> results = toListAndCloseIterator(theStore.reverseRange("a", "e")); assertTrue(results.contains(new KeyValue<>("a", "a"))); assertTrue(results.contains(new KeyValue<>("b", "b"))); assertTrue(results.contains(new KeyValue<>("c", "c"))); @@ -401,7 +401,7 @@ public class CompositeReadOnlyKeyValueStoreTest { cache.put("d", "d"); cache.put("x", "x"); - final List> results = toList(theStore.all()); + final List> results = toListAndCloseIterator(theStore.all()); assertTrue(results.contains(new KeyValue<>("a", "a"))); assertTrue(results.contains(new KeyValue<>("b", "b"))); assertTrue(results.contains(new KeyValue<>("c", "c"))); @@ -424,7 +424,7 @@ public class CompositeReadOnlyKeyValueStoreTest { cache.put("d", "d"); cache.put("x", "x"); - final List> results = toList(theStore.reverseAll()); + final List> results = toListAndCloseIterator(theStore.reverseAll()); assertTrue(results.contains(new KeyValue<>("a", "a"))); assertTrue(results.contains(new KeyValue<>("b", "b"))); assertTrue(results.contains(new KeyValue<>("c", "c"))); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java index 6e4e42ae7ec..59d6ad1e175 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java @@ -36,7 +36,7 @@ import java.util.Arrays; import java.util.List; 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.core.IsEqual.equalTo; 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(10, 10)), 2L); - final List, Long>> results = toList(sessionStore.fetch("a")); + final List, Long>> results = toListAndCloseIterator(sessionStore.fetch("a")); assertEquals(Arrays.asList(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 1L), KeyValue.pair(new Windowed<>("a", new SessionWindow(10, 10)), 2L)), results); @@ -93,8 +93,8 @@ public class CompositeReadOnlySessionStoreTest { underlyingSessionStore.put(keyOne, 0L); secondUnderlying.put(keyTwo, 10L); - final List, Long>> keyOneResults = toList(sessionStore.fetch("key-one")); - final List, Long>> keyTwoResults = toList(sessionStore.fetch("key-two")); + final List, Long>> keyOneResults = toListAndCloseIterator(sessionStore.fetch("key-one")); + final List, Long>> keyTwoResults = toListAndCloseIterator(sessionStore.fetch("key-two")); assertEquals(singletonList(KeyValue.pair(keyOne, 0L)), keyOneResults); assertEquals(singletonList(KeyValue.pair(keyTwo, 10L)), keyTwoResults); @@ -146,7 +146,7 @@ public class CompositeReadOnlySessionStoreTest { stubProviderTwo.addStore(storeName, secondUnderlying); underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 0L); secondUnderlying.put(new Windowed<>("b", new SessionWindow(0, 0)), 10L); - final List, Long>> results = StreamsTestUtils.toList(sessionStore.fetch("a", "b")); + final List, Long>> results = StreamsTestUtils.toListAndCloseIterator(sessionStore.fetch("a", "b")); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 0L), KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), 10L)))); @@ -159,7 +159,7 @@ public class CompositeReadOnlySessionStoreTest { stubProviderTwo.addStore(storeName, secondUnderlying); underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 0L); secondUnderlying.put(new Windowed<>("b", new SessionWindow(0, 0)), 10L); - final List, Long>> results = StreamsTestUtils.toList(sessionStore.fetch(null, "b")); + final List, Long>> results = StreamsTestUtils.toListAndCloseIterator(sessionStore.fetch(null, "b")); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 0L), KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), 10L)))); @@ -172,7 +172,7 @@ public class CompositeReadOnlySessionStoreTest { stubProviderTwo.addStore(storeName, secondUnderlying); underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 0L); secondUnderlying.put(new Windowed<>("b", new SessionWindow(0, 0)), 10L); - final List, Long>> results = StreamsTestUtils.toList(sessionStore.fetch("a", null)); + final List, Long>> results = StreamsTestUtils.toListAndCloseIterator(sessionStore.fetch("a", null)); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 0L), KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), 10L)))); @@ -185,7 +185,7 @@ public class CompositeReadOnlySessionStoreTest { stubProviderTwo.addStore(storeName, secondUnderlying); underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 0L); secondUnderlying.put(new Windowed<>("b", new SessionWindow(0, 0)), 10L); - final List, Long>> results = StreamsTestUtils.toList(sessionStore.fetch(null, null)); + final List, Long>> results = StreamsTestUtils.toListAndCloseIterator(sessionStore.fetch(null, null)); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 0L), KeyValue.pair(new Windowed<>("b", new SessionWindow(0, 0)), 10L)))); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java index 18e775033f0..2d22e6e15a7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java @@ -90,7 +90,7 @@ public class CompositeReadOnlyWindowStoreTest { assertEquals( 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( 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); final List> keyOneResults = - StreamsTestUtils.toList(windowStore.fetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L))); final List> 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(10L, "value-two")), keyTwoResults); @@ -150,9 +150,9 @@ public class CompositeReadOnlyWindowStoreTest { secondUnderlying.put("key-two", "value-two", 10L); final List> keyOneResults = - StreamsTestUtils.toList(windowStore.backwardFetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L))); final List> 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(10L, "value-two")), keyTwoResults); @@ -164,7 +164,7 @@ public class CompositeReadOnlyWindowStoreTest { underlyingWindowStore.put("some-key", "my-value", 1L); final List> 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); } @@ -174,7 +174,7 @@ public class CompositeReadOnlyWindowStoreTest { underlyingWindowStore.put("some-key", "my-value", 1L); final List> 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); } @@ -348,7 +348,7 @@ public class CompositeReadOnlyWindowStoreTest { underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); final List, 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( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), 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("c", "c", 10L); final List, 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( KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"), 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("c", "c", 10L); final List, 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( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), 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("c", "c", 10L); final List, 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( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), 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("c", "c", 10L); final List, 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( KeyValue.pair(new Windowed<>("c", new TimeWindow(10, 10 + WINDOW_SIZE)), "c"), 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("c", "c", 10L); final List, 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( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), 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("c", "c", 10L); final List, 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( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("c", new TimeWindow(10, 10 + WINDOW_SIZE)), "c"), @@ -448,7 +448,7 @@ public class CompositeReadOnlyWindowStoreTest { underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); final List, 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( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); @@ -473,7 +473,7 @@ public class CompositeReadOnlyWindowStoreTest { stubProviderTwo.addStore(storeName, secondUnderlying); underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); - final List, String>> results = StreamsTestUtils.toList(windowStore.all()); + final List, String>> results = StreamsTestUtils.toListAndCloseIterator(windowStore.all()); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); @@ -486,7 +486,7 @@ public class CompositeReadOnlyWindowStoreTest { stubProviderTwo.addStore(storeName, secondUnderlying); underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); - final List, String>> results = StreamsTestUtils.toList(windowStore.backwardAll()); + final List, String>> results = StreamsTestUtils.toListAndCloseIterator(windowStore.backwardAll()); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); @@ -500,7 +500,7 @@ public class CompositeReadOnlyWindowStoreTest { underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); final List, String>> results = - StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(0), ofEpochMilli(10))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetchAll(ofEpochMilli(0), ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); @@ -514,7 +514,7 @@ public class CompositeReadOnlyWindowStoreTest { underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); final List, String>> results = - StreamsTestUtils.toList(windowStore.backwardFetchAll(ofEpochMilli(0), ofEpochMilli(10))); + StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetchAll(ofEpochMilli(0), ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java index dd0d8bbf4a1..9c8fd2ce5f3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java @@ -30,7 +30,7 @@ import java.util.Collections; import java.util.List; 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.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -91,7 +91,7 @@ public class FilteredCacheIteratorTest { @Test public void shouldAllowEntryMatchingHasNextCondition() { - final List> keyValues = toList(allIterator); + final List> keyValues = toListAndCloseIterator(allIterator); assertThat(keyValues, equalTo(entries)); } @@ -122,7 +122,7 @@ public class FilteredCacheIteratorTest { @Test public void shouldFilterEntriesNotMatchingHasNextCondition() { - final List> keyValues = toList(firstEntryIterator); + final List> keyValues = toListAndCloseIterator(firstEntryIterator); assertThat(keyValues, equalTo(Collections.singletonList(firstEntry))); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemorySessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemorySessionStoreTest.java index fd93122d09b..1775d76e101 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemorySessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemorySessionStoreTest.java @@ -52,7 +52,11 @@ public class InMemorySessionStoreTest extends AbstractSessionBytesStoreTest { assertFalse(iterator.hasNext()); iterator.close(); - assertFalse(sessionStore.findSessions("a", "b", 0L, 20L).hasNext()); + + try (final KeyValueIterator, Long> it = + sessionStore.findSessions("a", "b", 0L, 20L)) { + assertFalse(it.hasNext()); + } } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ListValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ListValueStoreTest.java index d0a5c487aa2..b6e2578ee9f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ListValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ListValueStoreTest.java @@ -40,7 +40,7 @@ import java.io.File; import java.util.Collections; 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.assertThrows; @@ -103,7 +103,7 @@ public class ListValueStoreTest { assertEquals( asList(zero, zeroAgain, one, two), - toList(listStore.all()) + toListAndCloseIterator(listStore.all()) ); } @@ -130,7 +130,7 @@ public class ListValueStoreTest { assertEquals( asList(zero, two, four), - toList(listStore.all()) + toListAndCloseIterator(listStore.all()) ); } @@ -158,7 +158,7 @@ public class ListValueStoreTest { assertEquals( asList(zero, one, two1, two2, three, four), - toList(listStore.all()) + toListAndCloseIterator(listStore.all()) ); } @@ -185,7 +185,7 @@ public class ListValueStoreTest { it.close(); // 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 diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java index fbe6f1b73e9..e788c0f7426 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java @@ -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.mkMap; 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.verifyKeyValueList; import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue; @@ -945,7 +945,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest { windowedPair("a", "0001", 1), 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))) ); } else { @@ -954,7 +954,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest { windowedPair("aa", "0002", 0), 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))) ); } @@ -976,7 +976,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest { KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")) ); final List> 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); } @@ -996,7 +996,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest { KeyValue.pair(0L, bytesValue("0001")) ); final List> 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); } @@ -1016,14 +1016,14 @@ public class TimeOrderedCachingPersistentWindowStoreTest { windowedPair("a", "0003", 1), 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( asList( windowedPair("aa", "0002", 0), 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) { @@ -1035,7 +1035,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest { windowedPair("aa", "0004", 1), 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))) ); } else { @@ -1047,7 +1047,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest { windowedPair("aa", "0004", 1), 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))) ); } @@ -1069,14 +1069,14 @@ public class TimeOrderedCachingPersistentWindowStoreTest { windowedPair("a", "0003", 1), 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( asList( windowedPair("aa", "0004", 1), 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) { @@ -1089,7 +1089,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest { windowedPair("aa", "0002", 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))) ); } else { @@ -1102,7 +1102,7 @@ public class TimeOrderedCachingPersistentWindowStoreTest { windowedPair("a", "0003", 1), 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))) ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreTest.java index 856649d9503..a6440db60fc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreTest.java @@ -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.mkMap; 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.verifyKeyValueList; import static org.apache.kafka.test.StreamsTestUtils.verifyWindowedKeyValue; @@ -951,7 +951,7 @@ public class TimeOrderedWindowStoreTest { windowedPair("a", "0001", 1), 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))) ); } else { @@ -960,7 +960,7 @@ public class TimeOrderedWindowStoreTest { windowedPair("aa", "0002", 0), 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))) ); } @@ -982,7 +982,7 @@ public class TimeOrderedWindowStoreTest { KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")) ); final List> 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); } @@ -1002,7 +1002,7 @@ public class TimeOrderedWindowStoreTest { KeyValue.pair(0L, bytesValue("0001")) ); final List> 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); } @@ -1022,14 +1022,14 @@ public class TimeOrderedWindowStoreTest { windowedPair("a", "0003", 1), 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( asList( windowedPair("aa", "0002", 0), 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) { @@ -1041,7 +1041,7 @@ public class TimeOrderedWindowStoreTest { windowedPair("aa", "0004", 1), 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))) ); } else { @@ -1053,7 +1053,7 @@ public class TimeOrderedWindowStoreTest { windowedPair("aa", "0004", 1), 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))) ); } @@ -1075,14 +1075,14 @@ public class TimeOrderedWindowStoreTest { windowedPair("a", "0003", 1), 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( asList( windowedPair("aa", "0004", 1), 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) { @@ -1095,7 +1095,7 @@ public class TimeOrderedWindowStoreTest { windowedPair("aa", "0002", 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))) ); } else { @@ -1108,7 +1108,7 @@ public class TimeOrderedWindowStoreTest { windowedPair("a", "0003", 1), 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))) ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java index 8f73e94274f..b170c750d7a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java @@ -126,7 +126,7 @@ public class WindowKeySchemaTest { private final Window window = new TimeWindow(startTime, endTime); private final Windowed windowedKey = new Windowed<>(key, window); private KeySchema keySchema; - private final Serde> keySerde = new WindowedSerdes.TimeWindowedSerde<>(serde, Long.MAX_VALUE); + private final Serde> keySerde = new WindowedSerdes.TimeWindowedSerde<>(serde, endTime - startTime); private final StateSerdes stateSerdes = new StateSerdes<>("dummy", serde, Serdes.ByteArray()); public SchemaType schemaType; @@ -401,7 +401,7 @@ public class WindowKeySchemaTest { final byte[] bytes = keySerde.serializer().serialize(topic, windowedKey); final Windowed result = keySerde.deserializer().deserialize(topic, bytes); // 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) diff --git a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java index 6fb19fd5c72..6833f023cbd 100644 --- a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java @@ -35,8 +35,6 @@ import org.apache.kafka.streams.state.KeyValueIterator; import org.mockito.quality.Strictness; -import java.io.Closeable; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -116,20 +114,16 @@ public final class StreamsTestUtils { return getStreamsConfig(UUID.randomUUID().toString()); } - public static List> toList(final Iterator> iterator) { - final List> results = new ArrayList<>(); + public static List> toListAndCloseIterator(final KeyValueIterator iterator) { + try (iterator) { + final List> results = new ArrayList<>(); - while (iterator.hasNext()) { - results.add(iterator.next()); + while (iterator.hasNext()) { + results.add(iterator.next()); + } + + return results; } - - if (iterator instanceof Closeable) { - try { - ((Closeable) iterator).close(); - } catch (IOException e) { /* do nothing */ } - } - - return results; } public static Set> toSet(final Iterator> iterator) { @@ -141,7 +135,7 @@ public final class StreamsTestUtils { return results; } - public static Set valuesToSet(final Iterator> iterator) { + public static Set valuesToSet(final KeyValueIterator iterator) { final Set results = new HashSet<>(); while (iterator.hasNext()) { @@ -150,6 +144,12 @@ public final class StreamsTestUtils { return results; } + public static Set valuesToSetAndCloseIterator(final KeyValueIterator iterator) { + try (iterator) { + return valuesToSet(iterator); + } + } + public static void verifyKeyValueList(final List> expected, final List> actual) { assertThat(actual.size(), equalTo(expected.size())); 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)} )}. - * @return true if the stack context is within a {@link org.apache.kafka.streams.internals.ApiUtils#checkSupplier(Supplier)} )} call + * 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 */ public static boolean isCheckSupplierCall() { return Arrays.stream(Thread.currentThread().getStackTrace())