mirror of https://github.com/apache/kafka.git
				
				
				
			KAFKA-7277: Migrate Streams API to Duration instead of longMs times (#5682)
Reviewers: Johne Roesler <john@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
This commit is contained in:
		
							parent
							
								
									260b07a6da
								
							
						
					
					
						commit
						ca641b3e2e
					
				|  | @ -19,6 +19,7 @@ package org.apache.kafka.streams.examples.pageview; | |||
| 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.io.IOException; | |||
| 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 class PageViewTypedDemo { | |||
|             }) | ||||
|             .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) -> { | ||||
|  |  | |||
|  | @ -19,6 +19,7 @@ package org.apache.kafka.streams.examples.pageview; | |||
| 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 class PageViewUntypedDemo { | |||
|             }) | ||||
|             .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) -> { | ||||
|  |  | |||
|  | @ -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 org.apache.kafka.streams.kstream.WindowedSerdes; | |||
| 
 | ||||
| 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 class TemperatureDemo { | |||
|             // 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; | ||||
|  |  | |||
|  | @ -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 final class WordCountProcessorDemo { | |||
|                 @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<String, Integer> iter = kvStore.all()) { | ||||
|                             System.out.println("----------- " + timestamp + " ----------- "); | ||||
| 
 | ||||
|  |  | |||
|  | @ -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.common.utils.Time; | |||
| 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 class KafkaStreams { | |||
|      * @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 class KafkaStreams { | |||
|         } | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * 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}. | ||||
|  |  | |||
|  | @ -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); | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  | @ -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 @@ public final class JoinWindows extends Windows<Window> { | |||
|      * | ||||
|      * @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 final class JoinWindows extends Windows<Window> { | |||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * 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 final class JoinWindows extends Windows<Window> { | |||
|      * @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 | ||||
|  |  | |||
|  | @ -19,6 +19,7 @@ package org.apache.kafka.streams.kstream; | |||
| 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 class Materialized<K, V, S extends StateStore> { | |||
|      * 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<K, V, S> withRetention(final long retentionMs) { | ||||
|         if (retentionMs < 0) { | ||||
|     public Materialized<K, V, S> 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; | ||||
|     } | ||||
| } | ||||
|  |  | |||
|  | @ -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 @@ public final class SessionWindows { | |||
|      * @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 final class SessionWindows { | |||
|         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 <i>lower bound</i> for how long a window will be maintained. | ||||
|  | @ -124,18 +140,20 @@ public final class SessionWindows { | |||
|      * 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 | ||||
|         ); | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -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 @@ public final class TimeWindows extends Windows<TimeWindow> { | |||
|      * @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 final class TimeWindows extends Windows<TimeWindow> { | |||
|         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)}. | ||||
|      * <p> | ||||
|      * 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 final class TimeWindows extends Windows<TimeWindow> { | |||
|      * <p> | ||||
|      * 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 final class TimeWindows extends Windows<TimeWindow> { | |||
|         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)}. | ||||
|      * <p> | ||||
|      * 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<Long, TimeWindow> windowsFor(final long timestamp) { | ||||
|         long windowStart = (Math.max(0, timestamp - sizeMs + advanceMs) / advanceMs) * advanceMs; | ||||
|  | @ -152,15 +190,18 @@ public final class TimeWindows extends Windows<TimeWindow> { | |||
|      * | ||||
|      * 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 | ||||
|  |  | |||
|  | @ -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 @@ import org.apache.kafka.streams.processor.To; | |||
|  * 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. | ||||
|  * <p> | ||||
|  * Use {@link TransformerSupplier} to provide new instances of {@code Transformer} to Kafka Stream's runtime. | ||||
|  | @ -55,7 +56,7 @@ public interface Transformer<K, V, R> { | |||
|      * framework may later re-use the transformer by calling {@link #init(ProcessorContext)} again. | ||||
|      * <p> | ||||
|      * 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. | ||||
|      * <p> | ||||
|      * Note, that {@link ProcessorContext} is updated in the background with the current record's meta data. | ||||
|  |  | |||
|  | @ -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 final class UnlimitedWindows extends Windows<UnlimitedWindow> { | |||
|      * @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 final class UnlimitedWindows extends Windows<UnlimitedWindow> { | |||
|         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<Long, UnlimitedWindow> windowsFor(final long timestamp) { | ||||
|         // always return the single unlimited window | ||||
|  |  | |||
|  | @ -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 @@ import org.apache.kafka.streams.processor.To; | |||
|  * 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. | ||||
|  * <p> | ||||
|  | @ -54,7 +55,7 @@ public interface ValueTransformer<V, VR> { | |||
|      * framework may later re-use the transformer by calling {@link #init(ProcessorContext)} again. | ||||
|      * <p> | ||||
|      * 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. | ||||
|      * <p> | ||||
|      * Note that {@link ProcessorContext} is updated in the background with the current record's meta data. | ||||
|  |  | |||
|  | @ -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 @@ import org.apache.kafka.streams.processor.To; | |||
|  * 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<K, V, VR> { | |||
|      * This is called once per instance when the topology gets initialized. | ||||
|      * <p> | ||||
|      * 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. | ||||
|      * <p> | ||||
|      * Note that {@link ProcessorContext} is updated in the background with the current record's meta data. | ||||
|  |  | |||
|  | @ -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 abstract class Window { | |||
|         } | ||||
|         this.startMs = startMs; | ||||
|         this.endMs = endMs; | ||||
| 
 | ||||
|         this.startTime = Instant.ofEpochMilli(startMs); | ||||
|         this.endTime = Instant.ofEpochMilli(endMs); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|  | @ -69,6 +76,24 @@ public abstract class Window { | |||
|         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 | ||||
|  |  | |||
|  | @ -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 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K | |||
|         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, | ||||
|  |  | |||
|  | @ -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 @@ public class TimeWindowedKStreamImpl<K, V, W extends Window> extends AbstractStr | |||
| 
 | ||||
|                 supplier = Stores.persistentWindowStore( | ||||
|                     materialized.storeName(), | ||||
|                     retentionPeriod, | ||||
|                     windows.size(), | ||||
|                     Duration.ofMillis(retentionPeriod), | ||||
|                     Duration.ofMillis(windows.size()), | ||||
|                     false | ||||
|                 ); | ||||
| 
 | ||||
|  |  | |||
|  | @ -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 | ||||
|  */ | ||||
|  |  | |||
|  | @ -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<K, V> { | |||
|      * framework may later re-use the processor by calling {@link #init()} again. | ||||
|      * <p> | ||||
|      * 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 | ||||
|  |  | |||
|  | @ -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 @@ public interface ProcessorContext { | |||
|      * 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: | ||||
|      * <ul> | ||||
|      *   <li>{@link PunctuationType#STREAM_TIME} - uses "stream time", which is advanced by the processing of messages | ||||
|      *   <li>{@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. | ||||
|      *   <b>NOTE:</b> Only advanced if messages arrive</li> | ||||
|      *   <li>{@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. | ||||
|      *   <li>{@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. | ||||
|      *   <b>NOTE:</b> This is best effort only as its granularity is limited by how long an iteration of the | ||||
|      *   processing loop takes to complete</li> | ||||
|  | @ -124,11 +124,48 @@ public interface ProcessorContext { | |||
|      * @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: | ||||
|      * <ul> | ||||
|      *   <li>{@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. | ||||
|      *   <b>NOTE:</b> Only advanced if messages arrive</li> | ||||
|      *   <li>{@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. | ||||
|      *   <b>NOTE:</b> This is best effort only as its granularity is limited by how long an iteration of the | ||||
|      *   processing loop takes to complete</li> | ||||
|      * </ul> | ||||
|      * | ||||
|      * <b>Skipping punctuations:</b> 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: | ||||
|      * <ul> | ||||
|      *   <li>with {@link PunctuationType#STREAM_TIME}, when stream time advances more than interval</li> | ||||
|      *   <li>with {@link PunctuationType#WALL_CLOCK_TIME}, on GC pause, too short interval, ...</li> | ||||
|      * </ul> | ||||
|      * | ||||
|      * @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. | ||||
|  |  | |||
|  | @ -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}: | ||||
|  * <ul> | ||||
|  *   <li>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. | ||||
|  |  | |||
|  | @ -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 | ||||
|  */ | ||||
|  |  | |||
|  | @ -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 final class ForwardingDisabledProcessorContext implements ProcessorContex | |||
|     } | ||||
| 
 | ||||
|     @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 <K, V> void forward(final K key, final V value) { | ||||
|         throw new StreamsException("ProcessorContext#forward() not supported."); | ||||
|  |  | |||
|  | @ -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 class GlobalProcessorContextImpl extends AbstractProcessorContext { | |||
|      * @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."); | ||||
|  |  | |||
|  | @ -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 class ProcessorContextImpl extends AbstractProcessorContext implements Re | |||
|     } | ||||
| 
 | ||||
|     @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; | ||||
|     } | ||||
|  |  | |||
|  | @ -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 @@ class StandbyContextImpl extends AbstractProcessorContext implements RecordColle | |||
|      * @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 | ||||
|      */ | ||||
|  |  | |||
|  | @ -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<K, V> { | |||
|     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. | ||||
|      * <p> | ||||
|      * This iterator must be closed after use. | ||||
|      * <p> | ||||
|      * The time range is inclusive and applies to the starting timestamp of the window. | ||||
|  | @ -73,13 +73,51 @@ public interface ReadOnlyWindowStore<K, V> { | |||
|      * @return an iterator over key-value pairs {@code <timestamp, value>} | ||||
|      * @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<V> 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. | ||||
|      * <p> | ||||
|      * This iterator must be closed after use. | ||||
|      * <p> | ||||
|      * The time range is inclusive and applies to the starting timestamp of the window. | ||||
|      * For example, if we have the following windows: | ||||
|      * <p> | ||||
|      * <pre> | ||||
|      * +-------------------------------+ | ||||
|      * |  key  | start time | end time | | ||||
|      * +-------+------------+----------+ | ||||
|      * |   A   |     10     |    20    | | ||||
|      * +-------+------------+----------+ | ||||
|      * |   A   |     15     |    25    | | ||||
|      * +-------+------------+----------+ | ||||
|      * |   A   |     20     |    30    | | ||||
|      * +-------+------------+----------+ | ||||
|      * |   A   |     25     |    35    | | ||||
|      * +-------------------------------- | ||||
|      * </pre> | ||||
|      * 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. | ||||
|      * <p> | ||||
|      * 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 <timestamp, value>} | ||||
|      * @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<V> 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. | ||||
|      * <p> | ||||
|      * This iterator must be closed after use. | ||||
|      * | ||||
|      * @param from      the first key in the range | ||||
|  | @ -89,9 +127,28 @@ public interface ReadOnlyWindowStore<K, V> { | |||
|      * @return an iterator over windowed key-value pairs {@code <Windowed<K>, 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<Windowed<K>, 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. | ||||
|      * <p> | ||||
|      * 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 <Windowed<K>, 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<Windowed<K>, 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<K, V> { | |||
|      * @return an iterator over windowed key-value pairs {@code <Windowed<K>, 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<Windowed<K>, 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 <Windowed<K>, 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<Windowed<K>, V> fetchAll(Instant from, Instant to) throws IllegalArgumentException; | ||||
| } | ||||
|  |  | |||
|  | @ -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 class Stores { | |||
|      * @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 class Stores { | |||
|         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 class Stores { | |||
|      * @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 class Stores { | |||
|         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}. | ||||
|  |  | |||
|  | @ -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<K, V> extends StateStore, ReadOnlyWindowStore<K, V> | |||
|      * @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. | ||||
|      * <p> | ||||
|      * This iterator must be closed after use. | ||||
|      * <p> | ||||
|      * The time range is inclusive and applies to the starting timestamp of the window. | ||||
|      * For example, if we have the following windows: | ||||
|      * <p> | ||||
|      * <pre> | ||||
|      * +-------------------------------+ | ||||
|      * |  key  | start time | end time | | ||||
|      * +-------+------------+----------+ | ||||
|      * |   A   |     10     |    20    | | ||||
|      * +-------+------------+----------+ | ||||
|      * |   A   |     15     |    25    | | ||||
|      * +-------+------------+----------+ | ||||
|      * |   A   |     20     |    30    | | ||||
|      * +-------+------------+----------+ | ||||
|      * |   A   |     25     |    35    | | ||||
|      * +-------------------------------- | ||||
|      * </pre> | ||||
|      * 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. | ||||
|      * <p> | ||||
|      * 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 <timestamp, value>} | ||||
|      * @throws InvalidStateStoreException if the store is not initialized | ||||
|      * @throws NullPointerException If {@code null} is used for key. | ||||
|      */ | ||||
|     WindowStoreIterator<V> 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. | ||||
|      * <p> | ||||
|      * 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 <Windowed<K>, value>} | ||||
|      * @throws InvalidStateStoreException if the store is not initialized | ||||
|      * @throws NullPointerException If {@code null} is used for any key. | ||||
|      */ | ||||
|     KeyValueIterator<Windowed<K>, 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 <Windowed<K>, value>} | ||||
|      * @throws InvalidStateStoreException if the store is not initialized | ||||
|      * @throws NullPointerException if {@code null} is used for any key | ||||
|      */ | ||||
|     KeyValueIterator<Windowed<K>, V> fetchAll(long timeFrom, long timeTo); | ||||
| } | ||||
|  |  | |||
|  | @ -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. | ||||
|  |  | |||
|  | @ -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 @@ class CachingWindowStore<K, V> extends WrappedStateStore.AbstractStateStore impl | |||
|         return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public WindowStoreIterator<byte[]> 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<Windowed<Bytes>, 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 | ||||
|  | @ -232,6 +241,16 @@ class CachingWindowStore<K, V> extends WrappedStateStore.AbstractStateStore impl | |||
|         ); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public KeyValueIterator<Windowed<Bytes>, 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) { | ||||
|  | @ -275,4 +294,11 @@ class CachingWindowStore<K, V> extends WrappedStateStore.AbstractStateStore impl | |||
|                 cacheFunction | ||||
|         ); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public KeyValueIterator<Windowed<Bytes>, byte[]> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException { | ||||
|         ApiUtils.validateMillisecondInstant(from, "from"); | ||||
|         ApiUtils.validateMillisecondInstant(to, "to"); | ||||
|         return fetchAll(from.toEpochMilli(), to.toEpochMilli()); | ||||
|     } | ||||
| } | ||||
|  |  | |||
|  | @ -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 @@ class ChangeLoggingWindowBytesStore extends WrappedStateStore.AbstractStateStore | |||
|         return bytesStore.fetch(key, from, to); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public WindowStoreIterator<byte[]> 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<Windowed<Bytes>, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) { | ||||
|         return bytesStore.fetch(keyFrom, keyTo, from, to); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public KeyValueIterator<Windowed<Bytes>, 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<Windowed<Bytes>, byte[]> all() { | ||||
|         return bytesStore.all(); | ||||
|  | @ -71,6 +90,13 @@ class ChangeLoggingWindowBytesStore extends WrappedStateStore.AbstractStateStore | |||
|         return bytesStore.fetchAll(timeFrom, timeTo); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public KeyValueIterator<Windowed<Bytes>, 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()); | ||||
|  |  | |||
|  | @ -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 class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K | |||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     @Deprecated | ||||
|     public WindowStoreIterator<V> fetch(final K key, final long timeFrom, final long timeTo) { | ||||
|         Objects.requireNonNull(key, "key can't be null"); | ||||
|         final List<ReadOnlyWindowStore<K, V>> stores = provider.stores(storeName, windowStoreType); | ||||
|  | @ -84,6 +87,13 @@ public class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K | |||
|         return KeyValueIterators.emptyWindowStoreIterator(); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public WindowStoreIterator<V> 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<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { | ||||
|         Objects.requireNonNull(from, "from can't be null"); | ||||
|  | @ -100,6 +110,13 @@ public class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K | |||
|                                                                nextIteratorFunction)); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public KeyValueIterator<Windowed<K>, 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<Windowed<K>, V> all() { | ||||
|         final NextIteratorFunction<Windowed<K>, V, ReadOnlyWindowStore<K, V>> nextIteratorFunction = new NextIteratorFunction<Windowed<K>, V, ReadOnlyWindowStore<K, V>>() { | ||||
|  | @ -115,6 +132,7 @@ public class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K | |||
|     } | ||||
|      | ||||
|     @Override | ||||
|     @Deprecated | ||||
|     public KeyValueIterator<Windowed<K>, V> fetchAll(final long timeFrom, final long timeTo) { | ||||
|         final NextIteratorFunction<Windowed<K>, V, ReadOnlyWindowStore<K, V>> nextIteratorFunction = new NextIteratorFunction<Windowed<K>, V, ReadOnlyWindowStore<K, V>>() { | ||||
|             @Override | ||||
|  | @ -127,4 +145,11 @@ public class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K | |||
|                         provider.stores(storeName, windowStoreType).iterator(), | ||||
|                         nextIteratorFunction)); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public KeyValueIterator<Windowed<K>, V> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException { | ||||
|         ApiUtils.validateMillisecondInstant(from, "from"); | ||||
|         ApiUtils.validateMillisecondInstant(to, "to"); | ||||
|         return fetchAll(from.toEpochMilli(), to.toEpochMilli()); | ||||
|     } | ||||
| } | ||||
|  |  | |||
|  | @ -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 class MeteredWindowStore<K, V> extends WrappedStateStore.AbstractStateSto | |||
|                                                 time); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public WindowStoreIterator<V> 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<Windowed<K>, V> all() { | ||||
|         return new MeteredWindowedKeyValueIterator<>(inner.all(), fetchTime, metrics, serdes, time); | ||||
|  | @ -161,6 +170,13 @@ public class MeteredWindowStore<K, V> extends WrappedStateStore.AbstractStateSto | |||
|                                                      time); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public KeyValueIterator<Windowed<K>, 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<Windowed<K>, 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 class MeteredWindowStore<K, V> extends WrappedStateStore.AbstractStateSto | |||
|                                                      time); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public KeyValueIterator<Windowed<K>, 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(); | ||||
|  |  | |||
|  | @ -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 class RocksDBWindowStore<K, V> extends WrappedStateStore.AbstractStateSto | |||
|         return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).valuesIterator(); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public WindowStoreIterator<V> 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<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { | ||||
|         final KeyValueIterator<Bytes, byte[]> 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<Windowed<K>, 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<Windowed<K>, V> all() { | ||||
|         final KeyValueIterator<Bytes, byte[]> bytesIterator = bytesStore.all(); | ||||
|  | @ -109,6 +125,13 @@ public class RocksDBWindowStore<K, V> extends WrappedStateStore.AbstractStateSto | |||
|         return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).keyValueIterator(); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public KeyValueIterator<Windowed<K>, 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; | ||||
|  |  | |||
|  | @ -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 class KafkaStreamsTest { | |||
|                 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); | ||||
|  |  | |||
|  | @ -47,6 +47,7 @@ import java.util.Properties; | |||
| 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 class TopologyTest { | |||
|         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 class TopologyTest { | |||
|         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 class TopologyTest { | |||
|         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 class TopologyTest { | |||
|         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 class TopologyTest { | |||
|         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 class TopologyTest { | |||
|         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( | ||||
|  |  | |||
|  | @ -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 java.util.concurrent.TimeUnit; | |||
| 
 | ||||
| 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 @@ public abstract class AbstractResetIntegrationTest { | |||
| 
 | ||||
|     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 abstract class AbstractResetIntegrationTest { | |||
| 
 | ||||
|         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<Windowed<Long>, Long, KeyValue<Long, Long>>() { | ||||
|  |  | |||
|  | @ -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.ArrayList; | |||
| 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 class GlobalThreadShutDownOrderTest { | |||
|             } | ||||
|         }, 10000L, "Has not processed record within 10 seconds"); | ||||
| 
 | ||||
|         kafkaStreams.close(30, TimeUnit.SECONDS); | ||||
|         kafkaStreams.close(Duration.ofSeconds(30)); | ||||
| 
 | ||||
|         final List<Long> expectedRetrievedValues = Arrays.asList(1L, 2L, 3L, 4L); | ||||
|         assertEquals(expectedRetrievedValues, retrievedValuesList); | ||||
|  |  | |||
|  | @ -57,6 +57,8 @@ import java.util.Properties; | |||
| 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 class InternalTopicIntegrationTest { | |||
| 
 | ||||
|         textLines.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"))) | ||||
|             .groupBy(MockMapper.selectValueMapper()) | ||||
|             .windowedBy(TimeWindows.of(1000).grace(0L)) | ||||
|             .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("CountWindows").withRetention(2_000L)); | ||||
|             .windowedBy(TimeWindows.of(ofSeconds(1L)).grace(ofMillis(0L))) | ||||
|             .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("CountWindows").withRetention(ofSeconds(2L))); | ||||
| 
 | ||||
|         final KafkaStreams streams = new KafkaStreams(builder.build(), streamsProp); | ||||
|         streams.start(); | ||||
|  |  | |||
|  | @ -53,6 +53,8 @@ import java.util.Arrays; | |||
| 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 class KStreamAggregationDedupIntegrationTest { | |||
|         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 class KStreamAggregationDedupIntegrationTest { | |||
|         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())); | ||||
|  |  | |||
|  | @ -85,6 +85,8 @@ import java.util.Set; | |||
| 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 class KStreamAggregationIntegrationTest { | |||
| 
 | ||||
|         final Serde<Windowed<String>> 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 class KStreamAggregationIntegrationTest { | |||
|         produceMessages(secondTimestamp); | ||||
| 
 | ||||
|         final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class); | ||||
|         groupedStream.windowedBy(TimeWindows.of(500L)) | ||||
|         groupedStream.windowedBy(TimeWindows.of(ofMillis(500L))) | ||||
|                 .aggregate( | ||||
|                         initializer, | ||||
|                         aggregator, | ||||
|  | @ -427,7 +429,7 @@ public class KStreamAggregationIntegrationTest { | |||
|         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 class KStreamAggregationIntegrationTest { | |||
| 
 | ||||
|         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<Windowed<String>, Long, KeyValue<Object, Object>>() { | ||||
|  | @ -618,7 +620,7 @@ public class KStreamAggregationIntegrationTest { | |||
|         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 class KStreamAggregationIntegrationTest { | |||
| 
 | ||||
|         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<Windowed<String>, Long, KeyValue<Object, Object>>() { | ||||
|  |  | |||
|  | @ -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.Map; | |||
| 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 class KTableSourceTopicRestartIntegrationTest { | |||
| 
 | ||||
|             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 class KTableSourceTopicRestartIntegrationTest { | |||
| 
 | ||||
|             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 class KTableSourceTopicRestartIntegrationTest { | |||
| 
 | ||||
|             assertNumberValuesRead(readKeyValues, expectedValues, "Table did not get all values after restart"); | ||||
|         } finally { | ||||
|             streamsOne.close(5, TimeUnit.SECONDS); | ||||
|             streamsOne.close(Duration.ofSeconds(5)); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -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.List; | |||
| 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 class PurgeRepartitionTopicIntegrationTest { | |||
|     @After | ||||
|     public void shutdown() throws IOException { | ||||
|         if (kafkaStreams != null) { | ||||
|             kafkaStreams.close(30, TimeUnit.SECONDS); | ||||
|             kafkaStreams.close(Duration.ofSeconds(30)); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -86,6 +86,9 @@ import java.util.TreeSet; | |||
| 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 class QueryableStateIntegrationTest { | |||
|     @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 class QueryableStateIntegrationTest { | |||
| 
 | ||||
|         // 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.<String, Long, WindowStore<Bytes, byte[]>>as(windowStoreName + "-" + inputTopic)) | ||||
|             .toStream(new KeyValueMapper<Windowed<String>, Long, String>() { | ||||
|                 @Override | ||||
|  | @ -361,7 +364,7 @@ public class QueryableStateIntegrationTest { | |||
|                         final int index = metadata.hostInfo().port(); | ||||
|                         final KafkaStreams streamsWithKey = streamRunnables[index].getStream(); | ||||
|                         final ReadOnlyWindowStore<String, Long> store = streamsWithKey.store(storeName, QueryableStoreTypes.<String, Long>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 @@ public class QueryableStateIntegrationTest { | |||
| 
 | ||||
|         final String windowStoreName = "windowed-count"; | ||||
|         s1.groupByKey() | ||||
|             .windowedBy(TimeWindows.of(WINDOW_SIZE)) | ||||
|             .windowedBy(TimeWindows.of(ofMillis(WINDOW_SIZE))) | ||||
|             .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as(windowStoreName)); | ||||
|         kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration); | ||||
|         kafkaStreams.start(); | ||||
|  | @ -1017,7 +1020,7 @@ public class QueryableStateIntegrationTest { | |||
|     private Set<KeyValue<String, Long>> fetch(final ReadOnlyWindowStore<String, Long> store, | ||||
|                                               final String key) { | ||||
| 
 | ||||
|         final WindowStoreIterator<Long> fetch = store.fetch(key, 0, System.currentTimeMillis()); | ||||
|         final WindowStoreIterator<Long> fetch = store.fetch(key, ofEpochMilli(0), ofEpochMilli(System.currentTimeMillis())); | ||||
|         if (fetch.hasNext()) { | ||||
|             final KeyValue<Long, Long> next = fetch.next(); | ||||
|             return Collections.singleton(KeyValue.pair(key, next.value)); | ||||
|  | @ -1028,7 +1031,7 @@ public class QueryableStateIntegrationTest { | |||
|     private Map<String, Long> fetchMap(final ReadOnlyWindowStore<String, Long> store, | ||||
|                                        final String key) { | ||||
| 
 | ||||
|         final WindowStoreIterator<Long> fetch = store.fetch(key, 0, System.currentTimeMillis()); | ||||
|         final WindowStoreIterator<Long> fetch = store.fetch(key, ofEpochMilli(0), ofEpochMilli(System.currentTimeMillis())); | ||||
|         if (fetch.hasNext()) { | ||||
|             final KeyValue<Long, Long> next = fetch.next(); | ||||
|             return Collections.singletonMap(key, next.value); | ||||
|  |  | |||
|  | @ -55,10 +55,11 @@ import java.util.Arrays; | |||
| 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 @@ public class RepartitionOptimizingIntegrationTest { | |||
| 
 | ||||
|         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 @@ public class RepartitionOptimizingIntegrationTest { | |||
|         assertThat(3, equalTo(processorValueCollector.size())); | ||||
|         assertThat(processorValueCollector, equalTo(expectedCollectedProcessorValues)); | ||||
| 
 | ||||
|         streams.close(5, TimeUnit.SECONDS); | ||||
|         streams.close(ofSeconds(5)); | ||||
|     } | ||||
| 
 | ||||
| 
 | ||||
|  |  | |||
|  | @ -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.ArrayList; | |||
| 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 @@ public class RepartitionWithMergeOptimizingIntegrationTest { | |||
|         final List<KeyValue<String, String>> 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)); | ||||
|     } | ||||
| 
 | ||||
| 
 | ||||
|  |  | |||
|  | @ -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 @@ public class RestoreIntegrationTest { | |||
|     @After | ||||
|     public void shutdown() { | ||||
|         if (kafkaStreams != null) { | ||||
|             kafkaStreams.close(30, TimeUnit.SECONDS); | ||||
|             kafkaStreams.close(Duration.ofSeconds(30)); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -32,6 +32,7 @@ import java.util.Arrays; | |||
| 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 class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest | |||
|             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 class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest | |||
|         leftStream.map(MockMapper.<Long, String>noOpKeyValueMapper()) | ||||
|                 .join(rightStream.flatMap(MockMapper.<Long, String>noOpFlatKeyValueMapper()) | ||||
|                                  .selectKey(MockMapper.<Long, String>selectKeyKeyValueMapper()), | ||||
|                        valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); | ||||
|                        valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC); | ||||
| 
 | ||||
|         runTest(expectedResult); | ||||
|     } | ||||
|  | @ -136,7 +137,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest | |||
|             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 class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest | |||
|         leftStream.map(MockMapper.<Long, String>noOpKeyValueMapper()) | ||||
|                 .leftJoin(rightStream.flatMap(MockMapper.<Long, String>noOpFlatKeyValueMapper()) | ||||
|                                      .selectKey(MockMapper.<Long, String>selectKeyKeyValueMapper()), | ||||
|                         valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); | ||||
|                         valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC); | ||||
| 
 | ||||
|         runTest(expectedResult); | ||||
|     } | ||||
|  | @ -193,7 +194,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest | |||
|             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 class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest | |||
|         leftStream.map(MockMapper.<Long, String>noOpKeyValueMapper()) | ||||
|                 .outerJoin(rightStream.flatMap(MockMapper.<Long, String>noOpFlatKeyValueMapper()) | ||||
|                                 .selectKey(MockMapper.<Long, String>selectKeyKeyValueMapper()), | ||||
|                         valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); | ||||
|                         valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC); | ||||
| 
 | ||||
|         runTest(expectedResult); | ||||
|     } | ||||
|  | @ -254,8 +255,8 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest | |||
|                         "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); | ||||
|     } | ||||
|  |  | |||
|  | @ -434,7 +434,7 @@ public class SuppressionIntegrationTest { | |||
|                     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.<String, Long, WindowStore<Bytes, byte[]>>as("counts").withCachingDisabled().withLoggingDisabled()); | ||||
| 
 | ||||
|         valueCounts | ||||
|  |  | |||
|  | @ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream; | |||
| 
 | ||||
| import org.junit.Test; | ||||
| 
 | ||||
| import static java.time.Duration.ofMillis; | ||||
| import static org.apache.kafka.streams.EqualityCheck.verifyEquality; | ||||
| import static org.apache.kafka.streams.EqualityCheck.verifyInEquality; | ||||
| import static org.junit.Assert.assertEquals; | ||||
|  | @ -31,29 +32,29 @@ public class JoinWindowsTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void validWindows() { | ||||
|         JoinWindows.of(ANY_OTHER_SIZE)   // [ -anyOtherSize ; anyOtherSize ] | ||||
|                    .before(ANY_SIZE)                    // [ -anySize ; anyOtherSize ] | ||||
|                    .before(0)                          // [ 0 ; anyOtherSize ] | ||||
|                    .before(-ANY_SIZE)                   // [ anySize ; anyOtherSize ] | ||||
|                    .before(-ANY_OTHER_SIZE);             // [ anyOtherSize ; anyOtherSize ] | ||||
|         JoinWindows.of(ofMillis(ANY_OTHER_SIZE))   // [ -anyOtherSize ; anyOtherSize ] | ||||
|                    .before(ofMillis(ANY_SIZE))                    // [ -anySize ; anyOtherSize ] | ||||
|                    .before(ofMillis(0))                          // [ 0 ; anyOtherSize ] | ||||
|                    .before(ofMillis(-ANY_SIZE))                   // [ anySize ; anyOtherSize ] | ||||
|                    .before(ofMillis(-ANY_OTHER_SIZE));             // [ anyOtherSize ; anyOtherSize ] | ||||
| 
 | ||||
|         JoinWindows.of(ANY_OTHER_SIZE)   // [ -anyOtherSize ; anyOtherSize ] | ||||
|                    .after(ANY_SIZE)                     // [ -anyOtherSize ; anySize ] | ||||
|                    .after(0)                           // [ -anyOtherSize ; 0 ] | ||||
|                    .after(-ANY_SIZE)                    // [ -anyOtherSize ; -anySize ] | ||||
|                    .after(-ANY_OTHER_SIZE);              // [ -anyOtherSize ; -anyOtherSize ] | ||||
|         JoinWindows.of(ofMillis(ANY_OTHER_SIZE))   // [ -anyOtherSize ; anyOtherSize ] | ||||
|                    .after(ofMillis(ANY_SIZE))                     // [ -anyOtherSize ; anySize ] | ||||
|                    .after(ofMillis(0))                           // [ -anyOtherSize ; 0 ] | ||||
|                    .after(ofMillis(-ANY_SIZE))                    // [ -anyOtherSize ; -anySize ] | ||||
|                    .after(ofMillis(-ANY_OTHER_SIZE));              // [ -anyOtherSize ; -anyOtherSize ] | ||||
|     } | ||||
| 
 | ||||
|     @Test(expected = IllegalArgumentException.class) | ||||
|     public void timeDifferenceMustNotBeNegative() { | ||||
|         JoinWindows.of(-1); | ||||
|         JoinWindows.of(ofMillis(-1)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void endTimeShouldNotBeBeforeStart() { | ||||
|         final JoinWindows windowSpec = JoinWindows.of(ANY_SIZE); | ||||
|         final JoinWindows windowSpec = JoinWindows.of(ofMillis(ANY_SIZE)); | ||||
|         try { | ||||
|             windowSpec.after(-ANY_SIZE - 1); | ||||
|             windowSpec.after(ofMillis(-ANY_SIZE - 1)); | ||||
|             fail("window end time should not be before window start time"); | ||||
|         } catch (final IllegalArgumentException e) { | ||||
|             // expected | ||||
|  | @ -62,9 +63,9 @@ public class JoinWindowsTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void startTimeShouldNotBeAfterEnd() { | ||||
|         final JoinWindows windowSpec = JoinWindows.of(ANY_SIZE); | ||||
|         final JoinWindows windowSpec = JoinWindows.of(ofMillis(ANY_SIZE)); | ||||
|         try { | ||||
|             windowSpec.before(-ANY_SIZE - 1); | ||||
|             windowSpec.before(ofMillis(-ANY_SIZE - 1)); | ||||
|             fail("window start time should not be after window end time"); | ||||
|         } catch (final IllegalArgumentException e) { | ||||
|             // expected | ||||
|  | @ -74,7 +75,7 @@ public class JoinWindowsTest { | |||
|     @Deprecated | ||||
|     @Test | ||||
|     public void untilShouldSetMaintainDuration() { | ||||
|         final JoinWindows windowSpec = JoinWindows.of(ANY_SIZE); | ||||
|         final JoinWindows windowSpec = JoinWindows.of(ofMillis(ANY_SIZE)); | ||||
|         final long windowSize = windowSpec.size(); | ||||
|         assertEquals(windowSize, windowSpec.until(windowSize).maintainMs()); | ||||
|     } | ||||
|  | @ -82,7 +83,7 @@ public class JoinWindowsTest { | |||
|     @Deprecated | ||||
|     @Test | ||||
|     public void retentionTimeMustNoBeSmallerThanWindowSize() { | ||||
|         final JoinWindows windowSpec = JoinWindows.of(ANY_SIZE); | ||||
|         final JoinWindows windowSpec = JoinWindows.of(ofMillis(ANY_SIZE)); | ||||
|         final long windowSize = windowSpec.size(); | ||||
|         try { | ||||
|             windowSpec.until(windowSize - 1); | ||||
|  | @ -94,10 +95,10 @@ public class JoinWindowsTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void gracePeriodShouldEnforceBoundaries() { | ||||
|         JoinWindows.of(3L).grace(0L); | ||||
|         JoinWindows.of(ofMillis(3L)).grace(ofMillis(0L)); | ||||
| 
 | ||||
|         try { | ||||
|             JoinWindows.of(3L).grace(-1L); | ||||
|             JoinWindows.of(ofMillis(3L)).grace(ofMillis(-1L)); | ||||
|             fail("should not accept negatives"); | ||||
|         } catch (final IllegalArgumentException e) { | ||||
|             //expected | ||||
|  | @ -106,58 +107,58 @@ public class JoinWindowsTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void equalsAndHashcodeShouldBeValidForPositiveCases() { | ||||
|         verifyEquality(JoinWindows.of(3), JoinWindows.of(3)); | ||||
|         verifyEquality(JoinWindows.of(ofMillis(3)), JoinWindows.of(ofMillis(3))); | ||||
| 
 | ||||
|         verifyEquality(JoinWindows.of(3).after(2), JoinWindows.of(3).after(2)); | ||||
|         verifyEquality(JoinWindows.of(ofMillis(3)).after(ofMillis(2)), JoinWindows.of(ofMillis(3)).after(ofMillis(2))); | ||||
| 
 | ||||
|         verifyEquality(JoinWindows.of(3).before(2), JoinWindows.of(3).before(2)); | ||||
|         verifyEquality(JoinWindows.of(ofMillis(3)).before(ofMillis(2)), JoinWindows.of(ofMillis(3)).before(ofMillis(2))); | ||||
| 
 | ||||
|         verifyEquality(JoinWindows.of(3).grace(2), JoinWindows.of(3).grace(2)); | ||||
|         verifyEquality(JoinWindows.of(ofMillis(3)).grace(ofMillis(2)), JoinWindows.of(ofMillis(3)).grace(ofMillis(2))); | ||||
| 
 | ||||
|         verifyEquality(JoinWindows.of(3).until(60), JoinWindows.of(3).until(60)); | ||||
|         verifyEquality(JoinWindows.of(ofMillis(3)).until(60), JoinWindows.of(ofMillis(3)).until(60)); | ||||
| 
 | ||||
|         verifyEquality( | ||||
|             JoinWindows.of(3).before(1).after(2).grace(3).until(60), | ||||
|             JoinWindows.of(3).before(1).after(2).grace(3).until(60) | ||||
|             JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60), | ||||
|             JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60) | ||||
|         ); | ||||
|         // JoinWindows is a little weird in that before and after set the same fields as of. | ||||
|         verifyEquality( | ||||
|             JoinWindows.of(9).before(1).after(2).grace(3).until(60), | ||||
|             JoinWindows.of(3).before(1).after(2).grace(3).until(60) | ||||
|             JoinWindows.of(ofMillis(9)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60), | ||||
|             JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60) | ||||
|         ); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void equalsAndHashcodeShouldBeValidForNegativeCases() { | ||||
|         verifyInEquality(JoinWindows.of(9), JoinWindows.of(3)); | ||||
|         verifyInEquality(JoinWindows.of(ofMillis(9)), JoinWindows.of(ofMillis(3))); | ||||
| 
 | ||||
|         verifyInEquality(JoinWindows.of(3).after(9), JoinWindows.of(3).after(2)); | ||||
|         verifyInEquality(JoinWindows.of(ofMillis(3)).after(ofMillis(9)), JoinWindows.of(ofMillis(3)).after(ofMillis(2))); | ||||
| 
 | ||||
|         verifyInEquality(JoinWindows.of(3).before(9), JoinWindows.of(3).before(2)); | ||||
|         verifyInEquality(JoinWindows.of(ofMillis(3)).before(ofMillis(9)), JoinWindows.of(ofMillis(3)).before(ofMillis(2))); | ||||
| 
 | ||||
|         verifyInEquality(JoinWindows.of(3).grace(9), JoinWindows.of(3).grace(2)); | ||||
|         verifyInEquality(JoinWindows.of(ofMillis(3)).grace(ofMillis(9)), JoinWindows.of(ofMillis(3)).grace(ofMillis(2))); | ||||
| 
 | ||||
|         verifyInEquality(JoinWindows.of(3).until(90), JoinWindows.of(3).until(60)); | ||||
|         verifyInEquality(JoinWindows.of(ofMillis(3)).until(90), JoinWindows.of(ofMillis(3)).until(60)); | ||||
| 
 | ||||
| 
 | ||||
|         verifyInEquality( | ||||
|             JoinWindows.of(3).before(9).after(2).grace(3).until(60), | ||||
|             JoinWindows.of(3).before(1).after(2).grace(3).until(60) | ||||
|             JoinWindows.of(ofMillis(3)).before(ofMillis(9)).after(ofMillis(2)).grace(ofMillis(3)).until(60), | ||||
|             JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60) | ||||
|         ); | ||||
| 
 | ||||
|         verifyInEquality( | ||||
|             JoinWindows.of(3).before(1).after(9).grace(3).until(60), | ||||
|             JoinWindows.of(3).before(1).after(2).grace(3).until(60) | ||||
|             JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(9)).grace(ofMillis(3)).until(60), | ||||
|             JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60) | ||||
|         ); | ||||
| 
 | ||||
|         verifyInEquality( | ||||
|             JoinWindows.of(3).before(1).after(2).grace(9).until(60), | ||||
|             JoinWindows.of(3).before(1).after(2).grace(3).until(60) | ||||
|             JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(9)).until(60), | ||||
|             JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60) | ||||
|         ); | ||||
| 
 | ||||
|         verifyInEquality( | ||||
|             JoinWindows.of(3).before(1).after(2).grace(3).until(90), | ||||
|             JoinWindows.of(3).before(1).after(2).grace(3).until(60) | ||||
|             JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(90), | ||||
|             JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60) | ||||
|         ); | ||||
|     } | ||||
| } | ||||
|  | @ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream; | |||
| 
 | ||||
| import org.junit.Test; | ||||
| 
 | ||||
| import static java.time.Duration.ofMillis; | ||||
| import static org.apache.kafka.streams.EqualityCheck.verifyEquality; | ||||
| import static org.apache.kafka.streams.EqualityCheck.verifyInEquality; | ||||
| import static org.junit.Assert.assertEquals; | ||||
|  | @ -28,23 +29,23 @@ public class SessionWindowsTest { | |||
|     @Test | ||||
|     public void shouldSetWindowGap() { | ||||
|         final long anyGap = 42L; | ||||
|         assertEquals(anyGap, SessionWindows.with(anyGap).inactivityGap()); | ||||
|         assertEquals(anyGap, SessionWindows.with(ofMillis(anyGap)).inactivityGap()); | ||||
|     } | ||||
| 
 | ||||
|     @Deprecated | ||||
|     @Test | ||||
|     public void shouldSetWindowRetentionTime() { | ||||
|         final long anyRetentionTime = 42L; | ||||
|         assertEquals(anyRetentionTime, SessionWindows.with(1).until(anyRetentionTime).maintainMs()); | ||||
|         assertEquals(anyRetentionTime, SessionWindows.with(ofMillis(1)).until(anyRetentionTime).maintainMs()); | ||||
|     } | ||||
| 
 | ||||
| 
 | ||||
|     @Test | ||||
|     public void gracePeriodShouldEnforceBoundaries() { | ||||
|         SessionWindows.with(3L).grace(0L); | ||||
|         SessionWindows.with(ofMillis(3L)).grace(ofMillis(0)); | ||||
| 
 | ||||
|         try { | ||||
|             SessionWindows.with(3L).grace(-1L); | ||||
|             SessionWindows.with(ofMillis(3L)).grace(ofMillis(-1L)); | ||||
|             fail("should not accept negatives"); | ||||
|         } catch (final IllegalArgumentException e) { | ||||
|             //expected | ||||
|  | @ -53,25 +54,25 @@ public class SessionWindowsTest { | |||
| 
 | ||||
|     @Test(expected = IllegalArgumentException.class) | ||||
|     public void windowSizeMustNotBeNegative() { | ||||
|         SessionWindows.with(-1); | ||||
|         SessionWindows.with(ofMillis(-1)); | ||||
|     } | ||||
| 
 | ||||
|     @Test(expected = IllegalArgumentException.class) | ||||
|     public void windowSizeMustNotBeZero() { | ||||
|         SessionWindows.with(0); | ||||
|         SessionWindows.with(ofMillis(0)); | ||||
|     } | ||||
| 
 | ||||
|     @SuppressWarnings("deprecation") // specifically testing deprecated apis | ||||
|     @Test | ||||
|     public void retentionTimeShouldBeGapIfGapIsLargerThanDefaultRetentionTime() { | ||||
|         final long windowGap = 2 * SessionWindows.with(1).maintainMs(); | ||||
|         assertEquals(windowGap, SessionWindows.with(windowGap).maintainMs()); | ||||
|         final long windowGap = 2 * SessionWindows.with(ofMillis(1)).maintainMs(); | ||||
|         assertEquals(windowGap, SessionWindows.with(ofMillis(windowGap)).maintainMs()); | ||||
|     } | ||||
| 
 | ||||
|     @Deprecated | ||||
|     @Test | ||||
|     public void retentionTimeMustNotBeNegative() { | ||||
|         final SessionWindows windowSpec = SessionWindows.with(42); | ||||
|         final SessionWindows windowSpec = SessionWindows.with(ofMillis(42)); | ||||
|         try { | ||||
|             windowSpec.until(41); | ||||
|             fail("should not accept retention time smaller than gap"); | ||||
|  | @ -82,27 +83,27 @@ public class SessionWindowsTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void equalsAndHashcodeShouldBeValidForPositiveCases() { | ||||
|         verifyEquality(SessionWindows.with(1), SessionWindows.with(1)); | ||||
|         verifyEquality(SessionWindows.with(ofMillis(1)), SessionWindows.with(ofMillis(1))); | ||||
| 
 | ||||
|         verifyEquality(SessionWindows.with(1).grace(6), SessionWindows.with(1).grace(6)); | ||||
|         verifyEquality(SessionWindows.with(ofMillis(1)).grace(ofMillis(6)), SessionWindows.with(ofMillis(1)).grace(ofMillis(6))); | ||||
| 
 | ||||
|         verifyEquality(SessionWindows.with(1).until(7), SessionWindows.with(1).until(7)); | ||||
|         verifyEquality(SessionWindows.with(ofMillis(1)).until(7), SessionWindows.with(ofMillis(1)).until(7)); | ||||
| 
 | ||||
|         verifyEquality(SessionWindows.with(1).grace(6).until(7), SessionWindows.with(1).grace(6).until(7)); | ||||
|         verifyEquality(SessionWindows.with(ofMillis(1)).grace(ofMillis(6)).until(7), SessionWindows.with(ofMillis(1)).grace(ofMillis(6)).until(7)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void equalsAndHashcodeShouldBeValidForNegativeCases() { | ||||
|         verifyInEquality(SessionWindows.with(9), SessionWindows.with(1)); | ||||
|         verifyInEquality(SessionWindows.with(ofMillis(9)), SessionWindows.with(ofMillis(1))); | ||||
| 
 | ||||
|         verifyInEquality(SessionWindows.with(1).grace(9), SessionWindows.with(1).grace(6)); | ||||
|         verifyInEquality(SessionWindows.with(ofMillis(1)).grace(ofMillis(9)), SessionWindows.with(ofMillis(1)).grace(ofMillis(6))); | ||||
| 
 | ||||
|         verifyInEquality(SessionWindows.with(1).until(9), SessionWindows.with(1).until(7)); | ||||
|         verifyInEquality(SessionWindows.with(ofMillis(1)).until(9), SessionWindows.with(ofMillis(1)).until(7)); | ||||
| 
 | ||||
|         verifyInEquality(SessionWindows.with(2).grace(6).until(7), SessionWindows.with(1).grace(6).until(7)); | ||||
|         verifyInEquality(SessionWindows.with(ofMillis(2)).grace(ofMillis(6)).until(7), SessionWindows.with(ofMillis(1)).grace(ofMillis(6)).until(7)); | ||||
| 
 | ||||
|         verifyInEquality(SessionWindows.with(1).grace(0).until(7), SessionWindows.with(1).grace(6).until(7)); | ||||
|         verifyInEquality(SessionWindows.with(ofMillis(1)).grace(ofMillis(0)).until(7), SessionWindows.with(ofMillis(1)).grace(ofMillis(6)).until(7)); | ||||
| 
 | ||||
|         verifyInEquality(SessionWindows.with(1).grace(6).until(70), SessionWindows.with(1).grace(6).until(7)); | ||||
|         verifyInEquality(SessionWindows.with(ofMillis(1)).grace(ofMillis(6)).until(70), SessionWindows.with(ofMillis(1)).grace(ofMillis(6)).until(7)); | ||||
|     } | ||||
| } | ||||
|  | @ -21,6 +21,7 @@ import org.junit.Test; | |||
| 
 | ||||
| import java.util.Map; | ||||
| 
 | ||||
| import static java.time.Duration.ofMillis; | ||||
| import static org.apache.kafka.streams.EqualityCheck.verifyEquality; | ||||
| import static org.apache.kafka.streams.EqualityCheck.verifyInEquality; | ||||
| import static org.junit.Assert.assertEquals; | ||||
|  | @ -33,43 +34,43 @@ public class TimeWindowsTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void shouldSetWindowSize() { | ||||
|         assertEquals(ANY_SIZE, TimeWindows.of(ANY_SIZE).sizeMs); | ||||
|         assertEquals(ANY_SIZE, TimeWindows.of(ofMillis(ANY_SIZE)).sizeMs); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void shouldSetWindowAdvance() { | ||||
|         final long anyAdvance = 4; | ||||
|         assertEquals(anyAdvance, TimeWindows.of(ANY_SIZE).advanceBy(anyAdvance).advanceMs); | ||||
|         assertEquals(anyAdvance, TimeWindows.of(ofMillis(ANY_SIZE)).advanceBy(ofMillis(anyAdvance)).advanceMs); | ||||
|     } | ||||
| 
 | ||||
|     @SuppressWarnings("deprecation") // specifically testing deprecated APIs | ||||
|     @Test | ||||
|     public void shouldSetWindowRetentionTime() { | ||||
|         assertEquals(ANY_SIZE, TimeWindows.of(ANY_SIZE).until(ANY_SIZE).maintainMs()); | ||||
|         assertEquals(ANY_SIZE, TimeWindows.of(ofMillis(ANY_SIZE)).until(ANY_SIZE).maintainMs()); | ||||
|     } | ||||
| 
 | ||||
|     @SuppressWarnings("deprecation") // specifically testing deprecated APIs | ||||
|     @Test | ||||
|     public void shouldUseWindowSizeAsRentitionTimeIfWindowSizeIsLargerThanDefaultRetentionTime() { | ||||
|         final long windowSize = 2 * TimeWindows.of(1).maintainMs(); | ||||
|         assertEquals(windowSize, TimeWindows.of(windowSize).maintainMs()); | ||||
|         final long windowSize = 2 * TimeWindows.of(ofMillis(1)).maintainMs(); | ||||
|         assertEquals(windowSize, TimeWindows.of(ofMillis(windowSize)).maintainMs()); | ||||
|     } | ||||
| 
 | ||||
|     @Test(expected = IllegalArgumentException.class) | ||||
|     public void windowSizeMustNotBeZero() { | ||||
|         TimeWindows.of(0); | ||||
|         TimeWindows.of(ofMillis(0)); | ||||
|     } | ||||
| 
 | ||||
|     @Test(expected = IllegalArgumentException.class) | ||||
|     public void windowSizeMustNotBeNegative() { | ||||
|         TimeWindows.of(-1); | ||||
|         TimeWindows.of(ofMillis(-1)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void advanceIntervalMustNotBeZero() { | ||||
|         final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); | ||||
|         final TimeWindows windowSpec = TimeWindows.of(ofMillis(ANY_SIZE)); | ||||
|         try { | ||||
|             windowSpec.advanceBy(0); | ||||
|             windowSpec.advanceBy(ofMillis(0)); | ||||
|             fail("should not accept zero advance parameter"); | ||||
|         } catch (final IllegalArgumentException e) { | ||||
|             // expected | ||||
|  | @ -78,9 +79,9 @@ public class TimeWindowsTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void advanceIntervalMustNotBeNegative() { | ||||
|         final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); | ||||
|         final TimeWindows windowSpec = TimeWindows.of(ofMillis(ANY_SIZE)); | ||||
|         try { | ||||
|             windowSpec.advanceBy(-1); | ||||
|             windowSpec.advanceBy(ofMillis(-1)); | ||||
|             fail("should not accept negative advance parameter"); | ||||
|         } catch (final IllegalArgumentException e) { | ||||
|             // expected | ||||
|  | @ -90,9 +91,9 @@ public class TimeWindowsTest { | |||
|     @Deprecated | ||||
|     @Test | ||||
|     public void advanceIntervalMustNotBeLargerThanWindowSize() { | ||||
|         final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); | ||||
|         final TimeWindows windowSpec = TimeWindows.of(ofMillis(ANY_SIZE)); | ||||
|         try { | ||||
|             windowSpec.advanceBy(ANY_SIZE + 1); | ||||
|             windowSpec.advanceBy(ofMillis(ANY_SIZE + 1)); | ||||
|             fail("should not accept advance greater than window size"); | ||||
|         } catch (final IllegalArgumentException e) { | ||||
|             // expected | ||||
|  | @ -102,7 +103,7 @@ public class TimeWindowsTest { | |||
|     @Deprecated | ||||
|     @Test | ||||
|     public void retentionTimeMustNoBeSmallerThanWindowSize() { | ||||
|         final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); | ||||
|         final TimeWindows windowSpec = TimeWindows.of(ofMillis(ANY_SIZE)); | ||||
|         try { | ||||
|             windowSpec.until(ANY_SIZE - 1); | ||||
|             fail("should not accept retention time smaller than window size"); | ||||
|  | @ -113,10 +114,10 @@ public class TimeWindowsTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void gracePeriodShouldEnforceBoundaries() { | ||||
|         TimeWindows.of(3L).grace(0L); | ||||
|         TimeWindows.of(ofMillis(3L)).grace(ofMillis(0L)); | ||||
| 
 | ||||
|         try { | ||||
|             TimeWindows.of(3L).grace(-1L); | ||||
|             TimeWindows.of(ofMillis(3L)).grace(ofMillis(-1L)); | ||||
|             fail("should not accept negatives"); | ||||
|         } catch (final IllegalArgumentException e) { | ||||
|             //expected | ||||
|  | @ -125,7 +126,7 @@ public class TimeWindowsTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void shouldComputeWindowsForHoppingWindows() { | ||||
|         final TimeWindows windows = TimeWindows.of(12L).advanceBy(5L); | ||||
|         final TimeWindows windows = TimeWindows.of(ofMillis(12L)).advanceBy(ofMillis(5L)); | ||||
|         final Map<Long, TimeWindow> matched = windows.windowsFor(21L); | ||||
|         assertEquals(12L / 5L + 1, matched.size()); | ||||
|         assertEquals(new TimeWindow(10L, 22L), matched.get(10L)); | ||||
|  | @ -135,7 +136,7 @@ public class TimeWindowsTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void shouldComputeWindowsForBarelyOverlappingHoppingWindows() { | ||||
|         final TimeWindows windows = TimeWindows.of(6L).advanceBy(5L); | ||||
|         final TimeWindows windows = TimeWindows.of(ofMillis(6L)).advanceBy(ofMillis(5L)); | ||||
|         final Map<Long, TimeWindow> matched = windows.windowsFor(7L); | ||||
|         assertEquals(1, matched.size()); | ||||
|         assertEquals(new TimeWindow(5L, 11L), matched.get(5L)); | ||||
|  | @ -143,7 +144,7 @@ public class TimeWindowsTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void shouldComputeWindowsForTumblingWindows() { | ||||
|         final TimeWindows windows = TimeWindows.of(12L); | ||||
|         final TimeWindows windows = TimeWindows.of(ofMillis(12L)); | ||||
|         final Map<Long, TimeWindow> matched = windows.windowsFor(21L); | ||||
|         assertEquals(1, matched.size()); | ||||
|         assertEquals(new TimeWindow(12L, 24L), matched.get(12L)); | ||||
|  | @ -152,49 +153,49 @@ public class TimeWindowsTest { | |||
| 
 | ||||
|     @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) | ||||
|         ); | ||||
|     } | ||||
| } | ||||
|  | @ -21,6 +21,7 @@ import org.junit.Test; | |||
| 
 | ||||
| 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 class UnlimitedWindowsTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void shouldIncludeRecordsThatHappenedOnWindowStart() { | ||||
|         final UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime); | ||||
|         final UnlimitedWindows w = UnlimitedWindows.of().startOn(ofEpochMilli(anyStartTime)); | ||||
|         final Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(w.startMs); | ||||
|         assertEquals(1, matchedWindows.size()); | ||||
|         assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime)); | ||||
|  | @ -62,7 +63,7 @@ public class UnlimitedWindowsTest { | |||
| 
 | ||||
|     @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<Long, UnlimitedWindow> matchedWindows = w.windowsFor(timestamp); | ||||
|         assertEquals(1, matchedWindows.size()); | ||||
|  | @ -71,7 +72,7 @@ public class UnlimitedWindowsTest { | |||
| 
 | ||||
|     @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<Long, UnlimitedWindow> matchedWindows = w.windowsFor(timestamp); | ||||
|         assertTrue(matchedWindows.isEmpty()); | ||||
|  | @ -81,13 +82,13 @@ public class UnlimitedWindowsTest { | |||
|     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))); | ||||
|     } | ||||
| 
 | ||||
| } | ||||
|  | @ -59,6 +59,7 @@ import java.util.List; | |||
| 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 class KGroupedStreamImplTest { | |||
| 
 | ||||
|     @Test(expected = NullPointerException.class) | ||||
|     public void shouldNotHaveNullReducerWithWindowedReduce() { | ||||
|         groupedStream.windowedBy(TimeWindows.of(10)).reduce(null, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("store")); | ||||
|         groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).reduce(null, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("store")); | ||||
|     } | ||||
| 
 | ||||
|     @Test(expected = NullPointerException.class) | ||||
|  | @ -106,7 +107,7 @@ public class KGroupedStreamImplTest { | |||
| 
 | ||||
|     @Test(expected = InvalidTopicException.class) | ||||
|     public void shouldNotHaveInvalidStoreNameWithWindowedReduce() { | ||||
|         groupedStream.windowedBy(TimeWindows.of(10)).reduce(MockReducer.STRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as(INVALID_STORE_NAME)); | ||||
|         groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).reduce(MockReducer.STRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as(INVALID_STORE_NAME)); | ||||
|     } | ||||
| 
 | ||||
|     @Test(expected = NullPointerException.class) | ||||
|  | @ -126,12 +127,12 @@ public class KGroupedStreamImplTest { | |||
| 
 | ||||
|     @Test(expected = NullPointerException.class) | ||||
|     public void shouldNotHaveNullInitializerOnWindowedAggregate() { | ||||
|         groupedStream.windowedBy(TimeWindows.of(10)).aggregate(null, MockAggregator.TOSTRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("store")); | ||||
|         groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).aggregate(null, MockAggregator.TOSTRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("store")); | ||||
|     } | ||||
| 
 | ||||
|     @Test(expected = NullPointerException.class) | ||||
|     public void shouldNotHaveNullAdderOnWindowedAggregate() { | ||||
|         groupedStream.windowedBy(TimeWindows.of(10)).aggregate(MockInitializer.STRING_INIT, null, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("store")); | ||||
|         groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).aggregate(MockInitializer.STRING_INIT, null, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("store")); | ||||
|     } | ||||
| 
 | ||||
|     @Test(expected = NullPointerException.class) | ||||
|  | @ -141,7 +142,7 @@ public class KGroupedStreamImplTest { | |||
| 
 | ||||
|     @Test(expected = InvalidTopicException.class) | ||||
|     public void shouldNotHaveInvalidStoreNameOnWindowedAggregate() { | ||||
|         groupedStream.windowedBy(TimeWindows.of(10)).aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as(INVALID_STORE_NAME)); | ||||
|         groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as(INVALID_STORE_NAME)); | ||||
|     } | ||||
| 
 | ||||
|     private void doAggregateSessionWindows(final Map<Windowed<String>, Integer> results) { | ||||
|  | @ -161,7 +162,7 @@ public class KGroupedStreamImplTest { | |||
|     @Test | ||||
|     public void shouldAggregateSessionWindows() { | ||||
|         final Map<Windowed<String>, Integer> results = new HashMap<>(); | ||||
|         final KTable<Windowed<String>, Integer> table = groupedStream.windowedBy(SessionWindows.with(30)).aggregate(new Initializer<Integer>() { | ||||
|         final KTable<Windowed<String>, Integer> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(new Initializer<Integer>() { | ||||
|             @Override | ||||
|             public Integer apply() { | ||||
|                 return 0; | ||||
|  | @ -191,7 +192,7 @@ public class KGroupedStreamImplTest { | |||
|     @Test | ||||
|     public void shouldAggregateSessionWindowsWithInternalStoreName() { | ||||
|         final Map<Windowed<String>, Integer> results = new HashMap<>(); | ||||
|         final KTable<Windowed<String>, Integer> table = groupedStream.windowedBy(SessionWindows.with(30)).aggregate(new Initializer<Integer>() { | ||||
|         final KTable<Windowed<String>, Integer> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(new Initializer<Integer>() { | ||||
|             @Override | ||||
|             public Integer apply() { | ||||
|                 return 0; | ||||
|  | @ -234,7 +235,7 @@ public class KGroupedStreamImplTest { | |||
|     @Test | ||||
|     public void shouldCountSessionWindows() { | ||||
|         final Map<Windowed<String>, Long> results = new HashMap<>(); | ||||
|         final KTable<Windowed<String>, Long> table = groupedStream.windowedBy(SessionWindows.with(30)) | ||||
|         final KTable<Windowed<String>, Long> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))) | ||||
|                 .count(Materialized.<String, Long, SessionStore<Bytes, byte[]>>as("session-store")); | ||||
|         table.toStream().foreach(new ForeachAction<Windowed<String>, Long>() { | ||||
|             @Override | ||||
|  | @ -249,7 +250,7 @@ public class KGroupedStreamImplTest { | |||
|     @Test | ||||
|     public void shouldCountSessionWindowsWithInternalStoreName() { | ||||
|         final Map<Windowed<String>, Long> results = new HashMap<>(); | ||||
|         final KTable<Windowed<String>, Long> table = groupedStream.windowedBy(SessionWindows.with(30)).count(); | ||||
|         final KTable<Windowed<String>, Long> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).count(); | ||||
|         table.toStream().foreach(new ForeachAction<Windowed<String>, Long>() { | ||||
|             @Override | ||||
|             public void apply(final Windowed<String> key, final Long value) { | ||||
|  | @ -277,7 +278,7 @@ public class KGroupedStreamImplTest { | |||
|     @Test | ||||
|     public void shouldReduceSessionWindows() { | ||||
|         final Map<Windowed<String>, String> results = new HashMap<>(); | ||||
|         final KTable<Windowed<String>, String> table = groupedStream.windowedBy(SessionWindows.with(30)) | ||||
|         final KTable<Windowed<String>, String> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))) | ||||
|                 .reduce(new Reducer<String>() { | ||||
|                     @Override | ||||
|                     public String apply(final String value1, final String value2) { | ||||
|  | @ -297,7 +298,7 @@ public class KGroupedStreamImplTest { | |||
|     @Test | ||||
|     public void shouldReduceSessionWindowsWithInternalStoreName() { | ||||
|         final Map<Windowed<String>, String> results = new HashMap<>(); | ||||
|         final KTable<Windowed<String>, String> table = groupedStream.windowedBy(SessionWindows.with(30)) | ||||
|         final KTable<Windowed<String>, String> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))) | ||||
|                 .reduce(new Reducer<String>() { | ||||
|                     @Override | ||||
|                     public String apply(final String value1, final String value2) { | ||||
|  | @ -316,7 +317,7 @@ public class KGroupedStreamImplTest { | |||
| 
 | ||||
|     @Test(expected = NullPointerException.class) | ||||
|     public void shouldNotAcceptNullReducerWhenReducingSessionWindows() { | ||||
|         groupedStream.windowedBy(SessionWindows.with(30)).reduce(null, Materialized.<String, String, SessionStore<Bytes, byte[]>>as("store")); | ||||
|         groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).reduce(null, Materialized.<String, String, SessionStore<Bytes, byte[]>>as("store")); | ||||
|     } | ||||
| 
 | ||||
|     @Test(expected = NullPointerException.class) | ||||
|  | @ -326,17 +327,17 @@ public class KGroupedStreamImplTest { | |||
| 
 | ||||
|     @Test(expected = InvalidTopicException.class) | ||||
|     public void shouldNotAcceptInvalidStoreNameWhenReducingSessionWindows() { | ||||
|         groupedStream.windowedBy(SessionWindows.with(30)).reduce(MockReducer.STRING_ADDER, Materialized.<String, String, SessionStore<Bytes, byte[]>>as(INVALID_STORE_NAME)); | ||||
|         groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).reduce(MockReducer.STRING_ADDER, Materialized.<String, String, SessionStore<Bytes, byte[]>>as(INVALID_STORE_NAME)); | ||||
|     } | ||||
| 
 | ||||
|     @Test(expected = NullPointerException.class) | ||||
|     public void shouldNotAcceptNullStateStoreSupplierWhenReducingSessionWindows() { | ||||
|         groupedStream.windowedBy(SessionWindows.with(30)).reduce(null, Materialized.<String, String, SessionStore<Bytes, byte[]>>as(null)); | ||||
|         groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).reduce(null, Materialized.<String, String, SessionStore<Bytes, byte[]>>as(null)); | ||||
|     } | ||||
| 
 | ||||
|     @Test(expected = NullPointerException.class) | ||||
|     public void shouldNotAcceptNullInitializerWhenAggregatingSessionWindows() { | ||||
|         groupedStream.windowedBy(SessionWindows.with(30)).aggregate(null, MockAggregator.TOSTRING_ADDER, new Merger<String, String>() { | ||||
|         groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(null, MockAggregator.TOSTRING_ADDER, new Merger<String, String>() { | ||||
|             @Override | ||||
|             public String apply(final String aggKey, final String aggOne, final String aggTwo) { | ||||
|                 return null; | ||||
|  | @ -346,7 +347,7 @@ public class KGroupedStreamImplTest { | |||
| 
 | ||||
|     @Test(expected = NullPointerException.class) | ||||
|     public void shouldNotAcceptNullAggregatorWhenAggregatingSessionWindows() { | ||||
|         groupedStream.windowedBy(SessionWindows.with(30)).aggregate(MockInitializer.STRING_INIT, null, new Merger<String, String>() { | ||||
|         groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(MockInitializer.STRING_INIT, null, new Merger<String, String>() { | ||||
|             @Override | ||||
|             public String apply(final String aggKey, final String aggOne, final String aggTwo) { | ||||
|                 return null; | ||||
|  | @ -356,7 +357,7 @@ public class KGroupedStreamImplTest { | |||
| 
 | ||||
|     @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.<String, String, SessionStore<Bytes, byte[]>>as("storeName")); | ||||
|     } | ||||
|  | @ -368,7 +369,7 @@ public class KGroupedStreamImplTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void shouldAcceptNullStoreNameWhenAggregatingSessionWindows() { | ||||
|         groupedStream.windowedBy(SessionWindows.with(10)) | ||||
|         groupedStream.windowedBy(SessionWindows.with(ofMillis(10))) | ||||
|                 .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, new Merger<String, String>() { | ||||
|                     @Override | ||||
|                     public String apply(final String aggKey, final String aggOne, final String aggTwo) { | ||||
|  | @ -379,7 +380,7 @@ public class KGroupedStreamImplTest { | |||
| 
 | ||||
|     @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<String, String>() { | ||||
|                     @Override | ||||
|                     public String apply(final String aggKey, final String aggOne, final String aggTwo) { | ||||
|  | @ -558,7 +559,7 @@ public class KGroupedStreamImplTest { | |||
|     @Test | ||||
|     public void shouldCountWindowed() { | ||||
|         final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>(); | ||||
|         groupedStream.windowedBy(TimeWindows.of(500L)).count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("aggregate-by-key-windowed")) | ||||
|         groupedStream.windowedBy(TimeWindows.of(ofMillis(500L))).count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("aggregate-by-key-windowed")) | ||||
|             .toStream() | ||||
|             .foreach(new ForeachAction<Windowed<String>, Long>() { | ||||
|                 @Override | ||||
|  | @ -573,7 +574,7 @@ public class KGroupedStreamImplTest { | |||
|     @Test | ||||
|     public void shouldCountWindowedWithInternalStoreName() { | ||||
|         final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>(); | ||||
|         groupedStream.windowedBy(TimeWindows.of(500L)).count() | ||||
|         groupedStream.windowedBy(TimeWindows.of(ofMillis(500L))).count() | ||||
|             .toStream() | ||||
|             .foreach(new ForeachAction<Windowed<String>, Long>() { | ||||
|                 @Override | ||||
|  |  | |||
|  | @ -63,6 +63,7 @@ import java.util.Properties; | |||
| 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 class KStreamImplTest { | |||
|             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<Integer, Integer, Integer>() { | ||||
|             @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 class KStreamImplTest { | |||
|             }); | ||||
|         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 class KStreamImplTest { | |||
|         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 class KStreamImplTest { | |||
| 
 | ||||
|     @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 class KStreamImplTest { | |||
| 
 | ||||
|     @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 | ||||
|  |  | |||
|  | @ -41,6 +41,7 @@ import java.util.HashSet; | |||
| 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 class KStreamKStreamJoinTest { | |||
|                     return value1 + value2; | ||||
|                 } | ||||
|             }, | ||||
|             JoinWindows.of(100), | ||||
|             JoinWindows.of(ofMillis(100)), | ||||
|             Joined.with(Serdes.String(), Serdes.Integer(), Serdes.Integer()) | ||||
|         ); | ||||
| 
 | ||||
|  | @ -101,7 +102,7 @@ public class KStreamKStreamJoinTest { | |||
|         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 class KStreamKStreamJoinTest { | |||
|         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<Set<String>> copartitionGroups = TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups(); | ||||
|  | @ -308,7 +309,7 @@ public class KStreamKStreamJoinTest { | |||
|         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 class KStreamKStreamJoinTest { | |||
|         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 class KStreamKStreamJoinTest { | |||
|         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); | ||||
| 
 | ||||
|  |  | |||
|  | @ -39,6 +39,7 @@ import java.util.HashSet; | |||
| 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 class KStreamKStreamLeftJoinTest { | |||
| 
 | ||||
|         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 class KStreamKStreamLeftJoinTest { | |||
| 
 | ||||
|         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); | ||||
| 
 | ||||
|  |  | |||
|  | @ -50,6 +50,7 @@ import java.util.ArrayList; | |||
| 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<String, Long> sessionMerger = (aggKey, aggOne, aggTwo) -> aggOne + aggTwo; | ||||
|     private final KStreamSessionWindowAggregate<String, String, Long> sessionAggregator = | ||||
|         new KStreamSessionWindowAggregate<>( | ||||
|             SessionWindows.with(GAP_MS), | ||||
|             SessionWindows.with(ofMillis(GAP_MS)), | ||||
|             STORE_NAME, | ||||
|             initializer, | ||||
|             aggregator, | ||||
|  | @ -106,7 +107,7 @@ public class KStreamSessionWindowAggregateProcessorTest { | |||
|     } | ||||
| 
 | ||||
|     private void initStore(final boolean enableCaching) { | ||||
|         final StoreBuilder<SessionStore<String, Long>> storeBuilder = Stores.sessionStoreBuilder(Stores.persistentSessionStore(STORE_NAME, GAP_MS * 3), | ||||
|         final StoreBuilder<SessionStore<String, Long>> storeBuilder = Stores.sessionStoreBuilder(Stores.persistentSessionStore(STORE_NAME, ofMillis(GAP_MS * 3)), | ||||
|                                                                                                  Serdes.String(), | ||||
|                                                                                                  Serdes.Long()) | ||||
|             .withLoggingDisabled(); | ||||
|  | @ -322,7 +323,7 @@ public class KStreamSessionWindowAggregateProcessorTest { | |||
|         LogCaptureAppender.setClassLoggerToDebug(KStreamSessionWindowAggregate.class); | ||||
|         final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(); | ||||
|         final Processor<String, String> processor = new KStreamSessionWindowAggregate<>( | ||||
|             SessionWindows.with(10L).grace(10L), | ||||
|             SessionWindows.with(ofMillis(10L)).grace(ofMillis(10L)), | ||||
|             STORE_NAME, | ||||
|             initializer, | ||||
|             aggregator, | ||||
|  |  | |||
|  | @ -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 class KStreamTransformTest { | |||
| 
 | ||||
|                     @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); | ||||
|  |  | |||
|  | @ -47,6 +47,7 @@ import org.junit.Test; | |||
| 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 class KStreamWindowAggregateTest { | |||
|         final KTable<Windowed<String>, 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.<String, String, WindowStore<Bytes, byte[]>>as("topic1-Canonized").withValueSerde(Serdes.String())); | ||||
| 
 | ||||
|         final MockProcessorSupplier<Windowed<String>, String> supplier = new MockProcessorSupplier<>(); | ||||
|  | @ -128,7 +129,7 @@ public class KStreamWindowAggregateTest { | |||
|         final KTable<Windowed<String>, 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.<String, String, WindowStore<Bytes, byte[]>>as("topic1-Canonized").withValueSerde(Serdes.String())); | ||||
| 
 | ||||
|         final MockProcessorSupplier<Windowed<String>, String> supplier = new MockProcessorSupplier<>(); | ||||
|  | @ -137,7 +138,7 @@ public class KStreamWindowAggregateTest { | |||
|         final KTable<Windowed<String>, 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.<String, String, WindowStore<Bytes, byte[]>>as("topic2-Canonized").withValueSerde(Serdes.String())); | ||||
| 
 | ||||
|         table2.toStream().process(supplier); | ||||
|  | @ -232,7 +233,7 @@ public class KStreamWindowAggregateTest { | |||
| 
 | ||||
|         final KStream<String, String> 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 class KStreamWindowAggregateTest { | |||
| 
 | ||||
|         final KStream<String, String> 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 class KStreamWindowAggregateTest { | |||
| 
 | ||||
|         final KStream<String, String> 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("+"), | ||||
|  |  | |||
|  | @ -36,6 +36,7 @@ import org.junit.Test; | |||
| 
 | ||||
| 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 class KStreamWindowReduceTest { | |||
|         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 class KStreamWindowReduceTest { | |||
|         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)) | ||||
|  |  | |||
|  | @ -47,6 +47,7 @@ import java.util.List; | |||
| 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 class SessionWindowedKStreamImplTest { | |||
|     public void before() { | ||||
|         final KStream<String, String> 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 | ||||
|  |  | |||
|  | @ -368,7 +368,7 @@ public class SuppressScenarioTest { | |||
|         final KTable<Windowed<String>, 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.<String, Long, WindowStore<Bytes, byte[]>>as("counts").withCachingDisabled()); | ||||
|         valueCounts | ||||
|             .suppress(untilWindowCloses(unbounded())) | ||||
|  | @ -422,7 +422,7 @@ public class SuppressScenarioTest { | |||
|         final KTable<Windowed<String>, 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.<String, Long, WindowStore<Bytes, byte[]>>as("counts").withCachingDisabled().withKeySerde(STRING_SERDE)); | ||||
|         valueCounts | ||||
|             .suppress(untilWindowCloses(unbounded())) | ||||
|  | @ -481,7 +481,7 @@ public class SuppressScenarioTest { | |||
|         final KTable<Windowed<String>, 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.<String, Long, SessionStore<Bytes, byte[]>>as("counts").withCachingDisabled()); | ||||
|         valueCounts | ||||
|             .suppress(untilWindowCloses(unbounded())) | ||||
|  |  | |||
|  | @ -32,6 +32,7 @@ import org.junit.Test; | |||
| 
 | ||||
| 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 class SuppressTopologyTest { | |||
|         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.<String, Long, SessionStore<Bytes, byte[]>>as("counts").withCachingDisabled()) | ||||
|             .suppress(untilWindowCloses(unbounded())) | ||||
|             .toStream() | ||||
|  | @ -164,7 +165,7 @@ public class SuppressTopologyTest { | |||
|         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.<String, Long, SessionStore<Bytes, byte[]>>as("counts").withCachingDisabled()) | ||||
|             .suppress(untilWindowCloses(unbounded()).withName("myname")) | ||||
|             .toStream() | ||||
|  |  | |||
|  | @ -21,6 +21,7 @@ import org.junit.Test; | |||
| 
 | ||||
| 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 class TimeWindowTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void shouldReturnMatchedWindowsOrderedByTimestamp() { | ||||
|         final TimeWindows windows = TimeWindows.of(12L).advanceBy(5L); | ||||
|         final TimeWindows windows = TimeWindows.of(ofMillis(12L)).advanceBy(ofMillis(5L)); | ||||
|         final Map<Long, TimeWindow> matched = windows.windowsFor(21L); | ||||
| 
 | ||||
|         final Long[] expected = matched.keySet().toArray(new Long[matched.size()]); | ||||
|  |  | |||
|  | @ -46,6 +46,8 @@ import java.util.List; | |||
| 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<String, String> 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 class TimeWindowedKStreamImplTest { | |||
|         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) { | ||||
|             processData(driver); | ||||
|             final WindowStore<String, Long> windowStore = driver.getWindowStore("count-store"); | ||||
|             final List<KeyValue<Windowed<String>, Long>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", 0, 1000)); | ||||
|             final List<KeyValue<Windowed<String>, 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 class TimeWindowedKStreamImplTest { | |||
|         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) { | ||||
|             processData(driver); | ||||
|             final WindowStore<String, String> windowStore = driver.getWindowStore("reduced"); | ||||
|             final List<KeyValue<Windowed<String>, String>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", 0, 1000)); | ||||
|             final List<KeyValue<Windowed<String>, 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 class TimeWindowedKStreamImplTest { | |||
|         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) { | ||||
|             processData(driver); | ||||
|             final WindowStore<String, String> windowStore = driver.getWindowStore("aggregated"); | ||||
|             final List<KeyValue<Windowed<String>, String>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", 0, 1000)); | ||||
|             final List<KeyValue<Windowed<String>, 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"), | ||||
|  |  | |||
|  | @ -26,6 +26,7 @@ import org.apache.kafka.streams.processor.Processor; | |||
| 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 class GraphGraceSearchUtilTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void shouldExtractGraceFromKStreamWindowAggregateNode() { | ||||
|         final TimeWindows windows = TimeWindows.of(10L).grace(1234L); | ||||
|         final TimeWindows windows = TimeWindows.of(ofMillis(10L)).grace(ofMillis(1234L)); | ||||
|         final StatefulProcessorNode<String, Long> node = new StatefulProcessorNode<>( | ||||
|             "asdf", | ||||
|             new ProcessorParameters<>( | ||||
|  | @ -101,7 +102,7 @@ public class GraphGraceSearchUtilTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void shouldExtractGraceFromKStreamSessionWindowAggregateNode() { | ||||
|         final SessionWindows windows = SessionWindows.with(10L).grace(1234L); | ||||
|         final SessionWindows windows = SessionWindows.with(ofMillis(10L)).grace(ofMillis(1234L)); | ||||
| 
 | ||||
|         final StatefulProcessorNode<String, Long> node = new StatefulProcessorNode<>( | ||||
|             "asdf", | ||||
|  | @ -126,7 +127,7 @@ public class GraphGraceSearchUtilTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void shouldExtractGraceFromAncestorThroughStatefulParent() { | ||||
|         final SessionWindows windows = SessionWindows.with(10L).grace(1234L); | ||||
|         final SessionWindows windows = SessionWindows.with(ofMillis(10L)).grace(ofMillis(1234L)); | ||||
|         final StatefulProcessorNode<String, Long> graceGrandparent = new StatefulProcessorNode<>( | ||||
|             "asdf", | ||||
|             new ProcessorParameters<>(new KStreamSessionWindowAggregate<String, Long, Integer>( | ||||
|  | @ -167,7 +168,7 @@ public class GraphGraceSearchUtilTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void shouldExtractGraceFromAncestorThroughStatelessParent() { | ||||
|         final SessionWindows windows = SessionWindows.with(10L).grace(1234L); | ||||
|         final SessionWindows windows = SessionWindows.with(ofMillis(10L)).grace(ofMillis(1234L)); | ||||
|         final StatefulProcessorNode<String, Long> graceGrandparent = new StatefulProcessorNode<>( | ||||
|             "asdf", | ||||
|             new ProcessorParameters<>( | ||||
|  | @ -201,7 +202,7 @@ public class GraphGraceSearchUtilTest { | |||
|             "asdf", | ||||
|             new ProcessorParameters<>( | ||||
|                 new KStreamSessionWindowAggregate<String, Long, Integer>( | ||||
|                     SessionWindows.with(10L).grace(1234L), | ||||
|                     SessionWindows.with(ofMillis(10L)).grace(ofMillis(1234L)), | ||||
|                     "asdf", | ||||
|                     null, | ||||
|                     null, | ||||
|  | @ -218,7 +219,7 @@ public class GraphGraceSearchUtilTest { | |||
|             "asdf", | ||||
|             new ProcessorParameters<>( | ||||
|                 new KStreamWindowAggregate<String, Long, Integer, TimeWindow>( | ||||
|                     TimeWindows.of(10L).grace(4321L), | ||||
|                     TimeWindows.of(ofMillis(10L)).grace(ofMillis(4321L)), | ||||
|                     "asdf", | ||||
|                     null, | ||||
|                     null | ||||
|  |  | |||
|  | @ -34,6 +34,7 @@ import java.util.Properties; | |||
| 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 class StreamsGraphTest { | |||
|         final ValueJoiner<String, String, String> valueJoiner = (v, v2) -> v + v2; | ||||
| 
 | ||||
| 
 | ||||
|         final KStream<String, String> joinedStream = stream.join(streamII, valueJoiner, JoinWindows.of(5000)); | ||||
|         final KStream<String, String> joinedStream = stream.join(streamII, valueJoiner, JoinWindows.of(ofMillis(5000))); | ||||
| 
 | ||||
|         // build step one | ||||
|         assertEquals(expectedJoinedTopology, builder.build().describe().toString()); | ||||
|  | @ -100,7 +101,7 @@ public class StreamsGraphTest { | |||
|         final KStream<String, String> 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 @@ public class StreamsGraphTest { | |||
|         final KStream<String, String> 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); | ||||
| 
 | ||||
|  |  | |||
|  | @ -54,7 +54,6 @@ import org.apache.kafka.streams.state.Stores; | |||
| 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.Random; | |||
| 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 @@ public class SimpleBenchmark { | |||
|             consumer.seekToBeginning(partitions); | ||||
| 
 | ||||
|             while (true) { | ||||
|                 final ConsumerRecords<Integer, byte[]> records = consumer.poll(Duration.ofMillis(POLL_MS)); | ||||
|                 final ConsumerRecords<Integer, byte[]> records = consumer.poll(ofMillis(POLL_MS)); | ||||
|                 if (records.isEmpty()) { | ||||
|                     if (processedRecords == numRecords) { | ||||
|                         break; | ||||
|  | @ -370,7 +373,7 @@ public class SimpleBenchmark { | |||
|             consumer.seekToBeginning(partitions); | ||||
| 
 | ||||
|             while (true) { | ||||
|                 final ConsumerRecords<Integer, byte[]> records = consumer.poll(Duration.ofMillis(POLL_MS)); | ||||
|                 final ConsumerRecords<Integer, byte[]> records = consumer.poll(ofMillis(POLL_MS)); | ||||
|                 if (records.isEmpty()) { | ||||
|                     if (processedRecords == numRecords) { | ||||
|                         break; | ||||
|  | @ -498,7 +501,7 @@ public class SimpleBenchmark { | |||
|                     @Override | ||||
|                     public void process(final Integer key, final byte[] value) { | ||||
|                         final long timestamp = context().timestamp(); | ||||
|                         final KeyValueIterator<Windowed<Integer>, byte[]> iter = store.fetch(key - 10, key + 10, timestamp - 1000L, timestamp + 1000L); | ||||
|                         final KeyValueIterator<Windowed<Integer>, byte[]> iter = store.fetch(key - 10, key + 10, ofEpochMilli(timestamp - 1000L), ofEpochMilli(timestamp)); | ||||
|                         while (iter.hasNext()) { | ||||
|                             iter.next(); | ||||
|                         } | ||||
|  | @ -550,7 +553,7 @@ public class SimpleBenchmark { | |||
| 
 | ||||
|         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 @@ public class SimpleBenchmark { | |||
|         final KStream<Integer, byte[]> input1 = builder.stream(kStreamTopic1); | ||||
|         final KStream<Integer, byte[]> 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 @@ public class SimpleBenchmark { | |||
|             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)); | ||||
|             } | ||||
|         }); | ||||
| 
 | ||||
|  |  | |||
|  | @ -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 class YahooBenchmark { | |||
|         // 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.<String, Long, WindowStore<Bytes, byte[]>>as("time-windows")); | ||||
| 
 | ||||
|         return new KafkaStreams(builder.build(), streamConfig); | ||||
|  |  | |||
|  | @ -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 class AbstractProcessorContextTest { | |||
|             return null; | ||||
|         } | ||||
| 
 | ||||
|         @Override | ||||
|         public Cancellable schedule(final Duration interval, | ||||
|                                     final PunctuationType type, | ||||
|                                     final Punctuator callback) throws IllegalArgumentException { | ||||
|             return null; | ||||
|         } | ||||
| 
 | ||||
|         @Override | ||||
|         public <K, V> void forward(final K key, final V value) {} | ||||
| 
 | ||||
|  |  | |||
|  | @ -47,6 +47,7 @@ import java.util.Map; | |||
| 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 class InternalTopologyBuilderTest { | |||
|         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 class InternalTopologyBuilderTest { | |||
|         ); | ||||
|         builder.addStateStore( | ||||
|                 Stores.sessionStoreBuilder( | ||||
|                         Stores.persistentSessionStore("store2", 30000), Serdes.String(), Serdes.String() | ||||
|                         Stores.persistentSessionStore("store2", ofSeconds(30)), Serdes.String(), Serdes.String() | ||||
|                 ), | ||||
|                 "processor" | ||||
|         ); | ||||
|  |  | |||
|  | @ -72,6 +72,7 @@ import java.util.Map; | |||
| 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 class StandbyTaskTest { | |||
|         } | ||||
| 
 | ||||
|         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 class StandbyTaskTest { | |||
|         builder | ||||
|             .stream(Collections.singleton("topic"), new ConsumedInternal<>()) | ||||
|             .groupByKey() | ||||
|             .windowedBy(TimeWindows.of(60_000).grace(0L)) | ||||
|             .count(Materialized.<Object, Long, WindowStore<Bytes, byte[]>>as(storeName).withRetention(120_000L)); | ||||
|             .windowedBy(TimeWindows.of(ofMillis(60_000)).grace(ofMillis(0L))) | ||||
|             .count(Materialized.<Object, Long, WindowStore<Bytes, byte[]>>as(storeName).withRetention(ofMillis(120_000L))); | ||||
| 
 | ||||
|         builder.buildAndOptimizeTopology(); | ||||
| 
 | ||||
|  | @ -484,7 +485,7 @@ public class StandbyTaskTest { | |||
|     @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); | ||||
|     } | ||||
|  |  | |||
|  | @ -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 class StreamThreadTest { | |||
|                 return new Processor<Object, Object>() { | ||||
|                     @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); | ||||
|  |  | |||
|  | @ -60,6 +60,7 @@ import java.util.Set; | |||
| 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 class StreamsPartitionAssignorTest { | |||
|                         return null; | ||||
|                     } | ||||
|                 }, | ||||
|                 JoinWindows.of(0) | ||||
|                 JoinWindows.of(ofMillis(0)) | ||||
|             ); | ||||
| 
 | ||||
|         final UUID uuid = UUID.randomUUID(); | ||||
|  |  | |||
|  | @ -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 class NoOpWindowStore implements ReadOnlyWindowStore, StateStore { | |||
|     } | ||||
| 
 | ||||
|     @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<KeyValue> 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<KeyValue> all() { | ||||
|         return EMPTY_WINDOW_STORE_ITERATOR; | ||||
|     } | ||||
|      | ||||
|     @Override | ||||
|     @SuppressWarnings("deprecation") | ||||
|     public WindowStoreIterator<KeyValue> 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; | ||||
|     } | ||||
| } | ||||
|  |  | |||
|  | @ -24,6 +24,7 @@ import org.apache.kafka.streams.state.internals.RocksDBStore; | |||
| 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 class StoresTest { | |||
| 
 | ||||
|     @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 class StoresTest { | |||
| 
 | ||||
|     @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 class StoresTest { | |||
| 
 | ||||
|     @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<String, String> store = Stores.windowStoreBuilder( | ||||
|             Stores.persistentWindowStore("store", 3L, 3L, true), | ||||
|             Stores.persistentWindowStore("store", ofMillis(3L), ofMillis(3L), true), | ||||
|             Serdes.String(), | ||||
|             Serdes.String() | ||||
|         ).build(); | ||||
|  | @ -151,7 +153,7 @@ public class StoresTest { | |||
|     @Test | ||||
|     public void shouldBuildSessionStore() { | ||||
|         final SessionStore<String, String> store = Stores.sessionStoreBuilder( | ||||
|             Stores.persistentSessionStore("name", 10), | ||||
|             Stores.persistentSessionStore("name", ofMillis(10)), | ||||
|             Serdes.String(), | ||||
|             Serdes.String() | ||||
|         ).build(); | ||||
|  |  | |||
|  | @ -49,6 +49,9 @@ import java.util.List; | |||
| 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 class CachingWindowStoreTest { | |||
|         final StreamsBuilder builder = new StreamsBuilder(); | ||||
| 
 | ||||
|         final StoreBuilder<WindowStore<String, String>> 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 class CachingWindowStoreTest { | |||
|         assertThat(cachingStore.fetch(bytesKey("c"), 10), equalTo(null)); | ||||
|         assertThat(cachingStore.fetch(bytesKey("a"), 0), equalTo(null)); | ||||
| 
 | ||||
|         final WindowStoreIterator<byte[]> a = cachingStore.fetch(bytesKey("a"), 10, 10); | ||||
|         final WindowStoreIterator<byte[]> b = cachingStore.fetch(bytesKey("b"), 10, 10); | ||||
|         final WindowStoreIterator<byte[]> a = cachingStore.fetch(bytesKey("a"), ofEpochMilli(10), ofEpochMilli(10)); | ||||
|         final WindowStoreIterator<byte[]> 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 class CachingWindowStoreTest { | |||
|         cachingStore.put(bytesKey("a"), bytesValue("a")); | ||||
|         cachingStore.put(bytesKey("b"), bytesValue("b")); | ||||
| 
 | ||||
|         final KeyValueIterator<Windowed<Bytes>, byte[]> iterator = cachingStore.fetch(bytesKey("a"), bytesKey("b"), 10, 10); | ||||
|         final KeyValueIterator<Windowed<Bytes>, 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 class CachingWindowStoreTest { | |||
|             cachingStore.put(bytesKey(array[i]), bytesValue(array[i])); | ||||
|         } | ||||
| 
 | ||||
|         final KeyValueIterator<Windowed<Bytes>, byte[]> iterator = cachingStore.fetchAll(0, 7); | ||||
|         final KeyValueIterator<Windowed<Bytes>, 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<Windowed<Bytes>, byte[]> iterator1 = cachingStore.fetchAll(2, 4); | ||||
|         final KeyValueIterator<Windowed<Bytes>, 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<Windowed<Bytes>, byte[]> iterator2 = cachingStore.fetchAll(5, 7); | ||||
|         final KeyValueIterator<Windowed<Bytes>, 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 class CachingWindowStoreTest { | |||
|         cachingStore.flush(); | ||||
|         cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP); | ||||
| 
 | ||||
|         final WindowStoreIterator<byte[]> fetch = cachingStore.fetch(bytesKey("1"), DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP); | ||||
|         final WindowStoreIterator<byte[]> 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 class CachingWindowStoreTest { | |||
|         cachingStore.put(bytesKey("1"), bytesValue("a"), DEFAULT_TIMESTAMP); | ||||
|         cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE); | ||||
| 
 | ||||
|         final WindowStoreIterator<byte[]> fetch = cachingStore.fetch(bytesKey("1"), DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE); | ||||
|         final WindowStoreIterator<byte[]> 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 class CachingWindowStoreTest { | |||
|         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<byte[]> fetch = cachingStore.fetch(bytesKey("1"), DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE); | ||||
|         final WindowStoreIterator<byte[]> 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 class CachingWindowStoreTest { | |||
|         cachingStore.put(key, bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE); | ||||
| 
 | ||||
|         final KeyValueIterator<Windowed<Bytes>, 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 class CachingWindowStoreTest { | |||
|     @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 class CachingWindowStoreTest { | |||
|             KeyValue.pair(1L, bytesValue("0003")), | ||||
|             KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")) | ||||
|         ); | ||||
|         final List<KeyValue<Long, byte[]>> actual = toList(cachingStore.fetch(bytesKey("a"), 0, Long.MAX_VALUE)); | ||||
|         final List<KeyValue<Long, byte[]>> actual = toList(cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); | ||||
|         verifyKeyValueList(expected, actual); | ||||
|     } | ||||
| 
 | ||||
|  | @ -433,12 +436,12 @@ public class CachingWindowStoreTest { | |||
|                 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 class CachingWindowStoreTest { | |||
|                 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 class CachingWindowStoreTest { | |||
| 
 | ||||
|     @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<Windowed<Bytes>, byte[]> windowedPair(final String key, final String value, final long timestamp) { | ||||
|  |  | |||
|  | @ -36,6 +36,7 @@ import org.junit.runner.RunWith; | |||
| 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 class ChangeLoggingWindowBytesStoreTest { | |||
| 
 | ||||
|         init(); | ||||
| 
 | ||||
|         store.fetch(bytesKey, 0, 10); | ||||
|         store.fetch(bytesKey, ofEpochMilli(0), ofEpochMilli(10)); | ||||
|         EasyMock.verify(inner); | ||||
|     } | ||||
| 
 | ||||
|  | @ -111,7 +112,7 @@ public class ChangeLoggingWindowBytesStoreTest { | |||
| 
 | ||||
|         init(); | ||||
| 
 | ||||
|         store.fetch(bytesKey, bytesKey, 0, 1); | ||||
|         store.fetch(bytesKey, bytesKey, ofEpochMilli(0), ofEpochMilli(1)); | ||||
|         EasyMock.verify(inner); | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -35,6 +35,7 @@ import java.util.Collections; | |||
| 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 class CompositeReadOnlyWindowStoreTest { | |||
|         underlyingWindowStore.put("my-key", "my-value", 0L); | ||||
|         underlyingWindowStore.put("my-key", "my-later-value", 10L); | ||||
| 
 | ||||
|         final WindowStoreIterator<String> iterator = windowStore.fetch("my-key", 0L, 25L); | ||||
|         final WindowStoreIterator<String> iterator = windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); | ||||
|         final List<KeyValue<Long, String>> results = StreamsTestUtils.toList(iterator); | ||||
| 
 | ||||
|         assertEquals(asList(new KeyValue<>(0L, "my-value"), | ||||
|  | @ -87,7 +88,7 @@ public class CompositeReadOnlyWindowStoreTest { | |||
| 
 | ||||
|     @Test | ||||
|     public void shouldReturnEmptyIteratorIfNoData() { | ||||
|         final WindowStoreIterator<String> iterator = windowStore.fetch("my-key", 0L, 25L); | ||||
|         final WindowStoreIterator<String> iterator = windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L)); | ||||
|         assertEquals(false, iterator.hasNext()); | ||||
|     } | ||||
| 
 | ||||
|  | @ -100,10 +101,10 @@ public class CompositeReadOnlyWindowStoreTest { | |||
|         underlyingWindowStore.put("key-one", "value-one", 0L); | ||||
|         secondUnderlying.put("key-two", "value-two", 10L); | ||||
| 
 | ||||
|         final List<KeyValue<Long, String>> keyOneResults = StreamsTestUtils.toList(windowStore.fetch("key-one", 0L, | ||||
|                                                                                                      1L)); | ||||
|         final List<KeyValue<Long, String>> keyTwoResults = StreamsTestUtils.toList(windowStore.fetch("key-two", 10L, | ||||
|                                                                                                      11L)); | ||||
|         final List<KeyValue<Long, String>> keyOneResults = StreamsTestUtils.toList(windowStore.fetch("key-one", ofEpochMilli(0L), | ||||
|                                                                                                      ofEpochMilli(1L))); | ||||
|         final List<KeyValue<Long, String>> 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 class CompositeReadOnlyWindowStoreTest { | |||
|         otherUnderlyingStore.put("some-key", "some-value", 0L); | ||||
|         underlyingWindowStore.put("some-key", "my-value", 1L); | ||||
| 
 | ||||
|         final List<KeyValue<Long, String>> results = StreamsTestUtils.toList(windowStore.fetch("some-key", 0L, 2L)); | ||||
|         final List<KeyValue<Long, String>> 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<Object, Object> 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 class CompositeReadOnlyWindowStoreTest { | |||
|         final CompositeReadOnlyWindowStore<Object, Object> 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 class CompositeReadOnlyWindowStoreTest { | |||
|     public void emptyIteratorAlwaysReturnsFalse() { | ||||
|         final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new | ||||
|                 StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo"); | ||||
|         final WindowStoreIterator<Object> windowStoreIterator = store.fetch("key", 1, 10); | ||||
|         final WindowStoreIterator<Object> windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); | ||||
| 
 | ||||
|         Assert.assertFalse(windowStoreIterator.hasNext()); | ||||
|     } | ||||
|  | @ -151,7 +152,7 @@ public class CompositeReadOnlyWindowStoreTest { | |||
|     public void emptyIteratorPeekNextKeyShouldThrowNoSuchElementException() { | ||||
|         final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new | ||||
|                 StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo"); | ||||
|         final WindowStoreIterator<Object> windowStoreIterator = store.fetch("key", 1, 10); | ||||
|         final WindowStoreIterator<Object> windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); | ||||
| 
 | ||||
|         windowStoreIteratorException.expect(NoSuchElementException.class); | ||||
|         windowStoreIterator.peekNextKey(); | ||||
|  | @ -161,7 +162,7 @@ public class CompositeReadOnlyWindowStoreTest { | |||
|     public void emptyIteratorNextShouldThrowNoSuchElementException() { | ||||
|         final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new | ||||
|                 StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo"); | ||||
|         final WindowStoreIterator<Object> windowStoreIterator = store.fetch("key", 1, 10); | ||||
|         final WindowStoreIterator<Object> windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10)); | ||||
| 
 | ||||
|         windowStoreIteratorException.expect(NoSuchElementException.class); | ||||
|         windowStoreIterator.next(); | ||||
|  | @ -173,7 +174,7 @@ public class CompositeReadOnlyWindowStoreTest { | |||
|         stubProviderTwo.addStore(storeName, secondUnderlying); | ||||
|         underlyingWindowStore.put("a", "a", 0L); | ||||
|         secondUnderlying.put("b", "b", 10L); | ||||
|         final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.fetch("a", "b", 0, 10)); | ||||
|         final List<KeyValue<Windowed<String>, 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")))); | ||||
|  | @ -212,7 +213,7 @@ public class CompositeReadOnlyWindowStoreTest { | |||
|         stubProviderTwo.addStore(storeName, secondUnderlying); | ||||
|         underlyingWindowStore.put("a", "a", 0L); | ||||
|         secondUnderlying.put("b", "b", 10L); | ||||
|         final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.fetchAll(0, 10)); | ||||
|         final List<KeyValue<Windowed<String>, 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 class CompositeReadOnlyWindowStoreTest { | |||
| 
 | ||||
|     @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)); | ||||
|     } | ||||
| 
 | ||||
| } | ||||
|  |  | |||
|  | @ -40,6 +40,7 @@ import org.junit.Test; | |||
| 
 | ||||
| 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 class MeteredWindowStoreTest { | |||
|         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<MetricName, ? extends Metric> 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 class MeteredWindowStoreTest { | |||
|         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<MetricName, ? extends Metric> 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()); | ||||
|  |  | |||
|  | @ -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 class ReadOnlyWindowStoreStub<K, V> implements ReadOnlyWindowStore<K, V>, | |||
|         return new TheWindowStoreIterator<>(results.iterator()); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public WindowStoreIterator<V> 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<Windowed<K>, V> all() { | ||||
|         if (!open) { | ||||
|  | @ -164,6 +173,13 @@ public class ReadOnlyWindowStoreStub<K, V> implements ReadOnlyWindowStore<K, V>, | |||
|         }; | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public KeyValueIterator<Windowed<K>, 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<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { | ||||
|         if (!open) { | ||||
|  | @ -209,6 +225,15 @@ public class ReadOnlyWindowStoreStub<K, V> implements ReadOnlyWindowStore<K, V>, | |||
|         }; | ||||
|     } | ||||
| 
 | ||||
|     @Override public KeyValueIterator<Windowed<K>, 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<K, V>()); | ||||
|  |  | |||
|  | @ -54,6 +54,7 @@ import java.util.List; | |||
| 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 class RocksDBWindowStoreTest { | |||
|         setCurrentTime(currentTime); | ||||
|         windowStore.put(1, "three"); | ||||
| 
 | ||||
|         final WindowStoreIterator<String> iterator = windowStore.fetch(1, 0, currentTime); | ||||
|         final WindowStoreIterator<String> 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 class RocksDBWindowStoreTest { | |||
|         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 class RocksDBWindowStoreTest { | |||
|         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 class RocksDBWindowStoreTest { | |||
| 
 | ||||
|         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 class RocksDBWindowStoreTest { | |||
| 
 | ||||
|         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 class RocksDBWindowStoreTest { | |||
| 
 | ||||
|         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 class RocksDBWindowStoreTest { | |||
| 
 | ||||
|         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 class RocksDBWindowStoreTest { | |||
|         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 class RocksDBWindowStoreTest { | |||
|             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 class RocksDBWindowStoreTest { | |||
|         ); | ||||
| 
 | ||||
| 
 | ||||
|         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 class RocksDBWindowStoreTest { | |||
|             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 class RocksDBWindowStoreTest { | |||
|         ); | ||||
| 
 | ||||
| 
 | ||||
|         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 class RocksDBWindowStoreTest { | |||
|         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 class RocksDBWindowStoreTest { | |||
|         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 class RocksDBWindowStoreTest { | |||
|         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 class RocksDBWindowStoreTest { | |||
|         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 class RocksDBWindowStoreTest { | |||
| 
 | ||||
|         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 class RocksDBWindowStoreTest { | |||
|         windowStore.put(1, "two", 2L); | ||||
|         windowStore.put(1, "three", 3L); | ||||
| 
 | ||||
|         final WindowStoreIterator<String> iterator = windowStore.fetch(1, 1L, 3L); | ||||
|         final WindowStoreIterator<String> iterator = windowStore.fetch(1, ofEpochMilli(1L), ofEpochMilli(3L)); | ||||
|         assertTrue(iterator.hasNext()); | ||||
|         windowStore.close(); | ||||
| 
 | ||||
|  | @ -784,16 +785,17 @@ public class RocksDBWindowStoreTest { | |||
| 
 | ||||
| 
 | ||||
|         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<KeyValue<Windowed<String>, String>> list = StreamsTestUtils.toList(windowStore.fetch("a", "a", 0, Long.MAX_VALUE)); | ||||
|         List<KeyValue<Windowed<String>, 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 class RocksDBWindowStoreTest { | |||
|     @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 class RocksDBWindowStoreTest { | |||
|         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<Integer, String> store, | ||||
|  |  | |||
|  | @ -78,7 +78,7 @@ public class StreamThreadStateStoreProviderTest { | |||
|         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" | ||||
|  |  | |||
|  | @ -41,7 +41,6 @@ import java.time.Duration; | |||
| 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 class BrokerCompatibilityTest { | |||
|                 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"); | ||||
|  |  | |||
|  | @ -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 class EosTestClient extends SmokeTestUtil { | |||
|             @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 class EosTestClient extends SmokeTestUtil { | |||
|                 streams.start(); | ||||
|             } | ||||
|             if (uncaughtException) { | ||||
|                 streams.close(TimeUnit.SECONDS.toMillis(60), TimeUnit.SECONDS); | ||||
|                 streams.close(Duration.ofSeconds(60_000L)); | ||||
|                 streams = null; | ||||
|             } | ||||
|             sleep(1000); | ||||
|  |  | |||
|  | @ -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.ForeachAction; | |||
| import org.apache.kafka.streams.kstream.KStream; | ||||
| 
 | ||||
| import java.util.Properties; | ||||
| import java.util.concurrent.TimeUnit; | ||||
| 
 | ||||
| public class ShutdownDeadlockTest { | ||||
| 
 | ||||
|  | @ -65,7 +65,7 @@ public class ShutdownDeadlockTest { | |||
|         Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { | ||||
|             @Override | ||||
|             public void run() { | ||||
|                 streams.close(5, TimeUnit.SECONDS); | ||||
|                 streams.close(Duration.ofSeconds(5)); | ||||
|             } | ||||
|         })); | ||||
| 
 | ||||
|  |  | |||
|  | @ -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.Stores; | |||
| 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 class SmokeTestClient extends SmokeTestUtil { | |||
|     } | ||||
| 
 | ||||
|     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 class SmokeTestClient extends SmokeTestUtil { | |||
|             data.groupByKey(Serialized.with(stringSerde, intSerde)); | ||||
| 
 | ||||
|         groupedData | ||||
|             .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(1))) | ||||
|             .windowedBy(TimeWindows.of(Duration.ofDays(1))) | ||||
|             .aggregate( | ||||
|                 new Initializer<Integer>() { | ||||
|                     public Integer apply() { | ||||
|  | @ -154,7 +154,7 @@ public class SmokeTestClient extends SmokeTestUtil { | |||
| 
 | ||||
|         // max | ||||
|         groupedData | ||||
|             .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2))) | ||||
|             .windowedBy(TimeWindows.of(Duration.ofDays(2))) | ||||
|             .aggregate( | ||||
|                 new Initializer<Integer>() { | ||||
|                     public Integer apply() { | ||||
|  | @ -179,7 +179,7 @@ public class SmokeTestClient extends SmokeTestUtil { | |||
| 
 | ||||
|         // sum | ||||
|         groupedData | ||||
|             .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2))) | ||||
|             .windowedBy(TimeWindows.of(Duration.ofDays(2))) | ||||
|             .aggregate( | ||||
|                 new Initializer<Long>() { | ||||
|                     public Long apply() { | ||||
|  | @ -202,7 +202,7 @@ public class SmokeTestClient extends SmokeTestUtil { | |||
| 
 | ||||
|         // cnt | ||||
|         groupedData | ||||
|             .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2))) | ||||
|             .windowedBy(TimeWindows.of(Duration.ofDays(2))) | ||||
|             .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("uwin-cnt")) | ||||
|             .toStream(new Unwindow<String, Long>()) | ||||
|             .to("cnt", Produced.with(stringSerde, longSerde)); | ||||
|  | @ -252,7 +252,7 @@ public class SmokeTestClient extends SmokeTestUtil { | |||
|             @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)); | ||||
|             } | ||||
|         }); | ||||
| 
 | ||||
|  |  | |||
|  | @ -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.Collections; | |||
| 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 class StreamsBrokerDownResilienceTest { | |||
|             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 class StreamsBrokerDownResilienceTest { | |||
|         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(); | ||||
|             } | ||||
|  |  | |||
|  | @ -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.io.IOException; | |||
| 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 class StreamsStandByReplicaTest { | |||
|     } | ||||
| 
 | ||||
|     private static void shutdown(final KafkaStreams streams) { | ||||
|         streams.close(10, TimeUnit.SECONDS); | ||||
|         streams.close(Duration.ofSeconds(10)); | ||||
|     } | ||||
| 
 | ||||
|     private static boolean confirmCorrectConfigs(final Properties properties) { | ||||
|  |  | |||
|  | @ -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 class InternalMockProcessorContext extends AbstractProcessorContext imple | |||
|         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() { } | ||||
| 
 | ||||
|  |  | |||
|  | @ -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 class MockProcessor<K, V> extends AbstractProcessor<K, V> { | |||
|     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) { | ||||
|  |  | |||
|  | @ -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 class NoOpProcessorContext extends AbstractProcessorContext { | |||
|         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; | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -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) | ||||
| 
 | ||||
|  |  | |||
|  | @ -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) | ||||
| 
 | ||||
|  |  | |||
|  | @ -150,7 +150,7 @@ import java.util.regex.Pattern; | |||
|  * {@link ProducerRecord#equals(Object)} can simplify your code as you can ignore attributes you are not interested in. | ||||
|  * <p> | ||||
|  * 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)}. | ||||
|  * <p> | ||||
|  | @ -489,7 +489,7 @@ public class TopologyTestDriver implements Closeable { | |||
|     /** | ||||
|      * 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 | ||||
|      */ | ||||
|  |  | |||
|  | @ -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 class MockProcessorContext implements ProcessorContext, RecordCollector.S | |||
|     } | ||||
| 
 | ||||
|     @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 class MockProcessorContext implements ProcessorContext, RecordCollector.S | |||
|         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(...)}. | ||||
|      * | ||||
|  |  | |||
|  | @ -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 class MockProcessorContextTest { | |||
|             @Override | ||||
|             public void init(final ProcessorContext context) { | ||||
|                 context.schedule( | ||||
|                     1000L, | ||||
|                     Duration.ofSeconds(1L), | ||||
|                     PunctuationType.WALL_CLOCK_TIME, | ||||
|                     timestamp -> context.commit() | ||||
|                 ); | ||||
|  |  | |||
|  | @ -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 class TopologyTestDriverTest { | |||
|             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 @@ public class TopologyTestDriverTest { | |||
|         @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<String, Long>) context.getStateStore("aggStore"); | ||||
|         } | ||||
| 
 | ||||
|  |  | |||
		Loading…
	
		Reference in New Issue