From ca641b3e2e48c14ff308181c775775408f5f35f7 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Thu, 4 Oct 2018 23:51:39 +0300 Subject: [PATCH] KAFKA-7277: Migrate Streams API to Duration instead of longMs times (#5682) Reviewers: Johne Roesler , Matthias J. Sax , Bill Bejeck , Guozhang Wang --- .../examples/pageview/PageViewTypedDemo.java | 4 +- .../pageview/PageViewUntypedDemo.java | 3 +- .../examples/temperature/TemperatureDemo.java | 4 +- .../wordcount/WordCountProcessorDemo.java | 3 +- .../apache/kafka/streams/KafkaStreams.java | 20 ++ .../kafka/streams/internals/ApiUtils.java | 60 ++++ .../kafka/streams/kstream/JoinWindows.java | 70 ++++- .../kafka/streams/kstream/Materialized.java | 9 +- .../kafka/streams/kstream/SessionWindows.java | 26 +- .../kafka/streams/kstream/TimeWindows.java | 53 +++- .../kafka/streams/kstream/Transformer.java | 5 +- .../streams/kstream/UnlimitedWindows.java | 16 + .../streams/kstream/ValueTransformer.java | 5 +- .../kstream/ValueTransformerWithKey.java | 7 +- .../apache/kafka/streams/kstream/Window.java | 25 ++ .../kstream/internals/KStreamImpl.java | 5 +- .../internals/TimeWindowedKStreamImpl.java | 5 +- .../kafka/streams/processor/Cancellable.java | 4 +- .../kafka/streams/processor/Processor.java | 3 +- .../streams/processor/ProcessorContext.java | 47 ++- .../streams/processor/PunctuationType.java | 4 +- .../kafka/streams/processor/Punctuator.java | 4 +- .../ForwardingDisabledProcessorContext.java | 9 + .../internals/GlobalProcessorContextImpl.java | 10 + .../internals/ProcessorContextImpl.java | 11 + .../internals/StandbyContextImpl.java | 10 + .../streams/state/ReadOnlyWindowStore.java | 83 ++++- .../apache/kafka/streams/state/Stores.java | 44 +++ .../kafka/streams/state/WindowStore.java | 64 ++++ .../streams/state/WindowStoreIterator.java | 4 +- .../state/internals/CachingWindowStore.java | 32 +- .../ChangeLoggingWindowBytesStore.java | 28 +- .../CompositeReadOnlyWindowStore.java | 27 +- .../state/internals/MeteredWindowStore.java | 23 ++ .../state/internals/RocksDBWindowStore.java | 23 ++ .../kafka/streams/KafkaStreamsTest.java | 3 +- .../apache/kafka/streams/TopologyTest.java | 13 +- .../AbstractResetIntegrationTest.java | 6 +- .../GlobalThreadShutDownOrderTest.java | 4 +- .../InternalTopicIntegrationTest.java | 6 +- ...StreamAggregationDedupIntegrationTest.java | 6 +- .../KStreamAggregationIntegrationTest.java | 14 +- ...ableSourceTopicRestartIntegrationTest.java | 8 +- .../PurgeRepartitionTopicIntegrationTest.java | 4 +- .../QueryableStateIntegrationTest.java | 15 +- .../RepartitionOptimizingIntegrationTest.java | 9 +- ...ionWithMergeOptimizingIntegrationTest.java | 6 +- .../integration/RestoreIntegrationTest.java | 3 +- .../StreamStreamJoinIntegrationTest.java | 17 +- .../SuppressionIntegrationTest.java | 4 +- .../streams/kstream/JoinWindowsTest.java | 87 +++--- .../streams/kstream/SessionWindowsTest.java | 41 +-- .../streams/kstream/TimeWindowsTest.java | 77 ++--- .../streams/kstream/UnlimitedWindowsTest.java | 17 +- .../internals/KGroupedStreamImplTest.java | 45 +-- .../kstream/internals/KStreamImplTest.java | 17 +- .../internals/KStreamKStreamJoinTest.java | 13 +- .../internals/KStreamKStreamLeftJoinTest.java | 5 +- ...amSessionWindowAggregateProcessorTest.java | 9 +- .../internals/KStreamTransformTest.java | 3 +- .../internals/KStreamWindowAggregateTest.java | 13 +- .../internals/KStreamWindowReduceTest.java | 5 +- .../SessionWindowedKStreamImplTest.java | 5 +- .../internals/SuppressScenarioTest.java | 6 +- .../internals/SuppressTopologyTest.java | 5 +- .../kstream/internals/TimeWindowTest.java | 5 +- .../TimeWindowedKStreamImplTest.java | 12 +- .../graph/GraphGraceSearchUtilTest.java | 13 +- .../internals/graph/StreamsGraphTest.java | 7 +- .../kafka/streams/perf/SimpleBenchmark.java | 17 +- .../kafka/streams/perf/YahooBenchmark.java | 3 +- .../AbstractProcessorContextTest.java | 8 + .../InternalTopologyBuilderTest.java | 5 +- .../processor/internals/StandbyTaskTest.java | 9 +- .../processor/internals/StreamThreadTest.java | 5 +- .../StreamsPartitionAssignorTest.java | 3 +- .../kafka/streams/state/NoOpWindowStore.java | 24 +- .../kafka/streams/state/StoresTest.java | 18 +- .../internals/CachingWindowStoreTest.java | 43 +-- .../ChangeLoggingWindowBytesStoreTest.java | 7 +- .../CompositeReadOnlyWindowStoreTest.java | 37 +-- .../internals/MeteredWindowStoreTest.java | 7 +- .../internals/ReadOnlyWindowStoreStub.java | 25 ++ .../internals/RocksDBWindowStoreTest.java | 288 +++++++++--------- .../StreamThreadStateStoreProviderTest.java | 2 +- .../tests/BrokerCompatibilityTest.java | 3 +- .../kafka/streams/tests/EosTestClient.java | 5 +- .../streams/tests/ShutdownDeadlockTest.java | 4 +- .../kafka/streams/tests/SmokeTestClient.java | 14 +- .../StreamsBrokerDownResilienceTest.java | 6 +- .../tests/StreamsStandByReplicaTest.java | 4 +- .../test/InternalMockProcessorContext.java | 8 + .../org/apache/kafka/test/MockProcessor.java | 3 +- .../kafka/test/NoOpProcessorContext.java | 11 +- .../streams/scala/kstream/KStreamTest.scala | 4 +- .../scala/kstream/MaterializedTest.scala | 6 +- .../kafka/streams/TopologyTestDriver.java | 4 +- .../processor/MockProcessorContext.java | 11 + .../streams/MockProcessorContextTest.java | 3 +- .../kafka/streams/TopologyTestDriverTest.java | 7 +- 100 files changed, 1311 insertions(+), 536 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/internals/ApiUtils.java diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index 871d83606f17b..f8af99f691138 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -19,6 +19,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.databind.ObjectMapper; +import java.time.Duration; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; @@ -39,7 +40,6 @@ import java.util.Map; import java.util.Properties; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; /** * Demonstrates how to perform a join between a KStream and a KTable, i.e. an example of a stateful computation, @@ -207,7 +207,7 @@ public static void main(final String[] args) { }) .map((user, viewRegion) -> new KeyValue<>(viewRegion.region, viewRegion)) .groupByKey(Serialized.with(Serdes.String(), new JSONSerde<>())) - .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(7)).advanceBy(TimeUnit.SECONDS.toMillis(1))) + .windowedBy(TimeWindows.of(Duration.ofDays(7)).advanceBy(Duration.ofSeconds(1))) .count() .toStream() .map((key, value) -> { diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index e2b7c377682f4..25b10e8fa067c 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -19,6 +19,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.JsonNodeFactory; import com.fasterxml.jackson.databind.node.ObjectNode; +import java.time.Duration; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; @@ -87,7 +88,7 @@ public static void main(final String[] args) throws Exception { }) .map((user, viewRegion) -> new KeyValue<>(viewRegion.get("region").textValue(), viewRegion)) .groupByKey(Serialized.with(Serdes.String(), jsonSerde)) - .windowedBy(TimeWindows.of(7 * 24 * 60 * 60 * 1000L).advanceBy(1000)) + .windowedBy(TimeWindows.of(Duration.ofDays(7)).advanceBy(Duration.ofSeconds(1))) .count() .toStream() .map((key, value) -> { diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java index c1c7a427caf2c..93480e40d6b12 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.examples.temperature; +import java.time.Duration; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; @@ -30,7 +31,6 @@ import java.util.Properties; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; /** * Demonstrates, using the high-level KStream DSL, how to implement an IoT demo application @@ -88,7 +88,7 @@ public static void main(final String[] args) { // to group and reduce them, a key is needed ("temp" has been chosen) .selectKey((key, value) -> "temp") .groupByKey() - .windowedBy(TimeWindows.of(TimeUnit.SECONDS.toMillis(TEMPERATURE_WINDOW_SIZE))) + .windowedBy(TimeWindows.of(Duration.ofSeconds(TEMPERATURE_WINDOW_SIZE))) .reduce((value1, value2) -> { if (Integer.parseInt(value1) > Integer.parseInt(value2)) return value1; diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java index 4fa38dcf5374f..6b496415f6538 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.examples.wordcount; +import java.time.Duration; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KafkaStreams; @@ -63,7 +64,7 @@ public Processor get() { @SuppressWarnings("unchecked") public void init(final ProcessorContext context) { this.context = context; - this.context.schedule(1000, PunctuationType.STREAM_TIME, timestamp -> { + this.context.schedule(Duration.ofSeconds(1), PunctuationType.STREAM_TIME, timestamp -> { try (final KeyValueIterator iter = kvStore.all()) { System.out.println("----------- " + timestamp + " ----------- "); diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index da1a1c4f4f2c2..5fb89598507c8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams; +import java.time.Duration; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; @@ -36,6 +37,7 @@ import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.Produced; @@ -827,7 +829,9 @@ public void close() { * @return {@code true} if all threads were successfully stopped—{@code false} if the timeout was reached * before all threads stopped * Note that this method must not be called in the {@code onChange} callback of {@link StateListener}. + * @deprecated Use {@link #close(Duration)} instead */ + @Deprecated public synchronized boolean close(final long timeout, final TimeUnit timeUnit) { log.debug("Stopping Streams client with timeoutMillis = {} ms.", timeUnit.toMillis(timeout)); @@ -895,6 +899,22 @@ public void run() { } } + /** + * Shutdown this {@code KafkaStreams} by signaling all the threads to stop, and then wait up to the timeout for the + * threads to join. + * A {@code timeout} of 0 means to wait forever. + * + * @param timeout how long to wait for the threads to shutdown + * @return {@code true} if all threads were successfully stopped—{@code false} if the timeout was reached + * before all threads stopped + * Note that this method must not be called in the {@link StateListener#onChange(State, State)} callback of {@link StateListener}. + * @throws IllegalArgumentException if {@code timeout} can't be represented as {@code long milliseconds} + */ + public synchronized boolean close(final Duration timeout) throws IllegalArgumentException { + ApiUtils.validateMillisecondDuration(timeout, "timeout"); + return close(timeout.toMillis(), TimeUnit.MILLISECONDS); + } + /** * Do a clean up of the local {@link StateStore} directory ({@link StreamsConfig#STATE_DIR_CONFIG}) by deleting all * data with regard to the {@link StreamsConfig#APPLICATION_ID_CONFIG application ID}. diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/ApiUtils.java b/streams/src/main/java/org/apache/kafka/streams/internals/ApiUtils.java new file mode 100644 index 0000000000000..e888d7a120bb3 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/internals/ApiUtils.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.internals; + +import java.time.Duration; +import java.time.Instant; +import java.util.Objects; + +public final class ApiUtils { + private ApiUtils() { + } + + /** + * Validates that milliseconds from {@code duration} can be retrieved. + * @param duration Duration to check. + * @param name Name of params for an error message. + * @return Milliseconds from {@code duration}. + */ + public static long validateMillisecondDuration(final Duration duration, final String name) { + try { + if (duration == null) + throw new IllegalArgumentException("[" + Objects.toString(name) + "] shouldn't be null."); + + return duration.toMillis(); + } catch (final ArithmeticException e) { + throw new IllegalArgumentException("[" + name + "] can't be converted to milliseconds. ", e); + } + } + + /** + * Validates that milliseconds from {@code instant} can be retrieved. + * @param instant Instant to check. + * @param name Name of params for an error message. + * @return Milliseconds from {@code instant}. + */ + public static long validateMillisecondInstant(final Instant instant, final String name) { + try { + if (instant == null) + throw new IllegalArgumentException("[" + name + "] shouldn't be null."); + + return instant.toEpochMilli(); + } catch (final ArithmeticException e) { + throw new IllegalArgumentException("[" + name + "] can't be converted to milliseconds. ", e); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java index 5e742e1a25e0d..62eade4298a32 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.processor.TimestampExtractor; import java.time.Duration; @@ -110,42 +111,93 @@ private JoinWindows(final long beforeMs, * * @param timeDifferenceMs join window interval in milliseconds * @throws IllegalArgumentException if {@code timeDifferenceMs} is negative + * @deprecated Use {@link #of(Duration)} instead. */ + @Deprecated public static JoinWindows of(final long timeDifferenceMs) throws IllegalArgumentException { // This is a static factory method, so we initialize grace and retention to the defaults. return new JoinWindows(timeDifferenceMs, timeDifferenceMs, null, DEFAULT_RETENTION_MS); } + /** + * Specifies that records of the same key are joinable if their timestamps are within {@code timeDifference}, + * i.e., the timestamp of a record from the secondary stream is max {@code timeDifference} earlier or later than + * the timestamp of the record from the primary stream. + * + * @param timeDifference join window interval + * @throws IllegalArgumentException if {@code timeDifference} is negative or can't be represented as {@code long milliseconds} + */ + public static JoinWindows of(final Duration timeDifference) throws IllegalArgumentException { + ApiUtils.validateMillisecondDuration(timeDifference, "timeDifference"); + return of(timeDifference.toMillis()); + } + /** * Changes the start window boundary to {@code timeDifferenceMs} but keep the end window boundary as is. * Thus, records of the same key are joinable if the timestamp of a record from the secondary stream is at most * {@code timeDifferenceMs} earlier than the timestamp of the record from the primary stream. - * {@code timeDifferenceMs} can be negative but it's absolute value must not be larger than current window "after" + * {@code timeDifferenceMs} can be negative but its absolute value must not be larger than current window "after" * value (which would result in a negative window size). * * @param timeDifferenceMs relative window start time in milliseconds * @throws IllegalArgumentException if the resulting window size is negative + * @deprecated Use {@link #before(Duration)} instead. */ @SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this + @Deprecated public JoinWindows before(final long timeDifferenceMs) throws IllegalArgumentException { return new JoinWindows(timeDifferenceMs, afterMs, grace, maintainDurationMs, segments); } + /** + * Changes the start window boundary to {@code timeDifference} but keep the end window boundary as is. + * Thus, records of the same key are joinable if the timestamp of a record from the secondary stream is at most + * {@code timeDifference} earlier than the timestamp of the record from the primary stream. + * {@code timeDifference} can be negative but its absolute value must not be larger than current window "after" + * value (which would result in a negative window size). + * + * @param timeDifference relative window start time + * @throws IllegalArgumentException if the resulting window size is negative or {@code timeDifference} can't be represented as {@code long milliseconds} + */ + @SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this + public JoinWindows before(final Duration timeDifference) throws IllegalArgumentException { + ApiUtils.validateMillisecondDuration(timeDifference, "timeDifference"); + return before(timeDifference.toMillis()); + } + /** * Changes the end window boundary to {@code timeDifferenceMs} but keep the start window boundary as is. * Thus, records of the same key are joinable if the timestamp of a record from the secondary stream is at most * {@code timeDifferenceMs} later than the timestamp of the record from the primary stream. - * {@code timeDifferenceMs} can be negative but it's absolute value must not be larger than current window "before" + * {@code timeDifferenceMs} can be negative but its absolute value must not be larger than current window "before" * value (which would result in a negative window size). * * @param timeDifferenceMs relative window end time in milliseconds * @throws IllegalArgumentException if the resulting window size is negative + * @deprecated Use {@link #after(Duration)} instead */ @SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this + @Deprecated public JoinWindows after(final long timeDifferenceMs) throws IllegalArgumentException { return new JoinWindows(beforeMs, timeDifferenceMs, grace, maintainDurationMs, segments); } + /** + * Changes the end window boundary to {@code timeDifference} but keep the start window boundary as is. + * Thus, records of the same key are joinable if the timestamp of a record from the secondary stream is at most + * {@code timeDifference} later than the timestamp of the record from the primary stream. + * {@code timeDifference} can be negative but its absolute value must not be larger than current window "before" + * value (which would result in a negative window size). + * + * @param timeDifference relative window end time + * @throws IllegalArgumentException if the resulting window size is negative or {@code timeDifference} can't be represented as {@code long milliseconds} + */ + @SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this + public JoinWindows after(final Duration timeDifference) throws IllegalArgumentException { + ApiUtils.validateMillisecondDuration(timeDifference, "timeDifference"); + return after(timeDifference.toMillis()); + } + /** * Not supported by {@code JoinWindows}. * Throws {@link UnsupportedOperationException}. @@ -163,20 +215,22 @@ public long size() { } /** - * Reject late events that arrive more than {@code millisAfterWindowEnd} + * Reject late events that arrive more than {@code afterWindowEnd} * after the end of its window. * * Lateness is defined as (stream_time - record_timestamp). * - * @param millisAfterWindowEnd The grace period to admit late-arriving events to a window. + * @param afterWindowEnd The grace period to admit late-arriving events to a window. * @return this updated builder + * @throws IllegalArgumentException if the {@code afterWindowEnd} is negative of can't be represented as {@code long milliseconds} */ @SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this - public JoinWindows grace(final long millisAfterWindowEnd) { - if (millisAfterWindowEnd < 0) { + public JoinWindows grace(final Duration afterWindowEnd) throws IllegalArgumentException { + ApiUtils.validateMillisecondDuration(afterWindowEnd, "afterWindowEnd"); + if (afterWindowEnd.toMillis() < 0) { throw new IllegalArgumentException("Grace period must not be negative."); } - return new JoinWindows(beforeMs, afterMs, Duration.ofMillis(millisAfterWindowEnd), maintainDurationMs, segments); + return new JoinWindows(beforeMs, afterMs, afterWindowEnd, maintainDurationMs, segments); } @SuppressWarnings("deprecation") // continuing to support Windows#maintainMs/segmentInterval in fallback mode @@ -192,7 +246,7 @@ public long gracePeriodMs() { * @param durationMs the window retention time in milliseconds * @return itself * @throws IllegalArgumentException if {@code durationMs} is smaller than the window size - * @deprecated since 2.1. Use {@link JoinWindows#grace(long)} instead. + * @deprecated since 2.1. Use {@link JoinWindows#grace(Duration)} instead. */ @SuppressWarnings("deprecation") @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Materialized.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Materialized.java index 15ec6ce877202..a19412d7b13a8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Materialized.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Materialized.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.internals.Topic; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; @@ -243,12 +244,14 @@ public Materialized withCachingDisabled() { * from window-start through window-end, and for the entire grace period. * * @return itself + * @throws IllegalArgumentException if retention is negative or can't be represented as {@code long milliseconds} */ - public Materialized withRetention(final long retentionMs) { - if (retentionMs < 0) { + public Materialized withRetention(final Duration retention) throws IllegalArgumentException { + ApiUtils.validateMillisecondDuration(retention, "retention"); + if (retention.toMillis() < 0) { throw new IllegalArgumentException("Retention must not be negative."); } - retention = Duration.ofMillis(retentionMs); + this.retention = retention; return this; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java index 9efb78c786650..02c7cbfe79530 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.state.SessionBytesStoreSupplier; @@ -88,7 +89,9 @@ private SessionWindows(final long gapMs, final long maintainDurationMs, final Du * @return a new window specification with default maintain duration of 1 day * * @throws IllegalArgumentException if {@code inactivityGapMs} is zero or negative + * @deprecated User {@link #with(Duration)} instead. */ + @Deprecated public static SessionWindows with(final long inactivityGapMs) { if (inactivityGapMs <= 0) { throw new IllegalArgumentException("Gap time (inactivityGapMs) cannot be zero or negative."); @@ -96,6 +99,19 @@ public static SessionWindows with(final long inactivityGapMs) { return new SessionWindows(inactivityGapMs, DEFAULT_RETENTION_MS, null); } + /** + * Create a new window specification with the specified inactivity gap. + * + * @param inactivityGap the gap of inactivity between sessions + * @return a new window specification with default maintain duration of 1 day + * + * @throws IllegalArgumentException if {@code inactivityGap} is zero or negative or can't be represented as {@code long milliseconds} + */ + public static SessionWindows with(final Duration inactivityGap) { + ApiUtils.validateMillisecondDuration(inactivityGap, "inactivityGap"); + return with(inactivityGap.toMillis()); + } + /** * Set the window maintain duration (retention time) in milliseconds. * This retention time is a guaranteed lower bound for how long a window will be maintained. @@ -124,18 +140,20 @@ public SessionWindows until(final long durationMs) throws IllegalArgumentExcepti * close times can lead to surprising results in which a too-late event is rejected and then * a subsequent event moves the window boundary forward. * - * @param millisAfterWindowEnd The grace period to admit late-arriving events to a window. + * @param afterWindowEnd The grace period to admit late-arriving events to a window. * @return this updated builder + * @throws IllegalArgumentException if the {@code afterWindowEnd} is negative of can't be represented as {@code long milliseconds} */ - public SessionWindows grace(final long millisAfterWindowEnd) { - if (millisAfterWindowEnd < 0) { + public SessionWindows grace(final Duration afterWindowEnd) throws IllegalArgumentException { + ApiUtils.validateMillisecondDuration(afterWindowEnd, "afterWindowEnd"); + if (afterWindowEnd.toMillis() < 0) { throw new IllegalArgumentException("Grace period must not be negative."); } return new SessionWindows( gapMs, maintainDurationMs, - Duration.ofMillis(millisAfterWindowEnd) + afterWindowEnd ); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java index 6a58c2c09c4d7..46485b146d425 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.internals.TimeWindow; import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.state.WindowBytesStoreSupplier; @@ -100,7 +101,9 @@ private TimeWindows(final long sizeMs, * @param sizeMs The size of the window in milliseconds * @return a new window definition with default maintain duration of 1 day * @throws IllegalArgumentException if the specified window size is zero or negative + * @deprecated Use {@link #of(Duration)} instead */ + @Deprecated public static TimeWindows of(final long sizeMs) throws IllegalArgumentException { if (sizeMs <= 0) { throw new IllegalArgumentException("Window size (sizeMs) must be larger than zero."); @@ -109,6 +112,23 @@ public static TimeWindows of(final long sizeMs) throws IllegalArgumentException return new TimeWindows(sizeMs, sizeMs, null, DEFAULT_RETENTION_MS); } + /** + * Return a window definition with the given window size, and with the advance interval being equal to the window + * size. + * The time interval represented by the N-th window is: {@code [N * size, N * size + size)}. + *

+ * This provides the semantics of tumbling windows, which are fixed-sized, gap-less, non-overlapping windows. + * Tumbling windows are a special case of hopping windows with {@code advance == size}. + * + * @param size The size of the window + * @return a new window definition with default maintain duration of 1 day + * @throws IllegalArgumentException if the specified window size is zero or negative or can't be represented as {@code long milliseconds} + */ + public static TimeWindows of(final Duration size) throws IllegalArgumentException { + ApiUtils.validateMillisecondDuration(size, "size"); + return of(size.toMillis()); + } + /** * Return a window definition with the original size, but advance ("hop") the window by the given interval, which * specifies by how much a window moves forward relative to the previous one. @@ -116,12 +136,13 @@ public static TimeWindows of(final long sizeMs) throws IllegalArgumentException *

* This provides the semantics of hopping windows, which are fixed-sized, overlapping windows. * - * @param advanceMs The advance interval ("hop") in milliseconds of the window, with the requirement that - * {@code 0 < advanceMs ≤ sizeMs}. + * @param advanceMs The advance interval ("hop") in milliseconds of the window, with the requirement that {@code 0 < advanceMs <= sizeMs}. * @return a new window definition with default maintain duration of 1 day * @throws IllegalArgumentException if the advance interval is negative, zero, or larger-or-equal the window size + * @deprecated Use {@link #advanceBy(Duration)} instead */ @SuppressWarnings("deprecation") // will be fixed when we remove segments from Windows + @Deprecated public TimeWindows advanceBy(final long advanceMs) { if (advanceMs <= 0 || advanceMs > sizeMs) { throw new IllegalArgumentException(String.format("AdvanceMs must lie within interval (0, %d].", sizeMs)); @@ -129,6 +150,23 @@ public TimeWindows advanceBy(final long advanceMs) { return new TimeWindows(sizeMs, advanceMs, grace, maintainDurationMs, segments); } + /** + * Return a window definition with the original size, but advance ("hop") the window by the given interval, which + * specifies by how much a window moves forward relative to the previous one. + * The time interval represented by the N-th window is: {@code [N * advance, N * advance + size)}. + *

+ * This provides the semantics of hopping windows, which are fixed-sized, overlapping windows. + * + * @param advance The advance interval ("hop") of the window, with the requirement that {@code 0 < advance.toMillis() <= sizeMs}. + * @return a new window definition with default maintain duration of 1 day + * @throws IllegalArgumentException if the advance interval is negative, zero, or larger-or-equal the window size + */ + @SuppressWarnings("deprecation") // will be fixed when we remove segments from Windows + public TimeWindows advanceBy(final Duration advance) { + ApiUtils.validateMillisecondDuration(advance, "advance"); + return advanceBy(advance.toMillis()); + } + @Override public Map windowsFor(final long timestamp) { long windowStart = (Math.max(0, timestamp - sizeMs + advanceMs) / advanceMs) * advanceMs; @@ -152,15 +190,18 @@ public long size() { * * Lateness is defined as (stream_time - record_timestamp). * - * @param millisAfterWindowEnd The grace period to admit late-arriving events to a window. + * @param afterWindowEnd The grace period to admit late-arriving events to a window. * @return this updated builder + * @throws IllegalArgumentException if {@code afterWindowEnd} is negative or can't be represented as {@code long milliseconds} */ @SuppressWarnings("deprecation") // will be fixed when we remove segments from Windows - public TimeWindows grace(final long millisAfterWindowEnd) { - if (millisAfterWindowEnd < 0) { + public TimeWindows grace(final Duration afterWindowEnd) throws IllegalArgumentException { + ApiUtils.validateMillisecondDuration(afterWindowEnd, "afterWindowEnd"); + if (afterWindowEnd.toMillis() < 0) { throw new IllegalArgumentException("Grace period must not be negative."); } - return new TimeWindows(sizeMs, advanceMs, Duration.ofMillis(millisAfterWindowEnd), maintainDurationMs, segments); + + return new TimeWindows(sizeMs, advanceMs, afterWindowEnd, maintainDurationMs, segments); } @SuppressWarnings("deprecation") // continuing to support Windows#maintainMs/segmentInterval in fallback mode diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java index b7487b8e13d86..af8e87ee31ac7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream; +import java.time.Duration; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.PunctuationType; @@ -29,7 +30,7 @@ * This is a stateful record-by-record operation, i.e, {@link #transform(Object, Object)} is invoked individually for * each record of a stream and can access and modify a state that is available beyond a single call of * {@link #transform(Object, Object)} (cf. {@link KeyValueMapper} for stateless record transformation). - * Additionally, this {@code Transformer} can {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} + * Additionally, this {@code Transformer} can {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} * a method to be {@link Punctuator#punctuate(long) called periodically} with the provided context. *

* Use {@link TransformerSupplier} to provide new instances of {@code Transformer} to Kafka Stream's runtime. @@ -55,7 +56,7 @@ public interface Transformer { * framework may later re-use the transformer by calling {@link #init(ProcessorContext)} again. *

* The provided {@link ProcessorContext context} can be used to access topology and record meta data, to - * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be + * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} a method to be * {@link Punctuator#punctuate(long) called periodically} and to access attached {@link StateStore}s. *

* Note, that {@link ProcessorContext} is updated in the background with the current record's meta data. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java index 73aa9b1cd2abe..46d7270b332b9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams.kstream; +import java.time.Instant; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.internals.UnlimitedWindow; import org.apache.kafka.streams.processor.TimestampExtractor; @@ -62,7 +64,9 @@ public static UnlimitedWindows of() { * @param startMs the window start time * @return a new unlimited window that starts at {@code startMs} * @throws IllegalArgumentException if the start time is negative + * @deprecated Use {@link #startOn(Instant)} instead */ + @Deprecated public UnlimitedWindows startOn(final long startMs) throws IllegalArgumentException { if (startMs < 0) { throw new IllegalArgumentException("Window start time (startMs) cannot be negative."); @@ -70,6 +74,18 @@ public UnlimitedWindows startOn(final long startMs) throws IllegalArgumentExcept return new UnlimitedWindows(startMs); } + /** + * Return a new unlimited window for the specified start timestamp. + * + * @param start the window start time + * @return a new unlimited window that starts at {@code start} + * @throws IllegalArgumentException if the start time is negative or can't be represented as {@code long milliseconds} + */ + public UnlimitedWindows startOn(final Instant start) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(start, "start"); + return startOn(start.toEpochMilli()); + } + @Override public Map windowsFor(final long timestamp) { // always return the single unlimited window diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java index b02311bf95890..987cae591bd13 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream; +import java.time.Duration; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.ProcessorContext; @@ -29,7 +30,7 @@ * This is a stateful record-by-record operation, i.e, {@link #transform(Object)} is invoked individually for each * record of a stream and can access and modify a state that is available beyond a single call of * {@link #transform(Object)} (cf. {@link ValueMapper} for stateless value transformation). - * Additionally, this {@code ValueTransformer} can {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} + * Additionally, this {@code ValueTransformer} can {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} * a method to be {@link Punctuator#punctuate(long) called periodically} with the provided context. * If {@code ValueTransformer} is applied to a {@link KeyValue} pair record the record's key is preserved. *

@@ -54,7 +55,7 @@ public interface ValueTransformer { * framework may later re-use the transformer by calling {@link #init(ProcessorContext)} again. *

* The provided {@link ProcessorContext context} can be used to access topology and record meta data, to - * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be + * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} a method to be * {@link Punctuator#punctuate(long) called periodically} and to access attached {@link StateStore}s. *

* Note that {@link ProcessorContext} is updated in the background with the current record's meta data. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerWithKey.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerWithKey.java index 20c8692f13ea5..be37b0cfba61b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerWithKey.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerWithKey.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream; +import java.time.Duration; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.ProcessorContext; @@ -30,7 +31,7 @@ * record of a stream and can access and modify a state that is available beyond a single call of * {@link #transform(Object, Object)} (cf. {@link ValueMapper} for stateless value transformation). * Additionally, this {@code ValueTransformerWithKey} can - * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be + * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} a method to be * {@link Punctuator#punctuate(long) called periodically} with the provided context. * Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning. * If {@code ValueTransformerWithKey} is applied to a {@link KeyValue} pair record the record's key is preserved. @@ -57,7 +58,7 @@ public interface ValueTransformerWithKey { * This is called once per instance when the topology gets initialized. *

* The provided {@link ProcessorContext context} can be used to access topology and record meta data, to - * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be + * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} a method to be * {@link Punctuator#punctuate(long) called periodically} and to access attached {@link StateStore}s. *

* Note that {@link ProcessorContext} is updated in the background with the current record's meta data. @@ -97,4 +98,4 @@ public interface ValueTransformerWithKey { * will result in an {@link StreamsException exception}. */ void close(); -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java index f6250683282c8..ac49174b995af 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream; +import java.time.Instant; import org.apache.kafka.streams.processor.TimestampExtractor; /** @@ -36,6 +37,9 @@ public abstract class Window { protected final long startMs; protected final long endMs; + private final Instant startTime; + private final Instant endTime; + /** * Create a new window for the given start and end time. @@ -53,6 +57,9 @@ public Window(final long startMs, final long endMs) throws IllegalArgumentExcept } this.startMs = startMs; this.endMs = endMs; + + this.startTime = Instant.ofEpochMilli(startMs); + this.endTime = Instant.ofEpochMilli(endMs); } /** @@ -69,6 +76,24 @@ public long end() { return endMs; } + /** + * Return the start time of this window. + * + * @return The start time of this window. + */ + public Instant startTime() { + return startTime; + } + + /** + * Return the end time of this window. + * + * @return The end time of this window. + */ + public Instant endTime() { + return endTime; + } + /** * Check if the given window overlaps with this window. * Should throw an {@link IllegalArgumentException} if the {@code other} window has a different type than {@code diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index a3884c4e2e915..96fa8b9fc3ca4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream.internals; +import java.time.Duration; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.ForeachAction; @@ -852,8 +853,8 @@ private static StoreBuilder> joinWindowStoreBuilder(fin return Stores.windowStoreBuilder( Stores.persistentWindowStore( joinName + "-store", - windows.size() + windows.gracePeriodMs(), - windows.size(), + Duration.ofMillis(windows.size() + windows.gracePeriodMs()), + Duration.ofMillis(windows.size()), true ), keySerde, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java index fcb9c0240ac4c..8ba02bfefaf55 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream.internals; +import java.time.Duration; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; @@ -170,8 +171,8 @@ private StoreBuilder> materialize(final MaterializedInte supplier = Stores.persistentWindowStore( materialized.storeName(), - retentionPeriod, - windows.size(), + Duration.ofMillis(retentionPeriod), + Duration.ofMillis(windows.size()), false ); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Cancellable.java b/streams/src/main/java/org/apache/kafka/streams/processor/Cancellable.java index 2e56b560c3cfb..2acb7625ec1d2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/Cancellable.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/Cancellable.java @@ -16,8 +16,10 @@ */ package org.apache.kafka.streams.processor; +import java.time.Duration; + /** - * Cancellable interface returned in {@link ProcessorContext#schedule(long, PunctuationType, Punctuator)}. + * Cancellable interface returned in {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator)}. * * @see Punctuator */ diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java index e35337f73d0ab..f91f22f0baff8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor; +import java.time.Duration; import org.apache.kafka.common.annotation.InterfaceStability; /** @@ -33,7 +34,7 @@ public interface Processor { * framework may later re-use the processor by calling {@link #init()} again. *

* The provided {@link ProcessorContext context} can be used to access topology and record meta data, to - * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be + * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} a method to be * {@link Punctuator#punctuate(long) called periodically} and to access attached {@link StateStore}s. * * @param context the context; may not be null diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java index d21667fc6dd4e..8ec06d57c4d61 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor; +import java.time.Duration; import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.serialization.Serde; @@ -97,16 +98,15 @@ void register(final StateStore store, * Schedules a periodic operation for processors. A processor may call this method during * {@link Processor#init(ProcessorContext) initialization} or * {@link Processor#process(Object, Object) processing} to - * schedule a periodic callback - called a punctuation - to {@link Punctuator#punctuate(long)}. + * schedule a periodic callback — called a punctuation — to {@link Punctuator#punctuate(long)}. * The type parameter controls what notion of time is used for punctuation: *

    - *
  • {@link PunctuationType#STREAM_TIME} - uses "stream time", which is advanced by the processing of messages + *
  • {@link PunctuationType#STREAM_TIME} — uses "stream time", which is advanced by the processing of messages * in accordance with the timestamp as extracted by the {@link TimestampExtractor} in use. * The first punctuation will be triggered by the first record that is processed. * NOTE: Only advanced if messages arrive
  • - *
  • {@link PunctuationType#WALL_CLOCK_TIME} - uses system time (the wall-clock time), - * which is advanced at the polling interval ({@link org.apache.kafka.streams.StreamsConfig#POLL_MS_CONFIG}) - * independent of whether new messages arrive. + *
  • {@link PunctuationType#WALL_CLOCK_TIME} — uses system time (the wall-clock time), + * which is advanced independent of whether new messages arrive. * The first punctuation will be triggered after interval has elapsed. * NOTE: This is best effort only as its granularity is limited by how long an iteration of the * processing loop takes to complete
  • @@ -124,11 +124,48 @@ void register(final StateStore store, * @param type one of: {@link PunctuationType#STREAM_TIME}, {@link PunctuationType#WALL_CLOCK_TIME} * @param callback a function consuming timestamps representing the current stream or system time * @return a handle allowing cancellation of the punctuation schedule established by this method + * @deprecated Use {@link #schedule(Duration, PunctuationType, Punctuator)} instead */ + @Deprecated Cancellable schedule(final long intervalMs, final PunctuationType type, final Punctuator callback); + /** + * Schedules a periodic operation for processors. A processor may call this method during + * {@link Processor#init(ProcessorContext) initialization} or + * {@link Processor#process(Object, Object) processing} to + * schedule a periodic callback — called a punctuation — to {@link Punctuator#punctuate(long)}. + * The type parameter controls what notion of time is used for punctuation: + *
      + *
    • {@link PunctuationType#STREAM_TIME} — uses "stream time", which is advanced by the processing of messages + * in accordance with the timestamp as extracted by the {@link TimestampExtractor} in use. + * The first punctuation will be triggered by the first record that is processed. + * NOTE: Only advanced if messages arrive
    • + *
    • {@link PunctuationType#WALL_CLOCK_TIME} — uses system time (the wall-clock time), + * which is advanced independent of whether new messages arrive. + * The first punctuation will be triggered after interval has elapsed. + * NOTE: This is best effort only as its granularity is limited by how long an iteration of the + * processing loop takes to complete
    • + *
    + * + * Skipping punctuations: Punctuations will not be triggered more than once at any given timestamp. + * This means that "missed" punctuation will be skipped. + * It's possible to "miss" a punctuation if: + *
      + *
    • with {@link PunctuationType#STREAM_TIME}, when stream time advances more than interval
    • + *
    • with {@link PunctuationType#WALL_CLOCK_TIME}, on GC pause, too short interval, ...
    • + *
    + * + * @param interval the time interval between punctuations + * @param type one of: {@link PunctuationType#STREAM_TIME}, {@link PunctuationType#WALL_CLOCK_TIME} + * @param callback a function consuming timestamps representing the current stream or system time + * @return a handle allowing cancellation of the punctuation schedule established by this method + */ + Cancellable schedule(final Duration interval, + final PunctuationType type, + final Punctuator callback) throws IllegalArgumentException; + /** * Forwards a key/value pair to all downstream processors. * Used the input record's timestamp as timestamp for the output record. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/PunctuationType.java b/streams/src/main/java/org/apache/kafka/streams/processor/PunctuationType.java index bc0003da38cf1..32965e815dead 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/PunctuationType.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/PunctuationType.java @@ -16,8 +16,10 @@ */ package org.apache.kafka.streams.processor; +import java.time.Duration; + /** - * Controls what notion of time is used for punctuation scheduled via {@link ProcessorContext#schedule(long, PunctuationType, Punctuator)} schedule}: + * Controls what notion of time is used for punctuation scheduled via {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator)} schedule}: *
      *
    • STREAM_TIME - uses "stream time", which is advanced by the processing of messages * in accordance with the timestamp as extracted by the {@link TimestampExtractor} in use. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Punctuator.java b/streams/src/main/java/org/apache/kafka/streams/processor/Punctuator.java index 407270f60388f..1886dad827a35 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/Punctuator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/Punctuator.java @@ -16,8 +16,10 @@ */ package org.apache.kafka.streams.processor; +import java.time.Duration; + /** - * A functional interface used as an argument to {@link ProcessorContext#schedule(long, PunctuationType, Punctuator)}. + * A functional interface used as an argument to {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator)}. * * @see Cancellable */ diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java index 7e2610cc8fe05..0ef70b770305e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.time.Duration; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsMetrics; @@ -85,12 +86,20 @@ public StateStore getStateStore(final String name) { } @Override + @Deprecated public Cancellable schedule(final long intervalMs, final PunctuationType type, final Punctuator callback) { return delegate.schedule(intervalMs, type, callback); } + @Override + public Cancellable schedule(final Duration interval, + final PunctuationType type, + final Punctuator callback) throws IllegalArgumentException { + return delegate.schedule(interval, type, callback); + } + @Override public void forward(final K key, final V value) { throw new StreamsException("ProcessorContext#forward() not supported."); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java index 56af6e50a697d..5c5b84f155d67 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.time.Duration; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.Cancellable; import org.apache.kafka.streams.processor.PunctuationType; @@ -92,10 +93,19 @@ public void commit() { * @throws UnsupportedOperationException on every invocation */ @Override + @Deprecated public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) { throw new UnsupportedOperationException("this should not happen: schedule() not supported in global processor context."); } + /** + * @throws UnsupportedOperationException on every invocation + */ + @Override + public Cancellable schedule(final Duration interval, final PunctuationType type, final Punctuator callback) { + throw new UnsupportedOperationException("this should not happen: schedule() not supported in global processor context."); + } + @Override public long streamTime() { throw new RuntimeException("Stream time is not implemented for the global processor context."); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index ee21379c79ff9..21e1c17a8985f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams.processor.internals; +import java.time.Duration; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.Cancellable; @@ -150,10 +152,19 @@ public void commit() { } @Override + @Deprecated public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) { return task.schedule(interval, type, callback); } + @Override + public Cancellable schedule(final Duration interval, + final PunctuationType type, + final Punctuator callback) throws IllegalArgumentException { + ApiUtils.validateMillisecondDuration(interval, "interval"); + return schedule(interval.toMillis(), type, callback); + } + void setStreamTimeSupplier(final TimestampSupplier streamTimeSupplier) { this.streamTimeSupplier = streamTimeSupplier; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java index e8631aaf85a4f..6b835d97a3fb1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.time.Duration; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.header.Headers; @@ -189,10 +190,19 @@ public void commit() { * @throws UnsupportedOperationException on every invocation */ @Override + @SuppressWarnings("deprecation") public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) { throw new UnsupportedOperationException("this should not happen: schedule() not supported in standby tasks."); } + /** + * @throws UnsupportedOperationException on every invocation + */ + @Override + public Cancellable schedule(final Duration interval, final PunctuationType type, final Punctuator callback) throws IllegalArgumentException { + throw new UnsupportedOperationException("this should not happen: schedule() not supported in standby tasks."); + } + /** * @throws UnsupportedOperationException on every invocation */ diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java index 0c46fc2e87be7..08043388fb464 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.state; +import java.time.Instant; import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.kstream.Windowed; @@ -40,9 +41,8 @@ public interface ReadOnlyWindowStore { V fetch(K key, long time); /** - * Get all the key-value pairs with the given key and the time range from all - * the existing windows. - * + * Get all the key-value pairs with the given key and the time range from all the existing windows. + *

      * This iterator must be closed after use. *

      * The time range is inclusive and applies to the starting timestamp of the window. @@ -73,13 +73,51 @@ public interface ReadOnlyWindowStore { * @return an iterator over key-value pairs {@code } * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for key. + * @deprecated Use {@link #fetch(K, Instant, Instant)} instead */ + @Deprecated WindowStoreIterator fetch(K key, long timeFrom, long timeTo); /** - * Get all the key-value pairs in the given key range and time range from all - * the existing windows. + * Get all the key-value pairs with the given key and the time range from all the existing windows. + *

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

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

      + *

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

      + * For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest + * available window to the newest/latest window. * + * @param key the key to fetch + * @param from time range start (inclusive) + * @param from time range end (inclusive) + * @return an iterator over key-value pairs {@code } + * @throws InvalidStateStoreException if the store is not initialized + * @throws NullPointerException If {@code null} is used for key. + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + */ + WindowStoreIterator fetch(K key, Instant from, Instant to) throws IllegalArgumentException; + + /** + * Get all the key-value pairs in the given key range and time range from all the existing windows. + *

      * This iterator must be closed after use. * * @param from the first key in the range @@ -89,9 +127,28 @@ public interface ReadOnlyWindowStore { * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for any key. + * @deprecated Use {@link #fetch(Object, Object, Instant, Instant)} instead */ + @Deprecated KeyValueIterator, V> fetch(K from, K to, long timeFrom, long timeTo); - + + /** + * Get all the key-value pairs in the given key range and time range from all the existing windows. + *

      + * This iterator must be closed after use. + * + * @param from the first key in the range + * @param to the last key in the range + * @param fromTime time range start (inclusive) + * @param toTime time range end (inclusive) + * @return an iterator over windowed key-value pairs {@code , value>} + * @throws InvalidStateStoreException if the store is not initialized + * @throws NullPointerException If {@code null} is used for any key. + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + */ + KeyValueIterator, V> fetch(K from, K to, Instant fromTime, Instant toTime) + throws IllegalArgumentException; + /** * Gets all the key-value pairs in the existing windows. * @@ -108,6 +165,20 @@ public interface ReadOnlyWindowStore { * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException if {@code null} is used for any key + * @deprecated Use {@link #fetchAll(Instant, Instant)} instead */ + @Deprecated KeyValueIterator, V> fetchAll(long timeFrom, long timeTo); + + /** + * Gets all the key-value pairs that belong to the windows within in the given time range. + * + * @param from the beginning of the time slot from which to search (inclusive) + * @param to the end of the time slot from which to search (inclusive) + * @return an iterator over windowed key-value pairs {@code , value>} + * @throws InvalidStateStoreException if the store is not initialized + * @throws NullPointerException if {@code null} is used for any key + * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds} + */ + KeyValueIterator, V> fetchAll(Instant from, Instant to) throws IllegalArgumentException; } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java index 6e965fb27fab5..30e51403714c5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java @@ -16,11 +16,13 @@ */ package org.apache.kafka.streams.state; +import java.time.Duration; import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; import org.apache.kafka.streams.state.internals.MemoryNavigableLRUCache; @@ -186,7 +188,9 @@ public static WindowBytesStoreSupplier persistentWindowStore(final String name, * @param windowSize size of the windows (cannot be negative) * @param retainDuplicates whether or not to retain duplicates. * @return an instance of {@link WindowBytesStoreSupplier} + * @deprecated Use {@link #persistentWindowStore(String, Duration, Duration, boolean)} instead */ + @Deprecated public static WindowBytesStoreSupplier persistentWindowStore(final String name, final long retentionPeriod, final long windowSize, @@ -196,6 +200,29 @@ public static WindowBytesStoreSupplier persistentWindowStore(final String name, return persistentWindowStore(name, retentionPeriod, windowSize, retainDuplicates, defaultSegmentInterval); } + /** + * Create a persistent {@link WindowBytesStoreSupplier}. + * @param name name of the store (cannot be {@code null}) + * @param retentionPeriod length of time to retain data in the store (cannot be negative) + * Note that the retention period must be at least long enough to contain the + * windowed data's entire life cycle, from window-start through window-end, + * and for the entire grace period. + * @param windowSize size of the windows (cannot be negative) + * @param retainDuplicates whether or not to retain duplicates. + * @return an instance of {@link WindowBytesStoreSupplier} + * @throws IllegalArgumentException if {@code retentionPeriod} or {@code windowSize} can't be represented as {@code long milliseconds} + */ + public static WindowBytesStoreSupplier persistentWindowStore(final String name, + final Duration retentionPeriod, + final Duration windowSize, + final boolean retainDuplicates) throws IllegalArgumentException { + Objects.requireNonNull(name, "name cannot be null"); + ApiUtils.validateMillisecondDuration(retentionPeriod, "retentionPeriod"); + ApiUtils.validateMillisecondDuration(windowSize, "windowSize"); + + return persistentWindowStore(name, retentionPeriod.toMillis(), windowSize.toMillis(), retainDuplicates); + } + /** * Create a persistent {@link WindowBytesStoreSupplier}. * @param name name of the store (cannot be {@code null}) @@ -207,7 +234,9 @@ public static WindowBytesStoreSupplier persistentWindowStore(final String name, * @param windowSize size of the windows (cannot be negative) * @param retainDuplicates whether or not to retain duplicates. * @return an instance of {@link WindowBytesStoreSupplier} + * @deprecated Use {@link #persistentWindowStore(String, Duration, Duration, boolean)} instead */ + @Deprecated public static WindowBytesStoreSupplier persistentWindowStore(final String name, final long retentionPeriod, final long windowSize, @@ -250,6 +279,21 @@ public static SessionBytesStoreSupplier persistentSessionStore(final String name return new RocksDbSessionBytesStoreSupplier(name, retentionPeriod); } + /** + * Create a persistent {@link SessionBytesStoreSupplier}. + * @param name name of the store (cannot be {@code null}) + * @param retentionPeriod length ot time to retain data in the store (cannot be negative) + * Note that the retention period must be at least long enough to contain the + * windowed data's entire life cycle, from window-start through window-end, + * and for the entire grace period. + * @return an instance of a {@link SessionBytesStoreSupplier} + */ + public static SessionBytesStoreSupplier persistentSessionStore(final String name, + final Duration retentionPeriod) { + ApiUtils.validateMillisecondDuration(retentionPeriod, "retentionPeriod"); + return persistentSessionStore(name, retentionPeriod.toMillis()); + } + /** * Creates a {@link StoreBuilder} that can be used to build a {@link WindowStore}. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java index 1685123ff9687..ad74ae1e74d78 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams.state; +import org.apache.kafka.streams.errors.InvalidStateStoreException; +import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.StateStore; /** @@ -48,4 +50,66 @@ public interface WindowStore extends StateStore, ReadOnlyWindowStore * @throws NullPointerException If null is used for key. */ void put(K key, V value, long windowStartTimestamp); + + /** + * Get all the key-value pairs with the given key and the time range from all the existing windows. + *

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

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

      + *

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

      + * For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest + * available window to the newest/latest window. + * + * @param key the key to fetch + * @param timeFrom time range start (inclusive) + * @param timeTo time range end (inclusive) + * @return an iterator over key-value pairs {@code } + * @throws InvalidStateStoreException if the store is not initialized + * @throws NullPointerException If {@code null} is used for key. + */ + WindowStoreIterator fetch(K key, long timeFrom, long timeTo); + + /** + * Get all the key-value pairs in the given key range and time range from all the existing windows. + *

      + * This iterator must be closed after use. + * + * @param from the first key in the range + * @param to the last key in the range + * @param timeFrom time range start (inclusive) + * @param timeTo time range end (inclusive) + * @return an iterator over windowed key-value pairs {@code , value>} + * @throws InvalidStateStoreException if the store is not initialized + * @throws NullPointerException If {@code null} is used for any key. + */ + KeyValueIterator, V> fetch(K from, K to, long timeFrom, long timeTo); + + /** + * Gets all the key-value pairs that belong to the windows within in the given time range. + * + * @param timeFrom the beginning of the time slot from which to search (inclusive) + * @param timeTo the end of the time slot from which to search (inclusive) + * @return an iterator over windowed key-value pairs {@code , value>} + * @throws InvalidStateStoreException if the store is not initialized + * @throws NullPointerException if {@code null} is used for any key + */ + KeyValueIterator, V> fetchAll(long timeFrom, long timeTo); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java index c07130e0fe56a..14163514b0f9d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java @@ -16,12 +16,14 @@ */ package org.apache.kafka.streams.state; +import java.time.Instant; import org.apache.kafka.streams.KeyValue; import java.io.Closeable; /** - * Iterator interface of {@link KeyValue} with key typed {@link Long} used for {@link WindowStore#fetch(Object, long, long)}. + * Iterator interface of {@link KeyValue} with key typed {@link Long} used for {@link WindowStore#fetch(Object, long, long)} + * and {@link WindowStore#fetch(Object, Instant, Instant)} * * Users must call its {@code close} method explicitly upon completeness to release resources, * or use try-with-resources statement (available since JDK7) for this {@link Closeable} class. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index f8d9ad590a262..f6b62b2b9351b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -16,9 +16,11 @@ */ package org.apache.kafka.streams.state.internals; +import java.time.Instant; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.CacheFlushListener; import org.apache.kafka.streams.processor.ProcessorContext; @@ -203,6 +205,13 @@ public synchronized WindowStoreIterator fetch(final Bytes key, final lon return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator); } + @Override + public WindowStoreIterator fetch(final Bytes key, final Instant from, final Instant to) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(from, "from"); + ApiUtils.validateMillisecondInstant(to, "to"); + return fetch(key, from.toEpochMilli(), to.toEpochMilli()); + } + @Override public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to, final long timeFrom, final long timeTo) { // since this function may not access the underlying inner store, we need to validate @@ -231,7 +240,17 @@ public KeyValueIterator, byte[]> fetch(final Bytes from, final B cacheFunction ); } - + + @Override + public KeyValueIterator, byte[]> fetch(final Bytes from, + final Bytes to, + final Instant fromTime, + final Instant toTime) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(fromTime, "fromTime"); + ApiUtils.validateMillisecondInstant(toTime, "toTime"); + return fetch(from, to, fromTime.toEpochMilli(), toTime.toEpochMilli()); + } + private V fetchPrevious(final Bytes key, final long timestamp) { final byte[] value = underlying.fetch(key, timestamp); if (value != null) { @@ -259,10 +278,10 @@ public KeyValueIterator, byte[]> all() { @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { validateStoreOpen(); - + final KeyValueIterator, byte[]> underlyingIterator = underlying.fetchAll(timeFrom, timeTo); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.all(name); - + final HasNextCondition hasNextCondition = keySchema.hasNextCondition(null, null, timeFrom, timeTo); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, @@ -275,4 +294,11 @@ public KeyValueIterator, byte[]> fetchAll(final long timeFrom, f cacheFunction ); } + + @Override + public KeyValueIterator, byte[]> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(from, "from"); + ApiUtils.validateMillisecondInstant(to, "to"); + return fetchAll(from.toEpochMilli(), to.toEpochMilli()); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java index 785aacd25c1bd..d4e47c6d18f96 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java @@ -16,8 +16,10 @@ */ package org.apache.kafka.streams.state.internals; +import java.time.Instant; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; @@ -56,11 +58,28 @@ public WindowStoreIterator fetch(final Bytes key, final long from, final return bytesStore.fetch(key, from, to); } + @Override + public WindowStoreIterator fetch(final Bytes key, final Instant from, final Instant to) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(from, "from"); + ApiUtils.validateMillisecondInstant(to, "to"); + return fetch(key, from.toEpochMilli(), to.toEpochMilli()); + } + @Override public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) { return bytesStore.fetch(keyFrom, keyTo, from, to); } + @Override + public KeyValueIterator, byte[]> fetch(final Bytes from, + final Bytes to, + final Instant fromTime, + final Instant toTime) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(fromTime, "fromTime"); + ApiUtils.validateMillisecondInstant(toTime, "toTime"); + return fetch(from, to, fromTime.toEpochMilli(), toTime.toEpochMilli()); + } + @Override public KeyValueIterator, byte[]> all() { return bytesStore.all(); @@ -70,7 +89,14 @@ public KeyValueIterator, byte[]> all() { public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { return bytesStore.fetchAll(timeFrom, timeTo); } - + + @Override + public KeyValueIterator, byte[]> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(from, "from"); + ApiUtils.validateMillisecondInstant(to, "to"); + return fetchAll(from.toEpochMilli(), to.toEpochMilli()); + } + @Override public void put(final Bytes key, final byte[] value) { put(key, value, context.timestamp()); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java index 1b5d5e5611a9b..d95b44222e20b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams.state.internals; +import java.time.Instant; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.state.KeyValueIterator; @@ -64,6 +66,7 @@ public V fetch(final K key, final long time) { } @Override + @Deprecated public WindowStoreIterator fetch(final K key, final long timeFrom, final long timeTo) { Objects.requireNonNull(key, "key can't be null"); final List> stores = provider.stores(storeName, windowStoreType); @@ -84,6 +87,13 @@ public WindowStoreIterator fetch(final K key, final long timeFrom, final long return KeyValueIterators.emptyWindowStoreIterator(); } + @Override + public WindowStoreIterator fetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(from, "from"); + ApiUtils.validateMillisecondInstant(to, "to"); + return fetch(key, from.toEpochMilli(), to.toEpochMilli()); + } + @Override public KeyValueIterator, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { Objects.requireNonNull(from, "from can't be null"); @@ -99,7 +109,14 @@ public KeyValueIterator, V> apply(final ReadOnlyWindowStore st provider.stores(storeName, windowStoreType).iterator(), nextIteratorFunction)); } - + + @Override + public KeyValueIterator, V> fetch(final K from, final K to, final Instant fromTime, final Instant toTime) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(fromTime, "fromTime"); + ApiUtils.validateMillisecondInstant(toTime, "toTime"); + return fetch(from, to, fromTime.toEpochMilli(), toTime.toEpochMilli()); + } + @Override public KeyValueIterator, V> all() { final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = new NextIteratorFunction, V, ReadOnlyWindowStore>() { @@ -115,6 +132,7 @@ public KeyValueIterator, V> apply(final ReadOnlyWindowStore st } @Override + @Deprecated public KeyValueIterator, V> fetchAll(final long timeFrom, final long timeTo) { final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = new NextIteratorFunction, V, ReadOnlyWindowStore>() { @Override @@ -127,4 +145,11 @@ public KeyValueIterator, V> apply(final ReadOnlyWindowStore st provider.stores(storeName, windowStoreType).iterator(), nextIteratorFunction)); } + + @Override + public KeyValueIterator, V> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(from, "from"); + ApiUtils.validateMillisecondInstant(to, "to"); + return fetchAll(from.toEpochMilli(), to.toEpochMilli()); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index 5162eac8848d2..e1b6cd1d52e4f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -16,10 +16,12 @@ */ package org.apache.kafka.streams.state.internals; +import java.time.Instant; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; @@ -147,6 +149,13 @@ public WindowStoreIterator fetch(final K key, final long timeFrom, final long time); } + @Override + public WindowStoreIterator fetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(from, "from"); + ApiUtils.validateMillisecondInstant(to, "to"); + return fetch(key, from.toEpochMilli(), to.toEpochMilli()); + } + @Override public KeyValueIterator, V> all() { return new MeteredWindowedKeyValueIterator<>(inner.all(), fetchTime, metrics, serdes, time); @@ -161,6 +170,13 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, final long time); } + @Override + public KeyValueIterator, V> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(from, "from"); + ApiUtils.validateMillisecondInstant(to, "to"); + return fetchAll(from.toEpochMilli(), to.toEpochMilli()); + } + @Override public KeyValueIterator, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { return new MeteredWindowedKeyValueIterator<>(inner.fetch(keyBytes(from), keyBytes(to), timeFrom, timeTo), @@ -170,6 +186,13 @@ public KeyValueIterator, V> fetch(final K from, final K to, final lo time); } + @Override + public KeyValueIterator, V> fetch(final K from, final K to, final Instant fromTime, final Instant toTime) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(fromTime, "fromTime"); + ApiUtils.validateMillisecondInstant(toTime, "toTime"); + return fetch(from, to, fromTime.toEpochMilli(), toTime.toEpochMilli()); + } + @Override public void flush() { final long startNs = time.nanoseconds(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index d7bb523b049ff..e8037bc816358 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -16,8 +16,10 @@ */ package org.apache.kafka.streams.state.internals; +import java.time.Instant; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; @@ -91,12 +93,26 @@ public WindowStoreIterator fetch(final K key, final long timeFrom, final long return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).valuesIterator(); } + @Override + public WindowStoreIterator fetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(from, "from"); + ApiUtils.validateMillisecondInstant(to, "to"); + return fetch(key, from.toEpochMilli(), to.toEpochMilli()); + } + @Override public KeyValueIterator, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { final KeyValueIterator bytesIterator = bytesStore.fetch(Bytes.wrap(serdes.rawKey(from)), Bytes.wrap(serdes.rawKey(to)), timeFrom, timeTo); return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).keyValueIterator(); } + @Override + public KeyValueIterator, V> fetch(final K from, final K to, final Instant fromTime, final Instant toTime) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(fromTime, "fromTime"); + ApiUtils.validateMillisecondInstant(toTime, "toTime"); + return fetch(from, to, fromTime.toEpochMilli(), toTime.toEpochMilli()); + } + @Override public KeyValueIterator, V> all() { final KeyValueIterator bytesIterator = bytesStore.all(); @@ -109,6 +125,13 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, final long return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).keyValueIterator(); } + @Override + public KeyValueIterator, V> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(from, "from"); + ApiUtils.validateMillisecondInstant(to, "to"); + return fetchAll(from.toEpochMilli(), to.toEpochMilli()); + } + private void maybeUpdateSeqnumForDups() { if (retainDuplicates) { seqnum = (seqnum + 1) & 0x7FFFFFFF; diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java index 5e07703ddd571..abc4cb90b7d62 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams; +import java.time.Duration; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.MockProducer; @@ -463,7 +464,7 @@ public void shouldReturnFalseOnCloseWhenThreadsHaventTerminated() throws Excepti System.currentTimeMillis()); assertTrue("Timed out waiting to receive single message", latch.await(30, TimeUnit.SECONDS)); - assertFalse(streams.close(10, TimeUnit.MILLISECONDS)); + assertFalse(streams.close(Duration.ofMillis(10))); } finally { // stop the thread so we don't interfere with other tests etc keepRunning.set(false); diff --git a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java index 02840b4641cd6..289cdf0d6dc6c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java @@ -47,6 +47,7 @@ import java.util.Set; import java.util.regex.Pattern; +import static java.time.Duration.ofMillis; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -737,7 +738,7 @@ public void timeWindowZeroArgCountShouldPreserveTopologyStructure() { final StreamsBuilder builder = new StreamsBuilder(); builder.stream("input-topic") .groupByKey() - .windowedBy(TimeWindows.of(1)) + .windowedBy(TimeWindows.of(ofMillis(1))) .count(); final TopologyDescription describe = builder.build().describe(); assertEquals( @@ -757,7 +758,7 @@ public void timeWindowNamedMaterializedCountShouldPreserveTopologyStructure() { final StreamsBuilder builder = new StreamsBuilder(); builder.stream("input-topic") .groupByKey() - .windowedBy(TimeWindows.of(1)) + .windowedBy(TimeWindows.of(ofMillis(1))) .count(Materialized.as("count-store")); final TopologyDescription describe = builder.build().describe(); assertEquals( @@ -777,7 +778,7 @@ public void timeWindowAnonymousMaterializedCountShouldPreserveTopologyStructure( final StreamsBuilder builder = new StreamsBuilder(); builder.stream("input-topic") .groupByKey() - .windowedBy(TimeWindows.of(1)) + .windowedBy(TimeWindows.of(ofMillis(1))) .count(Materialized.with(null, Serdes.Long())); final TopologyDescription describe = builder.build().describe(); assertEquals( @@ -797,7 +798,7 @@ public void sessionWindowZeroArgCountShouldPreserveTopologyStructure() { final StreamsBuilder builder = new StreamsBuilder(); builder.stream("input-topic") .groupByKey() - .windowedBy(SessionWindows.with(1)) + .windowedBy(SessionWindows.with(ofMillis(1))) .count(); final TopologyDescription describe = builder.build().describe(); assertEquals( @@ -817,7 +818,7 @@ public void sessionWindowNamedMaterializedCountShouldPreserveTopologyStructure() final StreamsBuilder builder = new StreamsBuilder(); builder.stream("input-topic") .groupByKey() - .windowedBy(SessionWindows.with(1)) + .windowedBy(SessionWindows.with(ofMillis(1))) .count(Materialized.as("count-store")); final TopologyDescription describe = builder.build().describe(); assertEquals( @@ -837,7 +838,7 @@ public void sessionWindowAnonymousMaterializedCountShouldPreserveTopologyStructu final StreamsBuilder builder = new StreamsBuilder(); builder.stream("input-topic") .groupByKey() - .windowedBy(SessionWindows.with(1)) + .windowedBy(SessionWindows.with(ofMillis(1))) .count(Materialized.with(null, Serdes.Long())); final TopologyDescription describe = builder.build().describe(); assertEquals( diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java index caa02db185d97..bb199b73d5711 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.integration; +import java.time.Duration; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.ConsumerGroupDescription; @@ -66,6 +67,7 @@ import kafka.tools.StreamsResetter; +import static java.time.Duration.ofMillis; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; @@ -199,7 +201,7 @@ void prepareTest() throws Exception { void cleanupTest() throws Exception { if (streams != null) { - streams.close(30, TimeUnit.SECONDS); + streams.close(Duration.ofSeconds(30)); } IntegrationTestUtils.purgeLocalStreamsState(streamsConfig); } @@ -526,7 +528,7 @@ public KeyValue apply(final Long key, final String value) { input.through(INTERMEDIATE_USER_TOPIC) .groupByKey() - .windowedBy(TimeWindows.of(35).advanceBy(10)) + .windowedBy(TimeWindows.of(ofMillis(35)).advanceBy(ofMillis(10))) .count() .toStream() .map(new KeyValueMapper, Long, KeyValue>() { diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java index cae0f575ee51b..a0ada62e03f9c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.integration; +import java.time.Duration; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serdes; @@ -52,7 +53,6 @@ import java.util.Arrays; import java.util.List; import java.util.Properties; -import java.util.concurrent.TimeUnit; import kafka.utils.MockTime; @@ -148,7 +148,7 @@ public boolean conditionMet() { } }, 10000L, "Has not processed record within 10 seconds"); - kafkaStreams.close(30, TimeUnit.SECONDS); + kafkaStreams.close(Duration.ofSeconds(30)); final List expectedRetrievedValues = Arrays.asList(1L, 2L, 3L, 4L); assertEquals(expectedRetrievedValues, retrievedValuesList); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java index 153c5a1c5088a..9bd8c65c2dbc1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java @@ -57,6 +57,8 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import static java.time.Duration.ofMillis; +import static java.time.Duration.ofSeconds; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForCompletion; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -187,8 +189,8 @@ public void shouldCompactAndDeleteTopicsForWindowStoreChangelogs() throws Except textLines.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"))) .groupBy(MockMapper.selectValueMapper()) - .windowedBy(TimeWindows.of(1000).grace(0L)) - .count(Materialized.>as("CountWindows").withRetention(2_000L)); + .windowedBy(TimeWindows.of(ofSeconds(1L)).grace(ofMillis(0L))) + .count(Materialized.>as("CountWindows").withRetention(ofSeconds(2L))); final KafkaStreams streams = new KafkaStreams(builder.build(), streamsProp); streams.start(); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java index b51511e19e98a..08aa245b61651 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java @@ -53,6 +53,8 @@ import java.util.List; import java.util.Properties; +import static java.time.Duration.ofMillis; + /** * Similar to KStreamAggregationIntegrationTest but with dedupping enabled * by virtue of having a large commit interval @@ -144,7 +146,7 @@ public void shouldReduceWindowed() throws Exception { produceMessages(secondBatchTimestamp); groupedStream - .windowedBy(TimeWindows.of(500L)) + .windowedBy(TimeWindows.of(ofMillis(500L))) .reduce(reducer, Materialized.as("reduce-time-windows")) .toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start()) .to(outputTopic, Produced.with(Serdes.String(), Serdes.String())); @@ -179,7 +181,7 @@ public void shouldGroupByKey() throws Exception { produceMessages(timestamp); stream.groupByKey(Serialized.with(Serdes.Integer(), Serdes.String())) - .windowedBy(TimeWindows.of(500L)) + .windowedBy(TimeWindows.of(ofMillis(500L))) .count(Materialized.as("count-windows")) .toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start()) .to(outputTopic, Produced.with(Serdes.String(), Serdes.Long())); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java index ce6c352af9c38..7642f69b8e0fe 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java @@ -85,6 +85,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import static java.time.Duration.ofMillis; +import static java.time.Instant.ofEpochMilli; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.Is.is; @@ -202,7 +204,7 @@ public void shouldReduceWindowed() throws Exception { final Serde> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class); groupedStream - .windowedBy(TimeWindows.of(500L)) + .windowedBy(TimeWindows.of(ofMillis(500L))) .reduce(reducer) .toStream() .to(outputTopic, Produced.with(windowedSerde, Serdes.String())); @@ -307,7 +309,7 @@ public void shouldAggregateWindowed() throws Exception { produceMessages(secondTimestamp); final Serde> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class); - groupedStream.windowedBy(TimeWindows.of(500L)) + groupedStream.windowedBy(TimeWindows.of(ofMillis(500L))) .aggregate( initializer, aggregator, @@ -427,7 +429,7 @@ public void shouldGroupByKey() throws Exception { produceMessages(timestamp); stream.groupByKey(Serialized.with(Serdes.Integer(), Serdes.String())) - .windowedBy(TimeWindows.of(500L)) + .windowedBy(TimeWindows.of(ofMillis(500L))) .count() .toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start()).to(outputTopic, Produced.with(Serdes.String(), Serdes.Long())); @@ -520,7 +522,7 @@ public void shouldCountSessionWindows() throws Exception { builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String())) .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(SessionWindows.with(sessionGap)) + .windowedBy(SessionWindows.with(ofMillis(sessionGap))) .count() .toStream() .transform(() -> new Transformer, Long, KeyValue>() { @@ -618,7 +620,7 @@ public void shouldReduceSessionWindows() throws Exception { final String userSessionsStore = "UserSessionsStore"; builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String())) .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(SessionWindows.with(sessionGap)) + .windowedBy(SessionWindows.with(ofMillis(sessionGap))) .reduce((value1, value2) -> value1 + ":" + value2, Materialized.as(userSessionsStore)) .toStream() .foreach((key, value) -> { @@ -705,7 +707,7 @@ public void shouldCountUnlimitedWindows() throws Exception { builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String())) .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(UnlimitedWindows.of().startOn(startTime)) + .windowedBy(UnlimitedWindows.of().startOn(ofEpochMilli(startTime))) .count() .toStream() .transform(() -> new Transformer, Long, KeyValue>() { diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java index dd78190931a5f..d5781dd9c47cc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.integration; +import java.time.Duration; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Serdes; @@ -51,7 +52,6 @@ import java.util.Properties; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; @Category({IntegrationTest.class}) public class KTableSourceTopicRestartIntegrationTest { @@ -130,7 +130,7 @@ public void shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosDi assertNumberValuesRead(readKeyValues, expectedResultsWithDataWrittenDuringRestoreMap, "Table did not get all values after restart"); } finally { - streamsOne.close(5, TimeUnit.SECONDS); + streamsOne.close(Duration.ofSeconds(5)); } } @@ -155,7 +155,7 @@ public void shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosEn assertNumberValuesRead(readKeyValues, expectedResultsWithDataWrittenDuringRestoreMap, "Table did not get all values after restart"); } finally { - streamsOne.close(5, TimeUnit.SECONDS); + streamsOne.close(Duration.ofSeconds(5)); } } @@ -179,7 +179,7 @@ public void shouldRestoreAndProgressWhenTopicNotWrittenToDuringRestoration() thr assertNumberValuesRead(readKeyValues, expectedValues, "Table did not get all values after restart"); } finally { - streamsOne.close(5, TimeUnit.SECONDS); + streamsOne.close(Duration.ofSeconds(5)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index 2269a5dd88a99..96d7d14d6e60c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.integration; +import java.time.Duration; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.Config; import org.apache.kafka.clients.producer.ProducerConfig; @@ -51,7 +52,6 @@ import java.util.Properties; import java.util.Set; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; @Category({IntegrationTest.class}) public class PurgeRepartitionTopicIntegrationTest { @@ -173,7 +173,7 @@ public void setup() { @After public void shutdown() throws IOException { if (kafkaStreams != null) { - kafkaStreams.close(30, TimeUnit.SECONDS); + kafkaStreams.close(Duration.ofSeconds(30)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java index 97d1071aaa218..76eec71daf979 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java @@ -86,6 +86,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import static java.time.Duration.ofMillis; +import static java.time.Duration.ofSeconds; +import static java.time.Instant.ofEpochMilli; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsEqual.equalTo; import static org.junit.Assert.assertEquals; @@ -220,7 +223,7 @@ public int compare(final KeyValue o1, @After public void shutdown() throws IOException { if (kafkaStreams != null) { - kafkaStreams.close(30, TimeUnit.SECONDS); + kafkaStreams.close(ofSeconds(30)); } IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration); } @@ -256,7 +259,7 @@ public Iterable apply(final String value) { // Create a Windowed State Store that contains the word count for every 1 minute groupedByWord - .windowedBy(TimeWindows.of(WINDOW_SIZE)) + .windowedBy(TimeWindows.of(ofMillis(WINDOW_SIZE))) .count(Materialized.>as(windowStoreName + "-" + inputTopic)) .toStream(new KeyValueMapper, Long, String>() { @Override @@ -361,7 +364,7 @@ public boolean conditionMet() { final int index = metadata.hostInfo().port(); final KafkaStreams streamsWithKey = streamRunnables[index].getStream(); final ReadOnlyWindowStore store = streamsWithKey.store(storeName, QueryableStoreTypes.windowStore()); - return store != null && store.fetch(key, from, to) != null; + return store != null && store.fetch(key, ofEpochMilli(from), ofEpochMilli(to)) != null; } catch (final IllegalStateException e) { // Kafka Streams instance may have closed but rebalance hasn't happened return false; @@ -695,7 +698,7 @@ private void verifyCanQueryState(final int cacheSizeBytes) throws Exception { final String windowStoreName = "windowed-count"; s1.groupByKey() - .windowedBy(TimeWindows.of(WINDOW_SIZE)) + .windowedBy(TimeWindows.of(ofMillis(WINDOW_SIZE))) .count(Materialized.>as(windowStoreName)); kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration); kafkaStreams.start(); @@ -1017,7 +1020,7 @@ private void waitUntilAtLeastNumRecordProcessed(final String topic, final int nu private Set> fetch(final ReadOnlyWindowStore store, final String key) { - final WindowStoreIterator fetch = store.fetch(key, 0, System.currentTimeMillis()); + final WindowStoreIterator fetch = store.fetch(key, ofEpochMilli(0), ofEpochMilli(System.currentTimeMillis())); if (fetch.hasNext()) { final KeyValue next = fetch.next(); return Collections.singleton(KeyValue.pair(key, next.value)); @@ -1028,7 +1031,7 @@ private Set> fetch(final ReadOnlyWindowStore fetchMap(final ReadOnlyWindowStore store, final String key) { - final WindowStoreIterator fetch = store.fetch(key, 0, System.currentTimeMillis()); + final WindowStoreIterator fetch = store.fetch(key, ofEpochMilli(0), ofEpochMilli(System.currentTimeMillis())); if (fetch.hasNext()) { final KeyValue next = fetch.next(); return Collections.singletonMap(key, next.value); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionOptimizingIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionOptimizingIntegrationTest.java index cde2349fe5cd6..ec9df193657be 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionOptimizingIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionOptimizingIntegrationTest.java @@ -55,10 +55,11 @@ import java.util.List; import java.util.Locale; import java.util.Properties; -import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; +import static java.time.Duration.ofMillis; +import static java.time.Duration.ofSeconds; import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -161,7 +162,7 @@ private void runIntegrationTest(final String optimizationConfig, mappedStream.filter((k, v) -> k.equals("A")) .join(countStream, (v1, v2) -> v1 + ":" + v2.toString(), - JoinWindows.of(5000), + JoinWindows.of(ofMillis(5000)), Joined.with(Serdes.String(), Serdes.String(), Serdes.Long())) .to(JOINED_TOPIC); @@ -211,7 +212,7 @@ private void runIntegrationTest(final String optimizationConfig, assertThat(3, equalTo(processorValueCollector.size())); assertThat(processorValueCollector, equalTo(expectedCollectedProcessorValues)); - streams.close(5, TimeUnit.SECONDS); + streams.close(ofSeconds(5)); } @@ -431,4 +432,4 @@ public void process(final String key, final String value) { + " Sink: KSTREAM-SINK-0000000028 (topic: outputTopic_2)\n" + " <-- KTABLE-TOSTREAM-0000000027\n\n"; -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionWithMergeOptimizingIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionWithMergeOptimizingIntegrationTest.java index 29901a616d433..bf65264684608 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionWithMergeOptimizingIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionWithMergeOptimizingIntegrationTest.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.integration; +import java.time.Duration; import kafka.utils.MockTime; import org.apache.kafka.common.serialization.LongDeserializer; import org.apache.kafka.common.serialization.Serdes; @@ -46,7 +47,6 @@ import java.util.Arrays; import java.util.List; import java.util.Properties; -import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -167,7 +167,7 @@ private void runIntegrationTest(final String optimizationConfig, final List> expectedStringCountKeyValues = Arrays.asList(KeyValue.pair("A", "6"), KeyValue.pair("B", "6"), KeyValue.pair("C", "6")); IntegrationTestUtils.waitUntilFinalKeyValueRecordsReceived(consumerConfig2, COUNT_STRING_TOPIC, expectedStringCountKeyValues); - streams.close(5, TimeUnit.SECONDS); + streams.close(Duration.ofSeconds(5)); } @@ -293,4 +293,4 @@ private List> getKeyValues() { + " Sink: KSTREAM-SINK-0000000019 (topic: outputTopic_1)\n" + " <-- KSTREAM-MAPVALUES-0000000018\n\n"; -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java index 5eb4fc7e5b4c0..3fd82202a2d6f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.integration; +import java.time.Duration; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -113,7 +114,7 @@ private Properties props(final String applicationId) { @After public void shutdown() { if (kafkaStreams != null) { - kafkaStreams.close(30, TimeUnit.SECONDS); + kafkaStreams.close(Duration.ofSeconds(30)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java index 38bc7a68c1f66..646185ebb93c6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java @@ -32,6 +32,7 @@ import java.util.Collections; import java.util.List; +import static java.time.Duration.ofSeconds; /** * Tests all available joins of Kafka Streams DSL. @@ -79,7 +80,7 @@ public void testInner() throws Exception { Arrays.asList("D-a", "D-b", "D-c", "D-d") ); - leftStream.join(rightStream, valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); + leftStream.join(rightStream, valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC); runTest(expectedResult); } @@ -109,7 +110,7 @@ public void testInnerRepartitioned() throws Exception { leftStream.map(MockMapper.noOpKeyValueMapper()) .join(rightStream.flatMap(MockMapper.noOpFlatKeyValueMapper()) .selectKey(MockMapper.selectKeyKeyValueMapper()), - valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); + valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC); runTest(expectedResult); } @@ -136,7 +137,7 @@ public void testLeft() throws Exception { Arrays.asList("D-a", "D-b", "D-c", "D-d") ); - leftStream.leftJoin(rightStream, valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); + leftStream.leftJoin(rightStream, valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC); runTest(expectedResult); } @@ -166,7 +167,7 @@ public void testLeftRepartitioned() throws Exception { leftStream.map(MockMapper.noOpKeyValueMapper()) .leftJoin(rightStream.flatMap(MockMapper.noOpFlatKeyValueMapper()) .selectKey(MockMapper.selectKeyKeyValueMapper()), - valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); + valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC); runTest(expectedResult); } @@ -193,7 +194,7 @@ public void testOuter() throws Exception { Arrays.asList("D-a", "D-b", "D-c", "D-d") ); - leftStream.outerJoin(rightStream, valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); + leftStream.outerJoin(rightStream, valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC); runTest(expectedResult); } @@ -223,7 +224,7 @@ public void testOuterRepartitioned() throws Exception { leftStream.map(MockMapper.noOpKeyValueMapper()) .outerJoin(rightStream.flatMap(MockMapper.noOpFlatKeyValueMapper()) .selectKey(MockMapper.selectKeyKeyValueMapper()), - valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); + valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC); runTest(expectedResult); } @@ -254,8 +255,8 @@ public void testMultiInner() throws Exception { "D-c-b", "D-c-c", "D-c-d", "D-d-a", "D-d-b", "D-d-c", "D-d-d") ); - leftStream.join(rightStream, valueJoiner, JoinWindows.of(10000)) - .join(rightStream, valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); + leftStream.join(rightStream, valueJoiner, JoinWindows.of(ofSeconds(10))) + .join(rightStream, valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC); runTest(expectedResult); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java index 208f1eb3c50bc..94bc0570b0ff0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java @@ -434,7 +434,7 @@ public void shouldSupportFinalResultsForTimeWindows() { Consumed.with(STRING_SERDE, STRING_SERDE) ) .groupBy((String k1, String v1) -> k1, Grouped.with(STRING_SERDE, STRING_SERDE)) - .windowedBy(TimeWindows.of(scaledTime(2L)).grace(scaledTime(1L))) + .windowedBy(TimeWindows.of(ofMillis(scaledTime(2L))).grace(ofMillis(scaledTime(1L)))) .count(Materialized.>as("counts").withCachingDisabled().withLoggingDisabled()); valueCounts @@ -534,4 +534,4 @@ private void verifyOutput(final String topic, final List matched = windows.windowsFor(21L); assertEquals(12L / 5L + 1, matched.size()); assertEquals(new TimeWindow(10L, 22L), matched.get(10L)); @@ -135,7 +136,7 @@ public void shouldComputeWindowsForHoppingWindows() { @Test public void shouldComputeWindowsForBarelyOverlappingHoppingWindows() { - final TimeWindows windows = TimeWindows.of(6L).advanceBy(5L); + final TimeWindows windows = TimeWindows.of(ofMillis(6L)).advanceBy(ofMillis(5L)); final Map matched = windows.windowsFor(7L); assertEquals(1, matched.size()); assertEquals(new TimeWindow(5L, 11L), matched.get(5L)); @@ -143,7 +144,7 @@ public void shouldComputeWindowsForBarelyOverlappingHoppingWindows() { @Test public void shouldComputeWindowsForTumblingWindows() { - final TimeWindows windows = TimeWindows.of(12L); + final TimeWindows windows = TimeWindows.of(ofMillis(12L)); final Map matched = windows.windowsFor(21L); assertEquals(1, matched.size()); assertEquals(new TimeWindow(12L, 24L), matched.get(12L)); @@ -152,49 +153,49 @@ public void shouldComputeWindowsForTumblingWindows() { @Test public void equalsAndHashcodeShouldBeValidForPositiveCases() { - verifyEquality(TimeWindows.of(3), TimeWindows.of(3)); + verifyEquality(TimeWindows.of(ofMillis(3)), TimeWindows.of(ofMillis(3))); - verifyEquality(TimeWindows.of(3).advanceBy(1), TimeWindows.of(3).advanceBy(1)); + verifyEquality(TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1)), TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1))); - verifyEquality(TimeWindows.of(3).grace(1), TimeWindows.of(3).grace(1)); + verifyEquality(TimeWindows.of(ofMillis(3)).grace(ofMillis(1)), TimeWindows.of(ofMillis(3)).grace(ofMillis(1))); - verifyEquality(TimeWindows.of(3).until(4), TimeWindows.of(3).until(4)); + verifyEquality(TimeWindows.of(ofMillis(3)).until(4), TimeWindows.of(ofMillis(3)).until(4)); verifyEquality( - TimeWindows.of(3).advanceBy(1).grace(1).until(4), - TimeWindows.of(3).advanceBy(1).grace(1).until(4) + TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1)).grace(ofMillis(1)).until(4), + TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1)).grace(ofMillis(1)).until(4) ); } @Test public void equalsAndHashcodeShouldBeValidForNegativeCases() { - verifyInEquality(TimeWindows.of(9), TimeWindows.of(3)); + verifyInEquality(TimeWindows.of(ofMillis(9)), TimeWindows.of(ofMillis(3))); - verifyInEquality(TimeWindows.of(3).advanceBy(2), TimeWindows.of(3).advanceBy(1)); + verifyInEquality(TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)), TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1))); - verifyInEquality(TimeWindows.of(3).grace(2), TimeWindows.of(3).grace(1)); + verifyInEquality(TimeWindows.of(ofMillis(3)).grace(ofMillis(2)), TimeWindows.of(ofMillis(3)).grace(ofMillis(1))); - verifyInEquality(TimeWindows.of(3).until(9), TimeWindows.of(3).until(4)); + verifyInEquality(TimeWindows.of(ofMillis(3)).until(9), TimeWindows.of(ofMillis(3)).until(4)); verifyInEquality( - TimeWindows.of(4).advanceBy(2).grace(2).until(4), - TimeWindows.of(3).advanceBy(2).grace(2).until(4) + TimeWindows.of(ofMillis(4)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4), + TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4) ); verifyInEquality( - TimeWindows.of(3).advanceBy(1).grace(2).until(4), - TimeWindows.of(3).advanceBy(2).grace(2).until(4) + TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1)).grace(ofMillis(2)).until(4), + TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4) ); assertNotEquals( - TimeWindows.of(3).advanceBy(2).grace(1).until(4), - TimeWindows.of(3).advanceBy(2).grace(2).until(4) + TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(1)).until(4), + TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4) ); assertNotEquals( - TimeWindows.of(3).advanceBy(2).grace(2).until(9), - TimeWindows.of(3).advanceBy(2).grace(2).until(4) + TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(9), + TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4) ); } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java index a1406547e26b6..023e83fdde13a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java @@ -21,6 +21,7 @@ import java.util.Map; +import static java.time.Instant.ofEpochMilli; import static org.apache.kafka.streams.EqualityCheck.verifyEquality; import static org.apache.kafka.streams.EqualityCheck.verifyInEquality; import static org.junit.Assert.assertEquals; @@ -33,12 +34,12 @@ public class UnlimitedWindowsTest { @Test public void shouldSetWindowStartTime() { - assertEquals(anyStartTime, UnlimitedWindows.of().startOn(anyStartTime).startMs); + assertEquals(anyStartTime, UnlimitedWindows.of().startOn(ofEpochMilli(anyStartTime)).startMs); } @Test(expected = IllegalArgumentException.class) public void startTimeMustNotBeNegative() { - UnlimitedWindows.of().startOn(-1); + UnlimitedWindows.of().startOn(ofEpochMilli(-1)); } @Test @@ -54,7 +55,7 @@ public void shouldThrowOnUntil() { @Test public void shouldIncludeRecordsThatHappenedOnWindowStart() { - final UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime); + final UnlimitedWindows w = UnlimitedWindows.of().startOn(ofEpochMilli(anyStartTime)); final Map matchedWindows = w.windowsFor(w.startMs); assertEquals(1, matchedWindows.size()); assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime)); @@ -62,7 +63,7 @@ public void shouldIncludeRecordsThatHappenedOnWindowStart() { @Test public void shouldIncludeRecordsThatHappenedAfterWindowStart() { - final UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime); + final UnlimitedWindows w = UnlimitedWindows.of().startOn(ofEpochMilli(anyStartTime)); final long timestamp = w.startMs + 1; final Map matchedWindows = w.windowsFor(timestamp); assertEquals(1, matchedWindows.size()); @@ -71,7 +72,7 @@ public void shouldIncludeRecordsThatHappenedAfterWindowStart() { @Test public void shouldExcludeRecordsThatHappenedBeforeWindowStart() { - final UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime); + final UnlimitedWindows w = UnlimitedWindows.of().startOn(ofEpochMilli(anyStartTime)); final long timestamp = w.startMs - 1; final Map matchedWindows = w.windowsFor(timestamp); assertTrue(matchedWindows.isEmpty()); @@ -81,13 +82,13 @@ public void shouldExcludeRecordsThatHappenedBeforeWindowStart() { public void equalsAndHashcodeShouldBeValidForPositiveCases() { verifyEquality(UnlimitedWindows.of(), UnlimitedWindows.of()); - verifyEquality(UnlimitedWindows.of().startOn(1), UnlimitedWindows.of().startOn(1)); + verifyEquality(UnlimitedWindows.of().startOn(ofEpochMilli(1)), UnlimitedWindows.of().startOn(ofEpochMilli(1))); } @Test public void equalsAndHashcodeShouldBeValidForNegativeCases() { - verifyInEquality(UnlimitedWindows.of().startOn(9), UnlimitedWindows.of().startOn(1)); + verifyInEquality(UnlimitedWindows.of().startOn(ofEpochMilli(9)), UnlimitedWindows.of().startOn(ofEpochMilli(1))); } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java index a1f8b27f4d62e..18f36aa8bc580 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java @@ -59,6 +59,7 @@ import java.util.Map; import java.util.Properties; +import static java.time.Duration.ofMillis; import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.hasItem; @@ -96,7 +97,7 @@ public void shouldNotHaveInvalidStoreNameOnReduce() { @Test(expected = NullPointerException.class) public void shouldNotHaveNullReducerWithWindowedReduce() { - groupedStream.windowedBy(TimeWindows.of(10)).reduce(null, Materialized.>as("store")); + groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).reduce(null, Materialized.>as("store")); } @Test(expected = NullPointerException.class) @@ -106,7 +107,7 @@ public void shouldNotHaveNullWindowsWithWindowedReduce() { @Test(expected = InvalidTopicException.class) public void shouldNotHaveInvalidStoreNameWithWindowedReduce() { - groupedStream.windowedBy(TimeWindows.of(10)).reduce(MockReducer.STRING_ADDER, Materialized.>as(INVALID_STORE_NAME)); + groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).reduce(MockReducer.STRING_ADDER, Materialized.>as(INVALID_STORE_NAME)); } @Test(expected = NullPointerException.class) @@ -126,12 +127,12 @@ public void shouldNotHaveInvalidStoreNameOnAggregate() { @Test(expected = NullPointerException.class) public void shouldNotHaveNullInitializerOnWindowedAggregate() { - groupedStream.windowedBy(TimeWindows.of(10)).aggregate(null, MockAggregator.TOSTRING_ADDER, Materialized.>as("store")); + groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).aggregate(null, MockAggregator.TOSTRING_ADDER, Materialized.>as("store")); } @Test(expected = NullPointerException.class) public void shouldNotHaveNullAdderOnWindowedAggregate() { - groupedStream.windowedBy(TimeWindows.of(10)).aggregate(MockInitializer.STRING_INIT, null, Materialized.>as("store")); + groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).aggregate(MockInitializer.STRING_INIT, null, Materialized.>as("store")); } @Test(expected = NullPointerException.class) @@ -141,7 +142,7 @@ public void shouldNotHaveNullWindowsOnWindowedAggregate() { @Test(expected = InvalidTopicException.class) public void shouldNotHaveInvalidStoreNameOnWindowedAggregate() { - groupedStream.windowedBy(TimeWindows.of(10)).aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.>as(INVALID_STORE_NAME)); + groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.>as(INVALID_STORE_NAME)); } private void doAggregateSessionWindows(final Map, Integer> results) { @@ -161,7 +162,7 @@ private void doAggregateSessionWindows(final Map, Integer> resu @Test public void shouldAggregateSessionWindows() { final Map, Integer> results = new HashMap<>(); - final KTable, Integer> table = groupedStream.windowedBy(SessionWindows.with(30)).aggregate(new Initializer() { + final KTable, Integer> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(new Initializer() { @Override public Integer apply() { return 0; @@ -191,7 +192,7 @@ public void apply(final Windowed key, final Integer value) { @Test public void shouldAggregateSessionWindowsWithInternalStoreName() { final Map, Integer> results = new HashMap<>(); - final KTable, Integer> table = groupedStream.windowedBy(SessionWindows.with(30)).aggregate(new Initializer() { + final KTable, Integer> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(new Initializer() { @Override public Integer apply() { return 0; @@ -234,7 +235,7 @@ private void doCountSessionWindows(final Map, Long> results) { @Test public void shouldCountSessionWindows() { final Map, Long> results = new HashMap<>(); - final KTable, Long> table = groupedStream.windowedBy(SessionWindows.with(30)) + final KTable, Long> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))) .count(Materialized.>as("session-store")); table.toStream().foreach(new ForeachAction, Long>() { @Override @@ -249,7 +250,7 @@ public void apply(final Windowed key, final Long value) { @Test public void shouldCountSessionWindowsWithInternalStoreName() { final Map, Long> results = new HashMap<>(); - final KTable, Long> table = groupedStream.windowedBy(SessionWindows.with(30)).count(); + final KTable, Long> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).count(); table.toStream().foreach(new ForeachAction, Long>() { @Override public void apply(final Windowed key, final Long value) { @@ -277,7 +278,7 @@ private void doReduceSessionWindows(final Map, String> results) @Test public void shouldReduceSessionWindows() { final Map, String> results = new HashMap<>(); - final KTable, String> table = groupedStream.windowedBy(SessionWindows.with(30)) + final KTable, String> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))) .reduce(new Reducer() { @Override public String apply(final String value1, final String value2) { @@ -297,7 +298,7 @@ public void apply(final Windowed key, final String value) { @Test public void shouldReduceSessionWindowsWithInternalStoreName() { final Map, String> results = new HashMap<>(); - final KTable, String> table = groupedStream.windowedBy(SessionWindows.with(30)) + final KTable, String> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))) .reduce(new Reducer() { @Override public String apply(final String value1, final String value2) { @@ -316,7 +317,7 @@ public void apply(final Windowed key, final String value) { @Test(expected = NullPointerException.class) public void shouldNotAcceptNullReducerWhenReducingSessionWindows() { - groupedStream.windowedBy(SessionWindows.with(30)).reduce(null, Materialized.>as("store")); + groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).reduce(null, Materialized.>as("store")); } @Test(expected = NullPointerException.class) @@ -326,17 +327,17 @@ public void shouldNotAcceptNullSessionWindowsReducingSessionWindows() { @Test(expected = InvalidTopicException.class) public void shouldNotAcceptInvalidStoreNameWhenReducingSessionWindows() { - groupedStream.windowedBy(SessionWindows.with(30)).reduce(MockReducer.STRING_ADDER, Materialized.>as(INVALID_STORE_NAME)); + groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).reduce(MockReducer.STRING_ADDER, Materialized.>as(INVALID_STORE_NAME)); } @Test(expected = NullPointerException.class) public void shouldNotAcceptNullStateStoreSupplierWhenReducingSessionWindows() { - groupedStream.windowedBy(SessionWindows.with(30)).reduce(null, Materialized.>as(null)); + groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).reduce(null, Materialized.>as(null)); } @Test(expected = NullPointerException.class) public void shouldNotAcceptNullInitializerWhenAggregatingSessionWindows() { - groupedStream.windowedBy(SessionWindows.with(30)).aggregate(null, MockAggregator.TOSTRING_ADDER, new Merger() { + groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(null, MockAggregator.TOSTRING_ADDER, new Merger() { @Override public String apply(final String aggKey, final String aggOne, final String aggTwo) { return null; @@ -346,7 +347,7 @@ public String apply(final String aggKey, final String aggOne, final String aggTw @Test(expected = NullPointerException.class) public void shouldNotAcceptNullAggregatorWhenAggregatingSessionWindows() { - groupedStream.windowedBy(SessionWindows.with(30)).aggregate(MockInitializer.STRING_INIT, null, new Merger() { + groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(MockInitializer.STRING_INIT, null, new Merger() { @Override public String apply(final String aggKey, final String aggOne, final String aggTwo) { return null; @@ -356,7 +357,7 @@ public String apply(final String aggKey, final String aggOne, final String aggTw @Test(expected = NullPointerException.class) public void shouldNotAcceptNullSessionMergerWhenAggregatingSessionWindows() { - groupedStream.windowedBy(SessionWindows.with(30)).aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, + groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, null, Materialized.>as("storeName")); } @@ -368,7 +369,7 @@ public void shouldNotAcceptNullSessionWindowsWhenAggregatingSessionWindows() { @Test public void shouldAcceptNullStoreNameWhenAggregatingSessionWindows() { - groupedStream.windowedBy(SessionWindows.with(10)) + groupedStream.windowedBy(SessionWindows.with(ofMillis(10))) .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, new Merger() { @Override public String apply(final String aggKey, final String aggOne, final String aggTwo) { @@ -379,7 +380,7 @@ public String apply(final String aggKey, final String aggOne, final String aggTw @Test(expected = InvalidTopicException.class) public void shouldNotAcceptInvalidStoreNameWhenAggregatingSessionWindows() { - groupedStream.windowedBy(SessionWindows.with(10)) + groupedStream.windowedBy(SessionWindows.with(ofMillis(10))) .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, new Merger() { @Override public String apply(final String aggKey, final String aggOne, final String aggTwo) { @@ -558,7 +559,7 @@ private void doCountWindowed(final List, Long>> result @Test public void shouldCountWindowed() { final List, Long>> results = new ArrayList<>(); - groupedStream.windowedBy(TimeWindows.of(500L)).count(Materialized.>as("aggregate-by-key-windowed")) + groupedStream.windowedBy(TimeWindows.of(ofMillis(500L))).count(Materialized.>as("aggregate-by-key-windowed")) .toStream() .foreach(new ForeachAction, Long>() { @Override @@ -573,7 +574,7 @@ public void apply(final Windowed key, final Long value) { @Test public void shouldCountWindowedWithInternalStoreName() { final List, Long>> results = new ArrayList<>(); - groupedStream.windowedBy(TimeWindows.of(500L)).count() + groupedStream.windowedBy(TimeWindows.of(ofMillis(500L))).count() .toStream() .foreach(new ForeachAction, Long>() { @Override @@ -584,4 +585,4 @@ public void apply(final Windowed key, final Long value) { doCountWindowed(results); } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index 119a7b72d981f..d033e49f2409b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -63,6 +63,7 @@ import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; +import static java.time.Duration.ofMillis; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.junit.Assert.assertEquals; @@ -162,14 +163,14 @@ public boolean test(final String key, final Integer value) { public Integer apply(final Integer value1, final Integer value2) { return value1 + value2; } - }, JoinWindows.of(anyWindowSize), joined); + }, JoinWindows.of(ofMillis(anyWindowSize)), joined); streams2[1].join(streams3[1], new ValueJoiner() { @Override public Integer apply(final Integer value1, final Integer value2) { return value1 + value2; } - }, JoinWindows.of(anyWindowSize), joined); + }, JoinWindows.of(ofMillis(anyWindowSize)), joined); stream4.to("topic-5"); @@ -383,7 +384,7 @@ public void shouldUseRecordMetadataTimestampExtractorWhenInternalRepartitioningT }); stream.join(kStream, valueJoiner, - JoinWindows.of(windowSize).until(3 * windowSize), + JoinWindows.of(ofMillis(windowSize)).until(3 * windowSize), Joined.with(Serdes.String(), Serdes.String(), Serdes.String())) @@ -418,7 +419,7 @@ public void shouldUseRecordMetadataTimestampExtractorWhenInternalRepartitioningT stream.join( kStream, valueJoiner, - JoinWindows.of(windowSize).grace(3L * windowSize), + JoinWindows.of(ofMillis(windowSize)).grace(ofMillis(3L * windowSize)), Joined.with(Serdes.String(), Serdes.String(), Serdes.String()) ) .to("output-topic", Produced.with(Serdes.String(), Serdes.String())); @@ -537,12 +538,12 @@ public void shouldNotAllowNullProcessSupplier() { @Test(expected = NullPointerException.class) public void shouldNotAllowNullOtherStreamOnJoin() { - testStream.join(null, MockValueJoiner.TOSTRING_JOINER, JoinWindows.of(10)); + testStream.join(null, MockValueJoiner.TOSTRING_JOINER, JoinWindows.of(ofMillis(10))); } @Test(expected = NullPointerException.class) public void shouldNotAllowNullValueJoinerOnJoin() { - testStream.join(testStream, null, JoinWindows.of(10)); + testStream.join(testStream, null, JoinWindows.of(ofMillis(10))); } @Test(expected = NullPointerException.class) @@ -655,12 +656,12 @@ public void shouldThrowNullPointerOnJoinWithTableWhenJoinedIsNull() { @Test(expected = NullPointerException.class) public void shouldThrowNullPointerOnJoinWithStreamWhenJoinedIsNull() { - testStream.join(testStream, MockValueJoiner.TOSTRING_JOINER, JoinWindows.of(10), null); + testStream.join(testStream, MockValueJoiner.TOSTRING_JOINER, JoinWindows.of(ofMillis(10)), null); } @Test(expected = NullPointerException.class) public void shouldThrowNullPointerOnOuterJoinJoinedIsNull() { - testStream.outerJoin(testStream, MockValueJoiner.TOSTRING_JOINER, JoinWindows.of(10), null); + testStream.outerJoin(testStream, MockValueJoiner.TOSTRING_JOINER, JoinWindows.of(ofMillis(10)), null); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index 971ee62d284bd..f2e3cc9b39821 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -41,6 +41,7 @@ import java.util.Properties; import java.util.Set; +import static java.time.Duration.ofMillis; import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; import static org.hamcrest.CoreMatchers.hasItem; import static org.junit.Assert.assertEquals; @@ -71,7 +72,7 @@ public Integer apply(final Integer value1, final Integer value2) { return value1 + value2; } }, - JoinWindows.of(100), + JoinWindows.of(ofMillis(100)), Joined.with(Serdes.String(), Serdes.Integer(), Serdes.Integer()) ); @@ -101,7 +102,7 @@ public void testJoin() { joined = stream1.join( stream2, MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(100), + JoinWindows.of(ofMillis(100)), Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); joined.process(supplier); @@ -204,7 +205,7 @@ public void testOuterJoin() { joined = stream1.outerJoin( stream2, MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(100), + JoinWindows.of(ofMillis(100)), Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); joined.process(supplier); final Collection> copartitionGroups = TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups(); @@ -308,7 +309,7 @@ public void testWindowing() { joined = stream1.join( stream2, MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(100), + JoinWindows.of(ofMillis(100)), Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); joined.process(supplier); @@ -529,7 +530,7 @@ public void testAsymmetricWindowingAfter() { joined = stream1.join( stream2, MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(0).after(100), + JoinWindows.of(ofMillis(0)).after(ofMillis(100)), Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); @@ -640,7 +641,7 @@ public void testAsymmetricWindowingBefore() { joined = stream1.join( stream2, MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(0).before(100), + JoinWindows.of(ofMillis(0)).before(ofMillis(100)), Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); joined.process(supplier); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index 856de3d85c8db..b019411e3ff84 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -39,6 +39,7 @@ import java.util.Properties; import java.util.Set; +import static java.time.Duration.ofMillis; import static org.junit.Assert.assertEquals; public class KStreamKStreamLeftJoinTest { @@ -65,7 +66,7 @@ public void testLeftJoin() { joined = stream1.leftJoin(stream2, MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(100), + JoinWindows.of(ofMillis(100)), Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); joined.process(supplier); @@ -151,7 +152,7 @@ public void testWindowing() { joined = stream1.leftJoin(stream2, MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(100), + JoinWindows.of(ofMillis(100)), Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); joined.process(supplier); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index c7fd7cdc32a1a..419c861f118af 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -50,6 +50,7 @@ import java.util.Arrays; import java.util.List; +import static java.time.Duration.ofMillis; 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.getMetricByName; @@ -69,7 +70,7 @@ public class KStreamSessionWindowAggregateProcessorTest { private final Merger sessionMerger = (aggKey, aggOne, aggTwo) -> aggOne + aggTwo; private final KStreamSessionWindowAggregate sessionAggregator = new KStreamSessionWindowAggregate<>( - SessionWindows.with(GAP_MS), + SessionWindows.with(ofMillis(GAP_MS)), STORE_NAME, initializer, aggregator, @@ -106,7 +107,7 @@ public void forward(final K key, final V value) { } private void initStore(final boolean enableCaching) { - final StoreBuilder> storeBuilder = Stores.sessionStoreBuilder(Stores.persistentSessionStore(STORE_NAME, GAP_MS * 3), + final StoreBuilder> storeBuilder = Stores.sessionStoreBuilder(Stores.persistentSessionStore(STORE_NAME, ofMillis(GAP_MS * 3)), Serdes.String(), Serdes.Long()) .withLoggingDisabled(); @@ -322,7 +323,7 @@ public void shouldLogAndMeterWhenSkippingLateRecord() { LogCaptureAppender.setClassLoggerToDebug(KStreamSessionWindowAggregate.class); final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(); final Processor processor = new KStreamSessionWindowAggregate<>( - SessionWindows.with(10L).grace(10L), + SessionWindows.with(ofMillis(10L)).grace(ofMillis(10L)), STORE_NAME, initializer, aggregator, @@ -349,4 +350,4 @@ public void shouldLogAndMeterWhenSkippingLateRecord() { assertEquals(1.0, dropMetric.metricValue()); assertThat(appender.getMessages(), hasItem("Skipping record for expired window. key=[A] topic=[topic] partition=[-3] offset=[-2] timestamp=[0] window=[0,0) expiration=[10]")); } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java index a8ee681cb8c5a..74ad19cb2d5ad 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.kstream.internals; +import java.time.Duration; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.Consumed; @@ -112,7 +113,7 @@ public Transformer> get() { @Override public void init(final ProcessorContext context) { - context.schedule(1, PunctuationType.WALL_CLOCK_TIME, new Punctuator() { + context.schedule(Duration.ofMillis(1), PunctuationType.WALL_CLOCK_TIME, new Punctuator() { @Override public void punctuate(final long timestamp) { context.forward(-1, (int) timestamp); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java index 5a295b8a50a83..8ae628472d3ef 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java @@ -47,6 +47,7 @@ import java.util.List; import java.util.Properties; +import static java.time.Duration.ofMillis; 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.getMetricByName; @@ -70,7 +71,7 @@ public void testAggBasic() { final KTable, String> table2 = builder .stream(topic1, Consumed.with(Serdes.String(), Serdes.String())) .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(TimeWindows.of(10).advanceBy(5)) + .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5))) .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.>as("topic1-Canonized").withValueSerde(Serdes.String())); final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); @@ -128,7 +129,7 @@ public void testJoin() { final KTable, String> table1 = builder .stream(topic1, Consumed.with(Serdes.String(), Serdes.String())) .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(TimeWindows.of(10).advanceBy(5)) + .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5))) .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.>as("topic1-Canonized").withValueSerde(Serdes.String())); final MockProcessorSupplier, String> supplier = new MockProcessorSupplier<>(); @@ -137,7 +138,7 @@ public void testJoin() { final KTable, String> table2 = builder .stream(topic2, Consumed.with(Serdes.String(), Serdes.String())) .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(TimeWindows.of(10).advanceBy(5)) + .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5))) .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.>as("topic2-Canonized").withValueSerde(Serdes.String())); table2.toStream().process(supplier); @@ -232,7 +233,7 @@ public void shouldLogAndMeterWhenSkippingNullKey() { final KStream stream1 = builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String())); stream1.groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(TimeWindows.of(10).advanceBy(5)) + .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5))) .aggregate( MockInitializer.STRING_INIT, MockAggregator.toStringInstance("+"), @@ -257,7 +258,7 @@ public void shouldLogAndMeterWhenSkippingExpiredWindow() { final KStream stream1 = builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String())); stream1.groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(TimeWindows.of(10).advanceBy(5).until(100)) + .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5)).until(100)) .aggregate( () -> "", MockAggregator.toStringInstance("+"), @@ -316,7 +317,7 @@ public void shouldLogAndMeterWhenSkippingExpiredWindowByGrace() { final KStream stream1 = builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String())); stream1.groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(TimeWindows.of(10).advanceBy(5).grace(90L)) + .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5)).grace(ofMillis(90L))) .aggregate( () -> "", MockAggregator.toStringInstance("+"), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduceTest.java index 3746ae9a32226..913710fd46045 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduceTest.java @@ -36,6 +36,7 @@ import java.util.Properties; +import static java.time.Duration.ofMillis; 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.getMetricByName; @@ -58,7 +59,7 @@ public void shouldLogAndMeterOnNullKey() { builder .stream("TOPIC", Consumed.with(Serdes.String(), Serdes.String())) .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(TimeWindows.of(500L)) + .windowedBy(TimeWindows.of(ofMillis(500L))) .reduce((value1, value2) -> value1 + "+" + value2); @@ -80,7 +81,7 @@ public void shouldLogAndMeterOnExpiredEvent() { builder .stream("TOPIC", Consumed.with(Serdes.String(), Serdes.String())) .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(TimeWindows.of(5L).until(100)) + .windowedBy(TimeWindows.of(ofMillis(5L)).until(100)) .reduce((value1, value2) -> value1 + "+" + value2) .toStream() .map((key, value) -> new KeyValue<>(key.toString(), value)) 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 34a235ac482e7..9ee918c1c4b76 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 @@ -47,6 +47,7 @@ import java.util.Map; import java.util.Properties; +import static java.time.Duration.ofMillis; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; @@ -69,7 +70,7 @@ public String apply(final String aggKey, final String aggOne, final String aggTw public void before() { final KStream stream = builder.stream(TOPIC, Consumed.with(Serdes.String(), Serdes.String())); this.stream = stream.groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(SessionWindows.with(500)); + .windowedBy(SessionWindows.with(ofMillis(500))); } @Test @@ -260,4 +261,4 @@ private void processData(final TopologyTestDriver driver) { driver.pipeInput(recordFactory.create(TOPIC, "2", "1", 600)); } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java index 04b7ab8662169..65c51fc5f2843 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java @@ -368,7 +368,7 @@ public void shouldSupportFinalResultsForTimeWindows() { final KTable, Long> valueCounts = builder .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE)) .groupBy((String k, String v) -> k, Grouped.with(STRING_SERDE, STRING_SERDE)) - .windowedBy(TimeWindows.of(2L).grace(1L)) + .windowedBy(TimeWindows.of(2L).grace(ofMillis(1L))) .count(Materialized.>as("counts").withCachingDisabled()); valueCounts .suppress(untilWindowCloses(unbounded())) @@ -422,7 +422,7 @@ public void shouldSupportFinalResultsForTimeWindowsWithLargeJump() { final KTable, Long> valueCounts = builder .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE)) .groupBy((String k, String v) -> k, Grouped.with(STRING_SERDE, STRING_SERDE)) - .windowedBy(TimeWindows.of(2L).grace(2L)) + .windowedBy(TimeWindows.of(2L).grace(ofMillis(2L))) .count(Materialized.>as("counts").withCachingDisabled().withKeySerde(STRING_SERDE)); valueCounts .suppress(untilWindowCloses(unbounded())) @@ -481,7 +481,7 @@ public void shouldSupportFinalResultsForSessionWindows() { final KTable, Long> valueCounts = builder .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE)) .groupBy((String k, String v) -> k, Grouped.with(STRING_SERDE, STRING_SERDE)) - .windowedBy(SessionWindows.with(5L).grace(5L)) + .windowedBy(SessionWindows.with(5L).grace(ofMillis(5L))) .count(Materialized.>as("counts").withCachingDisabled()); valueCounts .suppress(untilWindowCloses(unbounded())) diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressTopologyTest.java index 70c25c7506001..97bdb1d39ec66 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressTopologyTest.java @@ -32,6 +32,7 @@ import java.time.Duration; +import static java.time.Duration.ofMillis; import static org.apache.kafka.streams.kstream.Suppressed.BufferConfig.unbounded; import static org.apache.kafka.streams.kstream.Suppressed.untilTimeLimit; import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; @@ -146,7 +147,7 @@ public void shouldUseNumberingForAnonymousFinalSuppressionNode() { anonymousNodeBuilder .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE)) .groupBy((String k, String v) -> k, Grouped.with(STRING_SERDE, STRING_SERDE)) - .windowedBy(SessionWindows.with(5L).grace(5L)) + .windowedBy(SessionWindows.with(ofMillis(5L)).grace(ofMillis(5L))) .count(Materialized.>as("counts").withCachingDisabled()) .suppress(untilWindowCloses(unbounded())) .toStream() @@ -164,7 +165,7 @@ public void shouldApplyNameToFinalSuppressionNode() { namedNodeBuilder .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE)) .groupBy((String k, String v) -> k, Grouped.with(STRING_SERDE, STRING_SERDE)) - .windowedBy(SessionWindows.with(5L).grace(5L)) + .windowedBy(SessionWindows.with(ofMillis(5L)).grace(ofMillis(5L))) .count(Materialized.>as("counts").withCachingDisabled()) .suppress(untilWindowCloses(unbounded()).withName("myname")) .toStream() diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java index f260bee62ef68..f6e06e421c5c2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java @@ -21,6 +21,7 @@ import java.util.Map; +import static java.time.Duration.ofMillis; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -124,7 +125,7 @@ public void cannotCompareTimeWindowWithDifferentWindowType() { @Test public void shouldReturnMatchedWindowsOrderedByTimestamp() { - final TimeWindows windows = TimeWindows.of(12L).advanceBy(5L); + final TimeWindows windows = TimeWindows.of(ofMillis(12L)).advanceBy(ofMillis(5L)); final Map matched = windows.windowsFor(21L); final Long[] expected = matched.keySet().toArray(new Long[matched.size()]); @@ -132,4 +133,4 @@ public void shouldReturnMatchedWindowsOrderedByTimestamp() { assertEquals(expected[1].longValue(), 15L); assertEquals(expected[2].longValue(), 20L); } -} \ No newline at end of file +} 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 0e541c91acc57..f951ebb415504 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 @@ -46,6 +46,8 @@ import java.util.Map; import java.util.Properties; +import static java.time.Duration.ofMillis; +import static java.time.Instant.ofEpochMilli; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; @@ -61,7 +63,7 @@ public class TimeWindowedKStreamImplTest { public void before() { final KStream stream = builder.stream(TOPIC, Consumed.with(Serdes.String(), Serdes.String())); windowedStream = stream.groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(TimeWindows.of(500L)); + .windowedBy(TimeWindows.of(ofMillis(500L))); } @Test @@ -136,7 +138,7 @@ public void shouldMaterializeCount() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) { processData(driver); final WindowStore windowStore = driver.getWindowStore("count-store"); - final List, Long>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", 0, 1000)); + final List, Long>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); assertThat(data, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), 2L), @@ -155,7 +157,7 @@ public void shouldMaterializeReduced() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) { processData(driver); final WindowStore windowStore = driver.getWindowStore("reduced"); - final List, String>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", 0, 1000)); + final List, String>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); assertThat(data, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), "1+2"), @@ -175,7 +177,7 @@ public void shouldMaterializeAggregated() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) { processData(driver); final WindowStore windowStore = driver.getWindowStore("aggregated"); - final List, String>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", 0, 1000)); + final List, String>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); assertThat(data, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), "0+1+2"), KeyValue.pair(new Windowed<>("1", new TimeWindow(500, 1000)), "0+3"), @@ -244,4 +246,4 @@ private void processData(final TopologyTestDriver driver) { driver.pipeInput(recordFactory.create(TOPIC, "2", "1", 500L)); } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java index 2b054230839ec..37265fae39917 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.junit.Test; +import static java.time.Duration.ofMillis; import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; @@ -78,7 +79,7 @@ public void close() {} @Test public void shouldExtractGraceFromKStreamWindowAggregateNode() { - final TimeWindows windows = TimeWindows.of(10L).grace(1234L); + final TimeWindows windows = TimeWindows.of(ofMillis(10L)).grace(ofMillis(1234L)); final StatefulProcessorNode node = new StatefulProcessorNode<>( "asdf", new ProcessorParameters<>( @@ -101,7 +102,7 @@ public void shouldExtractGraceFromKStreamWindowAggregateNode() { @Test public void shouldExtractGraceFromKStreamSessionWindowAggregateNode() { - final SessionWindows windows = SessionWindows.with(10L).grace(1234L); + final SessionWindows windows = SessionWindows.with(ofMillis(10L)).grace(ofMillis(1234L)); final StatefulProcessorNode node = new StatefulProcessorNode<>( "asdf", @@ -126,7 +127,7 @@ public void shouldExtractGraceFromKStreamSessionWindowAggregateNode() { @Test public void shouldExtractGraceFromAncestorThroughStatefulParent() { - final SessionWindows windows = SessionWindows.with(10L).grace(1234L); + final SessionWindows windows = SessionWindows.with(ofMillis(10L)).grace(ofMillis(1234L)); final StatefulProcessorNode graceGrandparent = new StatefulProcessorNode<>( "asdf", new ProcessorParameters<>(new KStreamSessionWindowAggregate( @@ -167,7 +168,7 @@ public void close() {} @Test public void shouldExtractGraceFromAncestorThroughStatelessParent() { - final SessionWindows windows = SessionWindows.with(10L).grace(1234L); + final SessionWindows windows = SessionWindows.with(ofMillis(10L)).grace(ofMillis(1234L)); final StatefulProcessorNode graceGrandparent = new StatefulProcessorNode<>( "asdf", new ProcessorParameters<>( @@ -201,7 +202,7 @@ public void shouldUseMaxIfMultiParentsDoNotAgreeOnGrace() { "asdf", new ProcessorParameters<>( new KStreamSessionWindowAggregate( - SessionWindows.with(10L).grace(1234L), + SessionWindows.with(ofMillis(10L)).grace(ofMillis(1234L)), "asdf", null, null, @@ -218,7 +219,7 @@ public void shouldUseMaxIfMultiParentsDoNotAgreeOnGrace() { "asdf", new ProcessorParameters<>( new KStreamWindowAggregate( - TimeWindows.of(10L).grace(4321L), + TimeWindows.of(ofMillis(10L)).grace(ofMillis(4321L)), "asdf", null, null diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java index 75e9f5120b033..bd43685a3c5cd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java @@ -34,6 +34,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +import static java.time.Duration.ofMillis; import static org.junit.Assert.assertEquals; public class StreamsGraphTest { @@ -51,7 +52,7 @@ public void shouldBeAbleToBuildTopologyIncrementally() { final ValueJoiner valueJoiner = (v, v2) -> v + v2; - final KStream joinedStream = stream.join(streamII, valueJoiner, JoinWindows.of(5000)); + final KStream joinedStream = stream.join(streamII, valueJoiner, JoinWindows.of(ofMillis(5000))); // build step one assertEquals(expectedJoinedTopology, builder.build().describe().toString()); @@ -100,7 +101,7 @@ private Topology getTopologyWithChangingValuesAfterChangingKey(final String opti final KStream mappedKeyStream = inputStream.selectKey((k, v) -> k + v); mappedKeyStream.mapValues(v -> v.toUpperCase(Locale.getDefault())).groupByKey().count().toStream().to("output"); - mappedKeyStream.flatMapValues(v -> Arrays.asList(v.split("\\s"))).groupByKey().windowedBy(TimeWindows.of(5000)).count().toStream().to("windowed-output"); + mappedKeyStream.flatMapValues(v -> Arrays.asList(v.split("\\s"))).groupByKey().windowedBy(TimeWindows.of(ofMillis(5000))).count().toStream().to("windowed-output"); return builder.build(properties); @@ -116,7 +117,7 @@ private Topology getTopologyWithThroughOperation(final String optimizeConfig) { final KStream mappedKeyStream = inputStream.selectKey((k, v) -> k + v).through("through-topic"); mappedKeyStream.groupByKey().count().toStream().to("output"); - mappedKeyStream.groupByKey().windowedBy(TimeWindows.of(5000)).count().toStream().to("windowed-output"); + mappedKeyStream.groupByKey().windowedBy(TimeWindows.of(ofMillis(5000))).count().toStream().to("windowed-output"); return builder.build(properties); diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java index 0fa058301ecb0..4008689d3efb5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java +++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java @@ -54,7 +54,6 @@ import org.apache.kafka.streams.state.WindowStore; import java.io.IOException; -import java.time.Duration; import java.util.ArrayList; import java.util.List; import java.util.Locale; @@ -63,6 +62,10 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import static java.time.Duration.ofMillis; +import static java.time.Duration.ofSeconds; +import static java.time.Instant.ofEpochMilli; + /** * Class that provides support for a series of benchmarks. It is usually driven by * tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py. @@ -332,7 +335,7 @@ private void consumeAndProduce(final String topic) { consumer.seekToBeginning(partitions); while (true) { - final ConsumerRecords records = consumer.poll(Duration.ofMillis(POLL_MS)); + final ConsumerRecords records = consumer.poll(ofMillis(POLL_MS)); if (records.isEmpty()) { if (processedRecords == numRecords) { break; @@ -370,7 +373,7 @@ private void consume(final String topic) { consumer.seekToBeginning(partitions); while (true) { - final ConsumerRecords records = consumer.poll(Duration.ofMillis(POLL_MS)); + final ConsumerRecords records = consumer.poll(ofMillis(POLL_MS)); if (records.isEmpty()) { if (processedRecords == numRecords) { break; @@ -498,7 +501,7 @@ public void init(final ProcessorContext context) { @Override public void process(final Integer key, final byte[] value) { final long timestamp = context().timestamp(); - final KeyValueIterator, byte[]> iter = store.fetch(key - 10, key + 10, timestamp - 1000L, timestamp + 1000L); + final KeyValueIterator, byte[]> iter = store.fetch(key - 10, key + 10, ofEpochMilli(timestamp - 1000L), ofEpochMilli(timestamp)); while (iter.hasNext()) { iter.next(); } @@ -550,7 +553,7 @@ private void countStreamsWindowed(final String sourceTopic) { input.peek(new CountDownAction(latch)) .groupByKey() - .windowedBy(TimeWindows.of(AGGREGATE_WINDOW_SIZE).advanceBy(AGGREGATE_WINDOW_ADVANCE)) + .windowedBy(TimeWindows.of(ofMillis(AGGREGATE_WINDOW_SIZE)).advanceBy(ofMillis(AGGREGATE_WINDOW_ADVANCE))) .count(); final KafkaStreams streams = createKafkaStreamsWithExceptionHandler(builder, props); @@ -593,7 +596,7 @@ private void streamStreamJoin(final String kStreamTopic1, final String kStreamTo final KStream input1 = builder.stream(kStreamTopic1); final KStream input2 = builder.stream(kStreamTopic2); - input1.leftJoin(input2, VALUE_JOINER, JoinWindows.of(STREAM_STREAM_JOIN_WINDOW)).foreach(new CountDownAction(latch)); + input1.leftJoin(input2, VALUE_JOINER, JoinWindows.of(ofMillis(STREAM_STREAM_JOIN_WINDOW))).foreach(new CountDownAction(latch)); final KafkaStreams streams = createKafkaStreamsWithExceptionHandler(builder, props); @@ -677,7 +680,7 @@ private KafkaStreams createKafkaStreamsWithExceptionHandler(final StreamsBuilder public void uncaughtException(final Thread t, final Throwable e) { System.out.println("FATAL: An unexpected exception is encountered on thread " + t + ": " + e); - streamsClient.close(30, TimeUnit.SECONDS); + streamsClient.close(ofSeconds(30)); } }); diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java index 6e09ad1319ca3..1109f9d9be391 100644 --- a/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java +++ b/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.perf; import com.fasterxml.jackson.databind.ObjectMapper; +import java.time.Duration; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; @@ -334,7 +335,7 @@ public String apply(final String key, final String value) { // calculate windowed counts keyedByCampaign .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) - .windowedBy(TimeWindows.of(10 * 1000)) + .windowedBy(TimeWindows.of(Duration.ofMillis(10 * 1000))) .count(Materialized.>as("time-windows")); return new KafkaStreams(builder.build(), streamConfig); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java index 070dba8efa06c..4ce9a9f4f177c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.time.Duration; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeader; @@ -202,6 +203,13 @@ public Cancellable schedule(final long interval, final PunctuationType type, fin return null; } + @Override + public Cancellable schedule(final Duration interval, + final PunctuationType type, + final Punctuator callback) throws IllegalArgumentException { + return null; + } + @Override public void forward(final K key, final V value) {} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index daf1f33c60979..8ddb0b5004076 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -47,6 +47,7 @@ import java.util.Set; import java.util.regex.Pattern; +import static java.time.Duration.ofSeconds; import static org.apache.kafka.common.utils.Utils.mkList; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.hamcrest.core.IsInstanceOf.instanceOf; @@ -588,7 +589,7 @@ public void shouldAddInternalTopicConfigForWindowStores() { builder.addProcessor("processor", new MockProcessorSupplier(), "source"); builder.addStateStore( Stores.windowStoreBuilder( - Stores.persistentWindowStore("store1", 30_000L, 10_000L, false), + Stores.persistentWindowStore("store1", ofSeconds(30L), ofSeconds(10L), false), Serdes.String(), Serdes.String() ), @@ -596,7 +597,7 @@ public void shouldAddInternalTopicConfigForWindowStores() { ); builder.addStateStore( Stores.sessionStoreBuilder( - Stores.persistentSessionStore("store2", 30000), Serdes.String(), Serdes.String() + Stores.persistentSessionStore("store2", ofSeconds(30)), Serdes.String(), Serdes.String() ), "processor" ); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index e66987956fd33..639ebf861f1df 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -72,6 +72,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import static java.time.Duration.ofMillis; import static java.util.Collections.emptyList; import static java.util.Collections.emptySet; import static java.util.Collections.singletonList; @@ -211,7 +212,7 @@ public void testUpdate() throws IOException { } restoreStateConsumer.seekToBeginning(partition); - task.update(partition2, restoreStateConsumer.poll(Duration.ofMillis(100)).records(partition2)); + task.update(partition2, restoreStateConsumer.poll(ofMillis(100)).records(partition2)); final StandbyContextImpl context = (StandbyContextImpl) task.context(); final MockKeyValueStore store1 = (MockKeyValueStore) context.getStateMgr().getStore(storeName1); @@ -239,8 +240,8 @@ public void shouldRestoreToWindowedStores() throws IOException { builder .stream(Collections.singleton("topic"), new ConsumedInternal<>()) .groupByKey() - .windowedBy(TimeWindows.of(60_000).grace(0L)) - .count(Materialized.>as(storeName).withRetention(120_000L)); + .windowedBy(TimeWindows.of(ofMillis(60_000)).grace(ofMillis(0L))) + .count(Materialized.>as(storeName).withRetention(ofMillis(120_000L))); builder.buildAndOptimizeTopology(); @@ -484,7 +485,7 @@ public void shouldInitializeStateStoreWithoutException() throws IOException { @Test public void shouldInitializeWindowStoreWithoutException() throws IOException { final InternalStreamsBuilder builder = new InternalStreamsBuilder(new InternalTopologyBuilder()); - builder.stream(Collections.singleton("topic"), new ConsumedInternal<>()).groupByKey().windowedBy(TimeWindows.of(100)).count(); + builder.stream(Collections.singleton("topic"), new ConsumedInternal<>()).groupByKey().windowedBy(TimeWindows.of(ofMillis(100))).count(); initializeStandbyStores(builder); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index e691c54d8ebe0..b4de5ec005706 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.time.Duration; import org.apache.kafka.clients.admin.MockAdminClient; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; @@ -1040,13 +1041,13 @@ public Processor get() { return new Processor() { @Override public void init(final ProcessorContext context) { - context.schedule(100L, PunctuationType.STREAM_TIME, new Punctuator() { + context.schedule(Duration.ofMillis(100L), PunctuationType.STREAM_TIME, new Punctuator() { @Override public void punctuate(final long timestamp) { punctuatedStreamTime.add(timestamp); } }); - context.schedule(100L, PunctuationType.WALL_CLOCK_TIME, new Punctuator() { + context.schedule(Duration.ofMillis(100L), PunctuationType.WALL_CLOCK_TIME, new Punctuator() { @Override public void punctuate(final long timestamp) { punctuatedWallClockTime.add(timestamp); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java index 610296973a0e3..649aa191ee0f5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java @@ -60,6 +60,7 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; +import static java.time.Duration.ofMillis; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; import static org.junit.Assert.assertEquals; @@ -961,7 +962,7 @@ public Object apply(final Object value1, final Object value2) { return null; } }, - JoinWindows.of(0) + JoinWindows.of(ofMillis(0)) ); final UUID uuid = UUID.randomUUID(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java index 8b20b0bc542cd..34d9050c74870 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.state; +import java.time.Instant; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; @@ -88,22 +89,43 @@ public Object fetch(final Object key, final long time) { } @Override + @SuppressWarnings("deprecation") public WindowStoreIterator fetch(final Object key, final long timeFrom, final long timeTo) { return EMPTY_WINDOW_STORE_ITERATOR; } @Override + public WindowStoreIterator fetch(final Object key, final Instant from, final Instant to) { + return EMPTY_WINDOW_STORE_ITERATOR; + } + + @Override + @SuppressWarnings("deprecation") public WindowStoreIterator fetch(final Object from, final Object to, final long timeFrom, final long timeTo) { return EMPTY_WINDOW_STORE_ITERATOR; } - + + @Override + public KeyValueIterator fetch(final Object from, + final Object to, + final Instant fromTime, + final Instant toTime) throws IllegalArgumentException { + return EMPTY_WINDOW_STORE_ITERATOR; + } + @Override public WindowStoreIterator all() { return EMPTY_WINDOW_STORE_ITERATOR; } @Override + @SuppressWarnings("deprecation") public WindowStoreIterator fetchAll(final long timeFrom, final long timeTo) { return EMPTY_WINDOW_STORE_ITERATOR; } + + @Override + public KeyValueIterator fetchAll(final Instant from, final Instant to) { + return EMPTY_WINDOW_STORE_ITERATOR; + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java index d0da15880cc19..b62364a4a2f3b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.streams.state.internals.RocksDBWindowStore; import org.junit.Test; +import static java.time.Duration.ofMillis; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsInstanceOf.instanceOf; @@ -70,7 +71,7 @@ public void shouldThrowIfIPersistentWindowStoreIfNumberOfSegmentsSmallerThanOne( @Test(expected = IllegalArgumentException.class) public void shouldThrowIfIPersistentWindowStoreIfWindowSizeIsNegative() { - Stores.persistentWindowStore("anyName", 0L, -1L, false); + Stores.persistentWindowStore("anyName", ofMillis(0L), ofMillis(-1L), false); } @Test(expected = IllegalArgumentException.class) @@ -80,12 +81,13 @@ public void shouldThrowIfIPersistentWindowStoreIfSegmentIntervalIsTooSmall() { @Test(expected = NullPointerException.class) public void shouldThrowIfIPersistentSessionStoreStoreNameIsNull() { - Stores.persistentSessionStore(null, 0); + Stores.persistentSessionStore(null, ofMillis(0)); + } @Test(expected = IllegalArgumentException.class) public void shouldThrowIfIPersistentSessionStoreRetentionPeriodIsNegative() { - Stores.persistentSessionStore("anyName", -1); + Stores.persistentSessionStore("anyName", ofMillis(-1)); } @Test(expected = NullPointerException.class) @@ -120,18 +122,18 @@ public void shouldCreateRocksDbStore() { @Test public void shouldCreateRocksDbWindowStore() { - assertThat(Stores.persistentWindowStore("store", 1L, 1L, false).get(), instanceOf(RocksDBWindowStore.class)); + assertThat(Stores.persistentWindowStore("store", ofMillis(1L), ofMillis(1L), false).get(), instanceOf(RocksDBWindowStore.class)); } @Test public void shouldCreateRocksDbSessionStore() { - assertThat(Stores.persistentSessionStore("store", 1).get(), instanceOf(RocksDBSessionStore.class)); + assertThat(Stores.persistentSessionStore("store", ofMillis(1)).get(), instanceOf(RocksDBSessionStore.class)); } @Test public void shouldBuildWindowStore() { final WindowStore store = Stores.windowStoreBuilder( - Stores.persistentWindowStore("store", 3L, 3L, true), + Stores.persistentWindowStore("store", ofMillis(3L), ofMillis(3L), true), Serdes.String(), Serdes.String() ).build(); @@ -151,10 +153,10 @@ public void shouldBuildKeyValueStore() { @Test public void shouldBuildSessionStore() { final SessionStore store = Stores.sessionStoreBuilder( - Stores.persistentSessionStore("name", 10), + Stores.persistentSessionStore("name", ofMillis(10)), Serdes.String(), Serdes.String() ).build(); assertThat(store, not(nullValue())); } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java index 1c8dd7b8a5aeb..71147319fb3ff 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java @@ -49,6 +49,9 @@ import java.util.Properties; import java.util.UUID; +import static java.time.Duration.ofHours; +import static java.time.Duration.ofMinutes; +import static java.time.Instant.ofEpochMilli; import static org.apache.kafka.common.utils.Utils.mkList; import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize; import static org.apache.kafka.test.StreamsTestUtils.toList; @@ -101,7 +104,7 @@ public void shouldNotReturnDuplicatesInRanges() { final StreamsBuilder builder = new StreamsBuilder(); final StoreBuilder> storeBuilder = Stores.windowStoreBuilder( - Stores.persistentWindowStore("store-name", 3600000L, 60000L, false), + Stores.persistentWindowStore("store-name", ofHours(1L), ofMinutes(1L), false), Serdes.String(), Serdes.String()) .withCachingEnabled(); @@ -197,8 +200,8 @@ public void shouldPutFetchFromCache() { assertThat(cachingStore.fetch(bytesKey("c"), 10), equalTo(null)); assertThat(cachingStore.fetch(bytesKey("a"), 0), equalTo(null)); - final WindowStoreIterator a = cachingStore.fetch(bytesKey("a"), 10, 10); - final WindowStoreIterator b = cachingStore.fetch(bytesKey("b"), 10, 10); + final WindowStoreIterator a = cachingStore.fetch(bytesKey("a"), ofEpochMilli(10), ofEpochMilli(10)); + final WindowStoreIterator b = cachingStore.fetch(bytesKey("b"), ofEpochMilli(10), ofEpochMilli(10)); verifyKeyValue(a.next(), DEFAULT_TIMESTAMP, "a"); verifyKeyValue(b.next(), DEFAULT_TIMESTAMP, "b"); assertFalse(a.hasNext()); @@ -224,7 +227,7 @@ public void shouldPutFetchRangeFromCache() { cachingStore.put(bytesKey("a"), bytesValue("a")); cachingStore.put(bytesKey("b"), bytesValue("b")); - final KeyValueIterator, byte[]> iterator = cachingStore.fetch(bytesKey("a"), bytesKey("b"), 10, 10); + final KeyValueIterator, byte[]> iterator = cachingStore.fetch(bytesKey("a"), bytesKey("b"), ofEpochMilli(10), ofEpochMilli(10)); verifyWindowedKeyValue(iterator.next(), new Windowed<>(bytesKey("a"), new TimeWindow(DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE)), "a"); verifyWindowedKeyValue(iterator.next(), new Windowed<>(bytesKey("b"), new TimeWindow(DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE)), "b"); assertFalse(iterator.hasNext()); @@ -258,21 +261,21 @@ public void shouldFetchAllWithinTimestampRange() { cachingStore.put(bytesKey(array[i]), bytesValue(array[i])); } - final KeyValueIterator, byte[]> iterator = cachingStore.fetchAll(0, 7); + final KeyValueIterator, byte[]> iterator = cachingStore.fetchAll(ofEpochMilli(0), ofEpochMilli(7)); for (int i = 0; i < array.length; i++) { final String str = array[i]; verifyWindowedKeyValue(iterator.next(), new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), str); } assertFalse(iterator.hasNext()); - final KeyValueIterator, byte[]> iterator1 = cachingStore.fetchAll(2, 4); + final KeyValueIterator, byte[]> iterator1 = cachingStore.fetchAll(ofEpochMilli(2), ofEpochMilli(4)); for (int i = 2; i <= 4; i++) { final String str = array[i]; verifyWindowedKeyValue(iterator1.next(), new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), str); } assertFalse(iterator1.hasNext()); - final KeyValueIterator, byte[]> iterator2 = cachingStore.fetchAll(5, 7); + final KeyValueIterator, byte[]> iterator2 = cachingStore.fetchAll(ofEpochMilli(5), ofEpochMilli(7)); for (int i = 5; i <= 7; i++) { final String str = array[i]; verifyWindowedKeyValue(iterator2.next(), new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), str); @@ -336,7 +339,7 @@ public void shouldTakeValueFromCacheIfSameTimestampFlushedToRocks() { cachingStore.flush(); cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP); - final WindowStoreIterator fetch = cachingStore.fetch(bytesKey("1"), DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP); + final WindowStoreIterator fetch = cachingStore.fetch(bytesKey("1"), ofEpochMilli(DEFAULT_TIMESTAMP), ofEpochMilli(DEFAULT_TIMESTAMP)); verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP, "b"); assertFalse(fetch.hasNext()); } @@ -346,7 +349,7 @@ public void shouldIterateAcrossWindows() { cachingStore.put(bytesKey("1"), bytesValue("a"), DEFAULT_TIMESTAMP); cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE); - final WindowStoreIterator fetch = cachingStore.fetch(bytesKey("1"), DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE); + final WindowStoreIterator fetch = cachingStore.fetch(bytesKey("1"), ofEpochMilli(DEFAULT_TIMESTAMP), ofEpochMilli(DEFAULT_TIMESTAMP + WINDOW_SIZE)); verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP, "a"); verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP + WINDOW_SIZE, "b"); assertFalse(fetch.hasNext()); @@ -357,7 +360,7 @@ public void shouldIterateCacheAndStore() { final Bytes key = Bytes.wrap("1".getBytes()); underlying.put(WindowKeySchema.toStoreKeyBinary(key, DEFAULT_TIMESTAMP, 0), "a".getBytes()); cachingStore.put(key, bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE); - final WindowStoreIterator fetch = cachingStore.fetch(bytesKey("1"), DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE); + final WindowStoreIterator fetch = cachingStore.fetch(bytesKey("1"), ofEpochMilli(DEFAULT_TIMESTAMP), ofEpochMilli(DEFAULT_TIMESTAMP + WINDOW_SIZE)); verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP, "a"); verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP + WINDOW_SIZE, "b"); assertFalse(fetch.hasNext()); @@ -370,7 +373,7 @@ public void shouldIterateCacheAndStoreKeyRange() { cachingStore.put(key, bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE); final KeyValueIterator, byte[]> fetchRange = - cachingStore.fetch(key, bytesKey("2"), DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE); + cachingStore.fetch(key, bytesKey("2"), ofEpochMilli(DEFAULT_TIMESTAMP), ofEpochMilli(DEFAULT_TIMESTAMP + WINDOW_SIZE)); verifyWindowedKeyValue(fetchRange.next(), new Windowed<>(key, new TimeWindow(DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE)), "a"); verifyWindowedKeyValue(fetchRange.next(), new Windowed<>(key, new TimeWindow(DEFAULT_TIMESTAMP + WINDOW_SIZE, DEFAULT_TIMESTAMP + WINDOW_SIZE + WINDOW_SIZE)), "b"); assertFalse(fetchRange.hasNext()); @@ -387,13 +390,13 @@ public void shouldClearNamespaceCacheOnClose() { @Test(expected = InvalidStateStoreException.class) public void shouldThrowIfTryingToFetchFromClosedCachingStore() { cachingStore.close(); - cachingStore.fetch(bytesKey("a"), 0, 10); + cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(10)); } @Test(expected = InvalidStateStoreException.class) public void shouldThrowIfTryingToFetchRangeFromClosedCachingStore() { cachingStore.close(); - cachingStore.fetch(bytesKey("a"), bytesKey("b"), 0, 10); + cachingStore.fetch(bytesKey("a"), bytesKey("b"), ofEpochMilli(0), ofEpochMilli(10)); } @Test(expected = InvalidStateStoreException.class) @@ -415,7 +418,7 @@ public void shouldFetchAndIterateOverExactKeys() { KeyValue.pair(1L, bytesValue("0003")), KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")) ); - final List> actual = toList(cachingStore.fetch(bytesKey("a"), 0, Long.MAX_VALUE)); + final List> actual = toList(cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); verifyKeyValueList(expected, actual); } @@ -433,12 +436,12 @@ public void shouldFetchAndIterateOverKeyRange() { windowedPair("a", "0003", 1), windowedPair("a", "0005", SEGMENT_INTERVAL) ), - toList(cachingStore.fetch(bytesKey("a"), bytesKey("a"), 0, Long.MAX_VALUE)) + toList(cachingStore.fetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) ); verifyKeyValueList( mkList(windowedPair("aa", "0002", 0), windowedPair("aa", "0004", 1)), - toList(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), 0, Long.MAX_VALUE)) + toList(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) ); verifyKeyValueList( @@ -449,7 +452,7 @@ public void shouldFetchAndIterateOverKeyRange() { windowedPair("aa", "0004", 1), windowedPair("a", "0005", SEGMENT_INTERVAL) ), - toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), 0, Long.MAX_VALUE)) + toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) ); } @@ -465,17 +468,17 @@ public void shouldNotThrowNullPointerExceptionOnPutNullValue() { @Test(expected = NullPointerException.class) public void shouldThrowNullPointerExceptionOnFetchNullKey() { - cachingStore.fetch(null, 1L, 2L); + cachingStore.fetch(null, ofEpochMilli(1L), ofEpochMilli(2L)); } @Test(expected = NullPointerException.class) public void shouldThrowNullPointerExceptionOnRangeNullFromKey() { - cachingStore.fetch(null, bytesKey("anyTo"), 1L, 2L); + cachingStore.fetch(null, bytesKey("anyTo"), ofEpochMilli(1L), ofEpochMilli(2L)); } @Test(expected = NullPointerException.class) public void shouldThrowNullPointerExceptionOnRangeNullToKey() { - cachingStore.fetch(bytesKey("anyFrom"), null, 1L, 2L); + cachingStore.fetch(bytesKey("anyFrom"), null, ofEpochMilli(1L), ofEpochMilli(2L)); } private static KeyValue, byte[]> windowedPair(final String key, final String value, final long timestamp) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java index ec81b93ae502e..6d2d994c1c6e6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java @@ -36,6 +36,7 @@ import java.util.HashMap; import java.util.Map; +import static java.time.Instant.ofEpochMilli; import static org.junit.Assert.assertArrayEquals; @RunWith(EasyMockRunner.class) @@ -101,7 +102,7 @@ public void shouldDelegateToUnderlyingStoreWhenFetching() { init(); - store.fetch(bytesKey, 0, 10); + store.fetch(bytesKey, ofEpochMilli(0), ofEpochMilli(10)); EasyMock.verify(inner); } @@ -111,7 +112,7 @@ public void shouldDelegateToUnderlyingStoreWhenFetchingRange() { init(); - store.fetch(bytesKey, bytesKey, 0, 1); + store.fetch(bytesKey, bytesKey, ofEpochMilli(0), ofEpochMilli(1)); EasyMock.verify(inner); } @@ -131,4 +132,4 @@ public void shouldRetainDuplicatesWhenSet() { EasyMock.verify(inner); } -} \ No newline at end of file +} 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 66f318aa10082..79afb7855c38e 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 @@ -35,6 +35,7 @@ import java.util.List; import java.util.NoSuchElementException; +import static java.time.Instant.ofEpochMilli; import static java.util.Arrays.asList; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsEqual.equalTo; @@ -77,7 +78,7 @@ public void shouldFetchValuesFromWindowStore() { underlyingWindowStore.put("my-key", "my-value", 0L); underlyingWindowStore.put("my-key", "my-later-value", 10L); - final WindowStoreIterator iterator = windowStore.fetch("my-key", 0L, 25L); + final WindowStoreIterator iterator = windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); final List> results = StreamsTestUtils.toList(iterator); assertEquals(asList(new KeyValue<>(0L, "my-value"), @@ -87,7 +88,7 @@ public void shouldFetchValuesFromWindowStore() { @Test public void shouldReturnEmptyIteratorIfNoData() { - final WindowStoreIterator iterator = windowStore.fetch("my-key", 0L, 25L); + final WindowStoreIterator iterator = windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); assertEquals(false, iterator.hasNext()); } @@ -100,10 +101,10 @@ public void shouldFindValueForKeyWhenMultiStores() { underlyingWindowStore.put("key-one", "value-one", 0L); secondUnderlying.put("key-two", "value-two", 10L); - final List> keyOneResults = StreamsTestUtils.toList(windowStore.fetch("key-one", 0L, - 1L)); - final List> keyTwoResults = StreamsTestUtils.toList(windowStore.fetch("key-two", 10L, - 11L)); + final List> keyOneResults = StreamsTestUtils.toList(windowStore.fetch("key-one", ofEpochMilli(0L), + ofEpochMilli(1L))); + final List> keyTwoResults = StreamsTestUtils.toList(windowStore.fetch("key-two", ofEpochMilli(10L), + ofEpochMilli(11L))); assertEquals(Collections.singletonList(KeyValue.pair(0L, "value-one")), keyOneResults); assertEquals(Collections.singletonList(KeyValue.pair(10L, "value-two")), keyTwoResults); @@ -114,14 +115,14 @@ public void shouldNotGetValuesFromOtherStores() { otherUnderlyingStore.put("some-key", "some-value", 0L); underlyingWindowStore.put("some-key", "my-value", 1L); - final List> results = StreamsTestUtils.toList(windowStore.fetch("some-key", 0L, 2L)); + final List> results = StreamsTestUtils.toList(windowStore.fetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L))); assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results); } @Test(expected = InvalidStateStoreException.class) public void shouldThrowInvalidStateStoreExceptionOnRebalance() { final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>(new StateStoreProviderStub(true), QueryableStoreTypes.windowStore(), "foo"); - store.fetch("key", 1, 10); + store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); } @Test @@ -130,7 +131,7 @@ public void shouldThrowInvalidStateStoreExceptionIfFetchThrows() { final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>(stubProviderOne, QueryableStoreTypes.windowStore(), "window-store"); try { - store.fetch("key", 1, 10); + store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); Assert.fail("InvalidStateStoreException was expected"); } catch (final InvalidStateStoreException e) { Assert.assertEquals("State store is not available anymore and may have been migrated to another instance; " + @@ -142,7 +143,7 @@ public void shouldThrowInvalidStateStoreExceptionIfFetchThrows() { public void emptyIteratorAlwaysReturnsFalse() { final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>(new StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo"); - final WindowStoreIterator windowStoreIterator = store.fetch("key", 1, 10); + final WindowStoreIterator windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); Assert.assertFalse(windowStoreIterator.hasNext()); } @@ -151,7 +152,7 @@ public void emptyIteratorAlwaysReturnsFalse() { public void emptyIteratorPeekNextKeyShouldThrowNoSuchElementException() { final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>(new StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo"); - final WindowStoreIterator windowStoreIterator = store.fetch("key", 1, 10); + final WindowStoreIterator windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); windowStoreIteratorException.expect(NoSuchElementException.class); windowStoreIterator.peekNextKey(); @@ -161,7 +162,7 @@ public void emptyIteratorPeekNextKeyShouldThrowNoSuchElementException() { public void emptyIteratorNextShouldThrowNoSuchElementException() { final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>(new StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo"); - final WindowStoreIterator windowStoreIterator = store.fetch("key", 1, 10); + final WindowStoreIterator windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); windowStoreIteratorException.expect(NoSuchElementException.class); windowStoreIterator.next(); @@ -173,7 +174,7 @@ public void shouldFetchKeyRangeAcrossStores() { stubProviderTwo.addStore(storeName, secondUnderlying); underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); - final List, String>> results = StreamsTestUtils.toList(windowStore.fetch("a", "b", 0, 10)); + final List, String>> results = StreamsTestUtils.toList(windowStore.fetch("a", "b", ofEpochMilli(0), ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); @@ -204,7 +205,7 @@ public void shouldGetAllAcrossStores() { KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); } - + @Test public void shouldFetchAllAcrossStores() { final ReadOnlyWindowStoreStub secondUnderlying = new @@ -212,7 +213,7 @@ public void shouldFetchAllAcrossStores() { stubProviderTwo.addStore(storeName, secondUnderlying); underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); - final List, String>> results = StreamsTestUtils.toList(windowStore.fetchAll(0, 10)); + final List, String>> results = StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(0), ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); @@ -220,17 +221,17 @@ public void shouldFetchAllAcrossStores() { @Test(expected = NullPointerException.class) public void shouldThrowNPEIfKeyIsNull() { - windowStore.fetch(null, 0, 0); + windowStore.fetch(null, ofEpochMilli(0), ofEpochMilli(0)); } @Test(expected = NullPointerException.class) public void shouldThrowNPEIfFromKeyIsNull() { - windowStore.fetch(null, "a", 0, 0); + windowStore.fetch(null, "a", ofEpochMilli(0), ofEpochMilli(0)); } @Test(expected = NullPointerException.class) public void shouldThrowNPEIfToKeyIsNull() { - windowStore.fetch("a", null, 0, 0); + windowStore.fetch("a", null, ofEpochMilli(0), ofEpochMilli(0)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java index 1ac6d94cfeec0..3a6a3b4987ffa 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java @@ -40,6 +40,7 @@ import java.util.Map; +import static java.time.Instant.ofEpochMilli; import static java.util.Collections.singletonMap; import static org.apache.kafka.test.StreamsTestUtils.getMetricByNameFilterByTags; import static org.junit.Assert.assertEquals; @@ -114,7 +115,7 @@ public void shouldRecordFetchLatency() { EasyMock.replay(innerStoreMock); store.init(context, store); - store.fetch("a", 1, 1).close(); // recorded on close; + store.fetch("a", ofEpochMilli(1), ofEpochMilli(1)).close(); // recorded on close; final Map metrics = context.metrics().metrics(); assertEquals(1.0, getMetricByNameFilterByTags(metrics, "fetch-total", "stream-scope-metrics", singletonMap("scope-id", "all")).metricValue()); assertEquals(1.0, getMetricByNameFilterByTags(metrics, "fetch-total", "stream-scope-metrics", singletonMap("scope-id", "mocked-store")).metricValue()); @@ -127,7 +128,7 @@ public void shouldRecordFetchRangeLatency() { EasyMock.replay(innerStoreMock); store.init(context, store); - store.fetch("a", "b", 1, 1).close(); // recorded on close; + store.fetch("a", "b", ofEpochMilli(1), ofEpochMilli(1)).close(); // recorded on close; final Map metrics = context.metrics().metrics(); assertEquals(1.0, getMetricByNameFilterByTags(metrics, "fetch-total", "stream-scope-metrics", singletonMap("scope-id", "all")).metricValue()); assertEquals(1.0, getMetricByNameFilterByTags(metrics, "fetch-total", "stream-scope-metrics", singletonMap("scope-id", "mocked-store")).metricValue()); @@ -171,4 +172,4 @@ public void shouldNotExceptionIfFetchReturnsNull() { assertNull(store.fetch("a", 0)); } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java index 4818d0990f2ad..99abdc4746b71 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams.state.internals; +import java.time.Instant; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.kstream.Windowed; @@ -73,6 +75,13 @@ public WindowStoreIterator fetch(final K key, final long timeFrom, final long return new TheWindowStoreIterator<>(results.iterator()); } + @Override + public WindowStoreIterator fetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(from, "from"); + ApiUtils.validateMillisecondInstant(to, "to"); + return fetch(key, from.toEpochMilli(), to.toEpochMilli()); + } + @Override public KeyValueIterator, V> all() { if (!open) { @@ -164,6 +173,13 @@ public void remove() { }; } + @Override + public KeyValueIterator, V> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(from, "from"); + ApiUtils.validateMillisecondInstant(to, "to"); + return fetchAll(from.toEpochMilli(), to.toEpochMilli()); + } + @Override public KeyValueIterator, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { if (!open) { @@ -209,6 +225,15 @@ public void remove() { }; } + @Override public KeyValueIterator, V> fetch(final K from, + final K to, + final Instant fromTime, + final Instant toTime) throws IllegalArgumentException { + ApiUtils.validateMillisecondInstant(fromTime, "fromTime"); + ApiUtils.validateMillisecondInstant(toTime, "toTime"); + return fetch(from, to, fromTime.toEpochMilli(), toTime.toEpochMilli()); + } + public void put(final K key, final V value, final long timestamp) { if (!data.containsKey(timestamp)) { data.put(timestamp, new TreeMap()); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java index 201a31efd7e89..08f019feffac3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java @@ -54,6 +54,7 @@ import java.util.Map; import java.util.Set; +import static java.time.Instant.ofEpochMilli; import static java.util.Objects.requireNonNull; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; @@ -142,7 +143,7 @@ public void shouldOnlyIterateOpenSegments() { setCurrentTime(currentTime); windowStore.put(1, "three"); - final WindowStoreIterator iterator = windowStore.fetch(1, 0, currentTime); + final WindowStoreIterator iterator = windowStore.fetch(1, ofEpochMilli(0), ofEpochMilli(currentTime)); // roll to the next segment that will close the first currentTime = currentTime + segmentInterval; @@ -177,12 +178,12 @@ public void testRangeAndSinglePointFetch() { assertEquals("four", windowStore.fetch(4, startTime + 4L)); assertEquals("five", windowStore.fetch(5, startTime + 5L)); - assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime + 0L - windowSize, startTime + 0L + windowSize))); - assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + 1L - windowSize, startTime + 1L + windowSize))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + 3L - windowSize, startTime + 3L + windowSize))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + 4L - windowSize, startTime + 4L + windowSize))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + 5L - windowSize, startTime + 5L + windowSize))); + assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize)))); + assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L + windowSize)))); + assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L + windowSize)))); + assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L + windowSize)))); + assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L + windowSize)))); putSecondBatch(windowStore, startTime, context); @@ -193,21 +194,21 @@ public void testRangeAndSinglePointFetch() { assertEquals("two+5", windowStore.fetch(2, startTime + 7L)); assertEquals("two+6", windowStore.fetch(2, startTime + 8L)); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime - 2L - windowSize, startTime - 2L + windowSize))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime - 1L - windowSize, startTime - 1L + windowSize))); - assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, startTime - windowSize, startTime + windowSize))); - assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, startTime + 1L - windowSize, startTime + 1L + windowSize))); - assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L + windowSize))); - assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, startTime + 3L - windowSize, startTime + 3L + windowSize))); - assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, startTime + 4L - windowSize, startTime + 4L + windowSize))); - assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 5L - windowSize, startTime + 5L + windowSize))); - assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 6L - windowSize, startTime + 6L + windowSize))); - assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 7L - windowSize, startTime + 7L + windowSize))); - assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 8L - windowSize, startTime + 8L + windowSize))); - assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 9L - windowSize, startTime + 9L + windowSize))); - assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, startTime + 10L - windowSize, startTime + 10L + windowSize))); - assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, startTime + 11L - windowSize, startTime + 11L + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 12L - windowSize, startTime + 12L + windowSize))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 2L - windowSize), ofEpochMilli(startTime - 2L + windowSize)))); + assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime - 1L - windowSize), ofEpochMilli(startTime - 1L + windowSize)))); + assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L + windowSize)))); + assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize)))); + assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L + windowSize)))); + assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L + windowSize)))); + assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L + windowSize)))); + assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 6L - windowSize), ofEpochMilli(startTime + 6L + windowSize)))); + assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 7L - windowSize), ofEpochMilli(startTime + 7L + windowSize)))); + assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 8L - windowSize), ofEpochMilli(startTime + 8L + windowSize)))); + assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 9L - windowSize), ofEpochMilli(startTime + 9L + windowSize)))); + assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 10L - windowSize), ofEpochMilli(startTime + 10L + windowSize)))); + assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L - windowSize), ofEpochMilli(startTime + 11L + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 12L - windowSize), ofEpochMilli(startTime + 12L + windowSize)))); // Flush the store and verify all current entries were properly flushed ... windowStore.flush(); @@ -257,17 +258,17 @@ public void shouldFetchAllInTimeRange() { assertEquals( Utils.mkList(one, two, four), - StreamsTestUtils.toList(windowStore.fetchAll(startTime + 1, startTime + 4)) + StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))) ); assertEquals( Utils.mkList(zero, one, two), - StreamsTestUtils.toList(windowStore.fetchAll(startTime + 0, startTime + 3)) + StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))) ); assertEquals( Utils.mkList(one, two, four, five), - StreamsTestUtils.toList(windowStore.fetchAll(startTime + 1, startTime + 5)) + StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))) ); } @@ -286,36 +287,36 @@ public void testFetchRange() { assertEquals( Utils.mkList(zero, one), - StreamsTestUtils.toList(windowStore.fetch(0, 1, startTime + 0L - windowSize, startTime + 0L + windowSize)) + StreamsTestUtils.toList(windowStore.fetch(0, 1, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize))) ); assertEquals( Utils.mkList(one), - StreamsTestUtils.toList(windowStore.fetch(1, 1, startTime + 0L - windowSize, startTime + 0L + windowSize)) + StreamsTestUtils.toList(windowStore.fetch(1, 1, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize))) ); assertEquals( Utils.mkList(one, two), - StreamsTestUtils.toList(windowStore.fetch(1, 3, startTime + 0L - windowSize, startTime + 0L + windowSize)) + StreamsTestUtils.toList(windowStore.fetch(1, 3, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize))) ); assertEquals( Utils.mkList(zero, one, two), - StreamsTestUtils.toList(windowStore.fetch(0, 5, startTime + 0L - windowSize, startTime + 0L + windowSize)) + StreamsTestUtils.toList(windowStore.fetch(0, 5, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize))) ); assertEquals( Utils.mkList(zero, one, two, four, five), - StreamsTestUtils.toList(windowStore.fetch(0, 5, startTime + 0L - windowSize, startTime + 0L + windowSize + 5L)) + StreamsTestUtils.toList(windowStore.fetch(0, 5, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize + 5L))) ); assertEquals( Utils.mkList(two, four, five), - StreamsTestUtils.toList(windowStore.fetch(0, 5, startTime + 2L, startTime + 0L + windowSize + 5L)) + StreamsTestUtils.toList(windowStore.fetch(0, 5, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + 0L + windowSize + 5L))) ); assertEquals( Utils.mkList(), - StreamsTestUtils.toList(windowStore.fetch(4, 5, startTime + 2L, startTime + windowSize)) + StreamsTestUtils.toList(windowStore.fetch(4, 5, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + windowSize))) ); assertEquals( Utils.mkList(), - StreamsTestUtils.toList(windowStore.fetch(0, 3, startTime + 3L, startTime + windowSize + 5)) + StreamsTestUtils.toList(windowStore.fetch(0, 3, ofEpochMilli(startTime + 3L), ofEpochMilli(startTime + windowSize + 5))) ); } @@ -326,30 +327,30 @@ public void testPutAndFetchBefore() { putFirstBatch(windowStore, startTime, context); - assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime + 0L - windowSize, startTime + 0L))); - assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + 1L - windowSize, startTime + 1L))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + 3L - windowSize, startTime + 3L))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + 4L - windowSize, startTime + 4L))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + 5L - windowSize, startTime + 5L))); + assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L)))); + assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L)))); + assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L)))); + assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L)))); + assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L)))); putSecondBatch(windowStore, startTime, context); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime - 1L - windowSize, startTime - 1L))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 0L - windowSize, startTime + 0L))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 1L - windowSize, startTime + 1L))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L))); - assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, startTime + 3L - windowSize, startTime + 3L))); - assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, startTime + 4L - windowSize, startTime + 4L))); - assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, startTime + 5L - windowSize, startTime + 5L))); - assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, startTime + 6L - windowSize, startTime + 6L))); - assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, startTime + 7L - windowSize, startTime + 7L))); - assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 8L - windowSize, startTime + 8L))); - assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 9L - windowSize, startTime + 9L))); - assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, startTime + 10L - windowSize, startTime + 10L))); - assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, startTime + 11L - windowSize, startTime + 11L))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 12L - windowSize, startTime + 12L))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 13L - windowSize, startTime + 13L))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 1L - windowSize), ofEpochMilli(startTime - 1L)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L)))); + assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L)))); + assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L)))); + assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L)))); + assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L)))); + assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 6L - windowSize), ofEpochMilli(startTime + 6L)))); + assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 7L - windowSize), ofEpochMilli(startTime + 7L)))); + assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 8L - windowSize), ofEpochMilli(startTime + 8L)))); + assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 9L - windowSize), ofEpochMilli(startTime + 9L)))); + assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 10L - windowSize), ofEpochMilli(startTime + 10L)))); + assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L - windowSize), ofEpochMilli(startTime + 11L)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 12L - windowSize), ofEpochMilli(startTime + 12L)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 13L - windowSize), ofEpochMilli(startTime + 13L)))); // Flush the store and verify all current entries were properly flushed ... windowStore.flush(); @@ -372,30 +373,30 @@ public void testPutAndFetchAfter() { putFirstBatch(windowStore, startTime, context); - assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime + 0L, startTime + 0L + windowSize))); - assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + 1L, startTime + 1L + windowSize))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L, startTime + 2L + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + 3L, startTime + 3L + windowSize))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + 4L, startTime + 4L + windowSize))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + 5L, startTime + 5L + windowSize))); + assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L), ofEpochMilli(startTime + 0L + windowSize)))); + assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L), ofEpochMilli(startTime + 1L + windowSize)))); + assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + 2L + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L), ofEpochMilli(startTime + 3L + windowSize)))); + assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L), ofEpochMilli(startTime + 4L + windowSize)))); + assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L), ofEpochMilli(startTime + 5L + windowSize)))); putSecondBatch(windowStore, startTime, context); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime - 2L, startTime - 2L + windowSize))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime - 1L, startTime - 1L + windowSize))); - assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, startTime, startTime + windowSize))); - assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, startTime + 1L, startTime + 1L + windowSize))); - assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, startTime + 2L, startTime + 2L + windowSize))); - assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, startTime + 3L, startTime + 3L + windowSize))); - assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, startTime + 4L, startTime + 4L + windowSize))); - assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 5L, startTime + 5L + windowSize))); - assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 6L, startTime + 6L + windowSize))); - assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, startTime + 7L, startTime + 7L + windowSize))); - assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, startTime + 8L, startTime + 8L + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 9L, startTime + 9L + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 10L, startTime + 10L + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 11L, startTime + 11L + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 12L, startTime + 12L + windowSize))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 2L), ofEpochMilli(startTime - 2L + windowSize)))); + assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime - 1L), ofEpochMilli(startTime - 1L + windowSize)))); + assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, ofEpochMilli(startTime), ofEpochMilli(startTime + windowSize)))); + assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 1L), ofEpochMilli(startTime + 1L + windowSize)))); + assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + 2L + windowSize)))); + assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 3L), ofEpochMilli(startTime + 3L + windowSize)))); + assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 4L), ofEpochMilli(startTime + 4L + windowSize)))); + assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 5L), ofEpochMilli(startTime + 5L + windowSize)))); + assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 6L), ofEpochMilli(startTime + 6L + windowSize)))); + assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 7L), ofEpochMilli(startTime + 7L + windowSize)))); + assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 8L), ofEpochMilli(startTime + 8L + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 9L), ofEpochMilli(startTime + 9L + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 10L), ofEpochMilli(startTime + 10L + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L), ofEpochMilli(startTime + 11L + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 12L), ofEpochMilli(startTime + 12L + windowSize)))); // Flush the store and verify all current entries were properly flushed ... windowStore.flush(); @@ -419,17 +420,17 @@ public void testPutSameKeyTimestamp() { setCurrentTime(startTime); windowStore.put(0, "zero"); - assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize))); + assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); windowStore.put(0, "zero"); windowStore.put(0, "zero+"); windowStore.put(0, "zero++"); - assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize))); - assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, startTime + 1L - windowSize, startTime + 1L + windowSize))); - assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, startTime + 2L - windowSize, startTime + 2L + windowSize))); - assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, startTime + 3L - windowSize, startTime + 3L + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime + 4L - windowSize, startTime + 4L + windowSize))); + assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L + windowSize)))); + assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize)))); + assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L + windowSize)))); // Flush the store and verify all current entries were properly flushed ... windowStore.flush(); @@ -487,12 +488,12 @@ public void testRolling() { segmentDirs(baseDir) ); - assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize))); - assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize))); + assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); + assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); + assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); + assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); setCurrentTime(startTime + increment * 6); windowStore.put(6, "six"); @@ -506,13 +507,13 @@ public void testRolling() { ); - assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize))); - assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); + assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); + assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); + assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); + assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); setCurrentTime(startTime + increment * 7); @@ -526,14 +527,14 @@ public void testRolling() { segmentDirs(baseDir) ); - assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize))); - assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize))); - assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); + assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); + assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); + assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); + assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); + assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize)))); setCurrentTime(startTime + increment * 8); windowStore.put(8, "eight"); @@ -547,15 +548,15 @@ public void testRolling() { ); - assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize))); - assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize))); - assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize))); - assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); + assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); + assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); + assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); + assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize)))); + assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize)))); // check segment directories windowStore.flush(); @@ -603,27 +604,27 @@ public void testRestore() throws IOException { Utils.delete(baseDir); windowStore = createWindowStore(context, false); - assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(8, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize)))); context.restore(windowName, changeLog); - assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize))); - assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize))); - assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize))); - assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); + assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); + assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); + assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); + assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); + assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize)))); + assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize)))); // check segment directories windowStore.flush(); @@ -662,7 +663,7 @@ public void testSegmentMaintenance() { WindowStoreIterator iter; int fetchedCount; - iter = windowStore.fetch(0, 0L, segmentInterval * 4); + iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(segmentInterval * 4)); fetchedCount = 0; while (iter.hasNext()) { iter.next(); @@ -678,7 +679,7 @@ public void testSegmentMaintenance() { setCurrentTime(segmentInterval * 3); windowStore.put(0, "v"); - iter = windowStore.fetch(0, 0L, segmentInterval * 4); + iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(segmentInterval * 4)); fetchedCount = 0; while (iter.hasNext()) { iter.next(); @@ -694,7 +695,7 @@ public void testSegmentMaintenance() { setCurrentTime(segmentInterval * 5); windowStore.put(0, "v"); - iter = windowStore.fetch(0, segmentInterval * 4, segmentInterval * 10); + iter = windowStore.fetch(0, ofEpochMilli(segmentInterval * 4), ofEpochMilli(segmentInterval * 10)); fetchedCount = 0; while (iter.hasNext()) { iter.next(); @@ -736,7 +737,7 @@ public void testInitialLoading() { assertEquals(expected, actual); - try (final WindowStoreIterator iter = windowStore.fetch(0, 0L, 1000000L)) { + try (final WindowStoreIterator iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(1000000L))) { while (iter.hasNext()) { iter.next(); } @@ -756,7 +757,7 @@ public void shouldCloseOpenIteratorsWhenStoreIsClosedAndNotThrowInvalidStateStor windowStore.put(1, "two", 2L); windowStore.put(1, "three", 3L); - final WindowStoreIterator iterator = windowStore.fetch(1, 1L, 3L); + final WindowStoreIterator iterator = windowStore.fetch(1, ofEpochMilli(1L), ofEpochMilli(3L)); assertTrue(iterator.hasNext()); windowStore.close(); @@ -784,16 +785,17 @@ public void shouldFetchAndIterateOverExactKeys() { final List expected = Utils.mkList("0001", "0003", "0005"); - assertThat(toList(windowStore.fetch("a", 0, Long.MAX_VALUE)), equalTo(expected)); + assertThat(toList(windowStore.fetch("a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expected)); - List, String>> list = StreamsTestUtils.toList(windowStore.fetch("a", "a", 0, Long.MAX_VALUE)); + List, String>> list = + StreamsTestUtils.toList(windowStore.fetch("a", "a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); assertThat(list, equalTo(Utils.mkList( windowedPair("a", "0001", 0, windowSize), windowedPair("a", "0003", 1, windowSize), windowedPair("a", "0005", 0x7a00000000000000L - 1, windowSize) ))); - list = StreamsTestUtils.toList(windowStore.fetch("aa", "aa", 0, Long.MAX_VALUE)); + list = StreamsTestUtils.toList(windowStore.fetch("aa", "aa", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); assertThat(list, equalTo(Utils.mkList( windowedPair("aa", "0002", 0, windowSize), windowedPair("aa", "0004", 1, windowSize) @@ -815,19 +817,19 @@ public void shouldNotThrowNullPointerExceptionOnPutNullValue() { @Test(expected = NullPointerException.class) public void shouldThrowNullPointerExceptionOnGetNullKey() { windowStore = createWindowStore(context, false); - windowStore.fetch(null, 1L, 2L); + windowStore.fetch(null, ofEpochMilli(1L), ofEpochMilli(2L)); } @Test(expected = NullPointerException.class) public void shouldThrowNullPointerExceptionOnRangeNullFromKey() { windowStore = createWindowStore(context, false); - windowStore.fetch(null, 2, 1L, 2L); + windowStore.fetch(null, 2, ofEpochMilli(1L), ofEpochMilli(2L)); } @Test(expected = NullPointerException.class) public void shouldThrowNullPointerExceptionOnRangeNullToKey() { windowStore = createWindowStore(context, false); - windowStore.fetch(1, null, 1L, 2L); + windowStore.fetch(1, null, ofEpochMilli(1L), ofEpochMilli(2L)); } @Test @@ -866,11 +868,11 @@ public void shouldFetchAndIterateOverExactBinaryKeys() { windowStore.put(key3, "9", 59999); final List expectedKey1 = Utils.mkList("1", "4", "7"); - assertThat(toList(windowStore.fetch(key1, 0, Long.MAX_VALUE)), equalTo(expectedKey1)); + assertThat(toList(windowStore.fetch(key1, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey1)); final List expectedKey2 = Utils.mkList("2", "5", "8"); - assertThat(toList(windowStore.fetch(key2, 0, Long.MAX_VALUE)), equalTo(expectedKey2)); + assertThat(toList(windowStore.fetch(key2, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey2)); final List expectedKey3 = Utils.mkList("3", "6", "9"); - assertThat(toList(windowStore.fetch(key3, 0, Long.MAX_VALUE)), equalTo(expectedKey3)); + assertThat(toList(windowStore.fetch(key3, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey3)); } private void putFirstBatch(final WindowStore store, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java index 711cdc1d93c60..ca059b4c60be0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java @@ -78,7 +78,7 @@ public void before() { topology.addStateStore(Stores.keyValueStoreBuilder(Stores.inMemoryKeyValueStore("kv-store"), Serdes.String(), Serdes.String()), "the-processor"); topology.addStateStore( Stores.windowStoreBuilder( - Stores.persistentWindowStore("window-store", 10L, 2L, false), + Stores.persistentWindowStore("window-store", Duration.ofMillis(10L), Duration.ofMillis(2L), false), Serdes.String(), Serdes.String()), "the-processor" diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java index 01333db6b12fb..6f801c9e93acf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java @@ -41,7 +41,6 @@ import java.util.Collections; import java.util.Locale; import java.util.Properties; -import java.util.concurrent.TimeUnit; public class BrokerCompatibilityTest { @@ -108,7 +107,7 @@ public void uncaughtException(final Thread t, final Throwable e) { System.err.println("FATAL: An unexpected exception " + cause); e.printStackTrace(System.err); System.err.flush(); - streams.close(30, TimeUnit.SECONDS); + streams.close(Duration.ofSeconds(30)); } }); System.out.println("start Kafka Streams"); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java index 1f8238ef51819..e292f16eda77a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.tests; +import java.time.Duration; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KafkaStreams; @@ -56,7 +57,7 @@ public void start() { @Override public void run() { isRunning = false; - streams.close(TimeUnit.SECONDS.toMillis(300), TimeUnit.SECONDS); + streams.close(Duration.ofSeconds(300)); // need to wait for callback to avoid race condition // -> make sure the callback printout to stdout is there as it is expected test output @@ -102,7 +103,7 @@ public void onChange(final KafkaStreams.State newState, final KafkaStreams.State streams.start(); } if (uncaughtException) { - streams.close(TimeUnit.SECONDS.toMillis(60), TimeUnit.SECONDS); + streams.close(Duration.ofSeconds(60_000L)); streams = null; } sleep(1000); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java index 0bbb8896485fb..a6a2ebb33781f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.tests; +import java.time.Duration; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; @@ -30,7 +31,6 @@ import org.apache.kafka.streams.kstream.KStream; import java.util.Properties; -import java.util.concurrent.TimeUnit; public class ShutdownDeadlockTest { @@ -65,7 +65,7 @@ public void uncaughtException(final Thread t, final Throwable e) { Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { @Override public void run() { - streams.close(5, TimeUnit.SECONDS); + streams.close(Duration.ofSeconds(5)); } })); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java index 51dc05c4b8d51..ddff7a892e2e1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.tests; +import java.time.Duration; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.serialization.Serdes; @@ -40,7 +41,6 @@ import org.apache.kafka.streams.state.WindowStore; import java.util.Properties; -import java.util.concurrent.TimeUnit; public class SmokeTestClient extends SmokeTestUtil { @@ -82,7 +82,7 @@ public void run() { } public void close() { - streams.close(5, TimeUnit.SECONDS); + streams.close(Duration.ofSeconds(5)); // do not remove these printouts since they are needed for health scripts if (!uncaughtException) { System.out.println("SMOKE-TEST-CLIENT-CLOSED"); @@ -129,7 +129,7 @@ public boolean test(final String key, final Integer value) { data.groupByKey(Serialized.with(stringSerde, intSerde)); groupedData - .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(1))) + .windowedBy(TimeWindows.of(Duration.ofDays(1))) .aggregate( new Initializer() { public Integer apply() { @@ -154,7 +154,7 @@ public Integer apply(final String aggKey, final Integer value, final Integer agg // max groupedData - .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2))) + .windowedBy(TimeWindows.of(Duration.ofDays(2))) .aggregate( new Initializer() { public Integer apply() { @@ -179,7 +179,7 @@ public Integer apply(final String aggKey, final Integer value, final Integer agg // sum groupedData - .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2))) + .windowedBy(TimeWindows.of(Duration.ofDays(2))) .aggregate( new Initializer() { public Long apply() { @@ -202,7 +202,7 @@ public Long apply(final String aggKey, final Integer value, final Long aggregate // cnt groupedData - .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2))) + .windowedBy(TimeWindows.of(Duration.ofDays(2))) .count(Materialized.>as("uwin-cnt")) .toStream(new Unwindow()) .to("cnt", Produced.with(stringSerde, longSerde)); @@ -252,7 +252,7 @@ public Double apply(final Long value1, final Long value2) { @Override public void uncaughtException(final Thread t, final Throwable e) { System.out.println("FATAL: An unexpected exception is encountered on thread " + t + ": " + e); - streamsClient.close(30, TimeUnit.SECONDS); + streamsClient.close(Duration.ofSeconds(30)); } }); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java index eb196888a455e..25c642e5268db 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.tests; +import java.time.Duration; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.serialization.Serde; @@ -33,7 +34,6 @@ import java.util.HashMap; import java.util.Map; import java.util.Properties; -import java.util.concurrent.TimeUnit; public class StreamsBrokerDownResilienceTest { @@ -109,7 +109,7 @@ public void apply(final String key, final String value) { public void uncaughtException(final Thread t, final Throwable e) { System.err.println("FATAL: An unexpected exception " + e); System.err.flush(); - streams.close(30, TimeUnit.SECONDS); + streams.close(Duration.ofSeconds(30)); } }); System.out.println("Start Kafka Streams"); @@ -118,7 +118,7 @@ public void uncaughtException(final Thread t, final Throwable e) { Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { @Override public void run() { - streams.close(30, TimeUnit.SECONDS); + streams.close(Duration.ofSeconds(30)); System.out.println("Complete shutdown of streams resilience test app now"); System.out.flush(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java index db160fe89ebad..66b20b283fb26 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.tests; +import java.time.Duration; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.serialization.Serde; @@ -38,7 +39,6 @@ import java.util.Map; import java.util.Properties; import java.util.Set; -import java.util.concurrent.TimeUnit; public class StreamsStandByReplicaTest { @@ -164,7 +164,7 @@ public void run() { } private static void shutdown(final KafkaStreams streams) { - streams.close(10, TimeUnit.SECONDS); + streams.close(Duration.ofSeconds(10)); } private static boolean confirmCorrectConfigs(final Properties properties) { diff --git a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java index 3b5a915bef4d9..2f356bff3a133 100644 --- a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.test; +import java.time.Duration; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeaders; @@ -209,6 +210,13 @@ public Cancellable schedule(final long interval, final PunctuationType type, fin throw new UnsupportedOperationException("schedule() not supported."); } + @Override + public Cancellable schedule(final Duration interval, + final PunctuationType type, + final Punctuator callback) throws IllegalArgumentException { + throw new UnsupportedOperationException("schedule() not supported."); + } + @Override public void commit() { } diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java index c95f4086ef1b8..e13e144a3c825 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.test; +import java.time.Duration; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Cancellable; import org.apache.kafka.streams.processor.ProcessorContext; @@ -55,7 +56,7 @@ public MockProcessor() { public void init(final ProcessorContext context) { super.init(context); if (scheduleInterval > 0L) { - scheduleCancellable = context.schedule(scheduleInterval, punctuationType, new Punctuator() { + scheduleCancellable = context.schedule(Duration.ofMillis(scheduleInterval), punctuationType, new Punctuator() { @Override public void punctuate(final long timestamp) { if (punctuationType == PunctuationType.STREAM_TIME) { diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java index ce9838919f0b8..36d049c58bad1 100644 --- a/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.test; +import java.time.Duration; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.Cancellable; @@ -52,7 +53,15 @@ public StateStore getStateStore(final String name) { return null; } - @Override public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) { + @Override + public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) { + return null; + } + + @Override + public Cancellable schedule(final Duration interval, + final PunctuationType type, + final Punctuator callback) throws IllegalArgumentException { return null; } diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KStreamTest.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KStreamTest.scala index f33975647ef12..8626be56aae1f 100644 --- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KStreamTest.scala +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KStreamTest.scala @@ -18,6 +18,8 @@ */ package org.apache.kafka.streams.scala.kstream +import java.time.Duration.ofSeconds + import org.apache.kafka.streams.kstream.JoinWindows import org.apache.kafka.streams.scala.ImplicitConversions._ import org.apache.kafka.streams.scala.Serdes._ @@ -143,7 +145,7 @@ class KStreamTest extends FlatSpec with Matchers with TestDriver { val stream1 = builder.stream[String, String](sourceTopic1) val stream2 = builder.stream[String, String](sourceTopic2) - stream1.join(stream2)((a, b) => s"$a-$b", JoinWindows.of(1000)).to(sinkTopic) + stream1.join(stream2)((a, b) => s"$a-$b", JoinWindows.of(ofSeconds(1))).to(sinkTopic) val testDriver = createTestDriver(builder) diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/MaterializedTest.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/MaterializedTest.scala index 14fc5d4f89df3..5df2916316f44 100644 --- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/MaterializedTest.scala +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/MaterializedTest.scala @@ -18,6 +18,8 @@ */ package org.apache.kafka.streams.scala.kstream +import java.time.Duration + import org.apache.kafka.streams.kstream.internals.MaterializedInternal import org.apache.kafka.streams.scala.Serdes._ import org.apache.kafka.streams.scala._ @@ -50,7 +52,7 @@ class MaterializedTest extends FlatSpec with Matchers { } "Create a Materialize with a window store supplier" should "create a Materialized with Serdes and a store supplier" in { - val storeSupplier = Stores.persistentWindowStore("store", 1, 1, true) + val storeSupplier = Stores.persistentWindowStore("store", Duration.ofMillis(1), Duration.ofMillis(1), true) val materialized: Materialized[String, Long, ByteArrayWindowStore] = Materialized.as[String, Long](storeSupplier) @@ -72,7 +74,7 @@ class MaterializedTest extends FlatSpec with Matchers { } "Create a Materialize with a session store supplier" should "create a Materialized with Serdes and a store supplier" in { - val storeSupplier = Stores.persistentSessionStore("store", 1) + val storeSupplier = Stores.persistentSessionStore("store", Duration.ofMillis(1)) val materialized: Materialized[String, Long, ByteArraySessionStore] = Materialized.as[String, Long](storeSupplier) diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index fd3dcfe7d35aa..d10a45c00bf73 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -150,7 +150,7 @@ * {@link ProducerRecord#equals(Object)} can simplify your code as you can ignore attributes you are not interested in. *

      * Note, that calling {@code pipeInput()} will also trigger {@link PunctuationType#STREAM_TIME event-time} base - * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) punctuation} callbacks. + * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) punctuation} callbacks. * However, you won't trigger {@link PunctuationType#WALL_CLOCK_TIME wall-clock} type punctuations that you must * trigger manually via {@link #advanceWallClockTime(long)}. *

      @@ -489,7 +489,7 @@ public void pipeInput(final List> records) { /** * Advances the internally mocked wall-clock time. * This might trigger a {@link PunctuationType#WALL_CLOCK_TIME wall-clock} type - * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) punctuations}. + * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) punctuations}. * * @param advanceMs the amount of time to advance wall-clock time in milliseconds */ diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java index 553428deb4b51..88a7fe79e2beb 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java @@ -16,10 +16,12 @@ */ package org.apache.kafka.streams.processor; +import java.time.Duration; import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsMetrics; @@ -378,6 +380,7 @@ public StateStore getStateStore(final String name) { } @Override + @Deprecated public Cancellable schedule(final long intervalMs, final PunctuationType type, final Punctuator callback) { final CapturedPunctuator capturedPunctuator = new CapturedPunctuator(intervalMs, type, callback); @@ -386,6 +389,14 @@ public Cancellable schedule(final long intervalMs, final PunctuationType type, f return capturedPunctuator::cancel; } + @Override + public Cancellable schedule(final Duration interval, + final PunctuationType type, + final Punctuator callback) throws IllegalArgumentException { + ApiUtils.validateMillisecondDuration(interval, "interval"); + return schedule(interval.toMillis(), type, callback); + } + /** * Get the punctuators scheduled so far. The returned list is not affected by subsequent calls to {@code schedule(...)}. * diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java index 878aa35748340..ba5286783b0f5 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams; +import java.time.Duration; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.MockProcessorContext; @@ -345,7 +346,7 @@ public void shouldCapturePunctuator() { @Override public void init(final ProcessorContext context) { context.schedule( - 1000L, + Duration.ofSeconds(1L), PunctuationType.WALL_CLOCK_TIME, timestamp -> context.commit() ); diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java index 497a6c374cd10..58f6e02cc9fdb 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams; +import java.time.Duration; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.header.Header; @@ -231,7 +232,7 @@ public void init(final ProcessorContext context) { initialized = true; this.context = context; for (final Punctuation punctuation : punctuations) { - this.context.schedule(punctuation.intervalMs, punctuation.punctuationType, punctuation.callback); + this.context.schedule(Duration.ofMillis(punctuation.intervalMs), punctuation.punctuationType, punctuation.callback); } } @@ -863,8 +864,8 @@ private class CustomMaxAggregator implements Processor { @Override public void init(final ProcessorContext context) { this.context = context; - context.schedule(60000, PunctuationType.WALL_CLOCK_TIME, timestamp -> flushStore()); - context.schedule(10000, PunctuationType.STREAM_TIME, timestamp -> flushStore()); + context.schedule(Duration.ofMinutes(1), PunctuationType.WALL_CLOCK_TIME, timestamp -> flushStore()); + context.schedule(Duration.ofSeconds(10), PunctuationType.STREAM_TIME, timestamp -> flushStore()); store = (KeyValueStore) context.getStateStore("aggStore"); }