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:
Nikolay 2018-10-04 23:51:39 +03:00 committed by Matthias J. Sax
parent 260b07a6da
commit ca641b3e2e
100 changed files with 1310 additions and 535 deletions

View File

@ -19,6 +19,7 @@ package org.apache.kafka.streams.examples.pageview;
import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import java.time.Duration;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Deserializer;
@ -39,7 +40,6 @@ import java.io.IOException;
import java.util.Map; import java.util.Map;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.CountDownLatch; 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, * 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)) .map((user, viewRegion) -> new KeyValue<>(viewRegion.region, viewRegion))
.groupByKey(Serialized.with(Serdes.String(), new JSONSerde<>())) .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() .count()
.toStream() .toStream()
.map((key, value) -> { .map((key, value) -> {

View File

@ -19,6 +19,7 @@ package org.apache.kafka.streams.examples.pageview;
import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.JsonNodeFactory; import com.fasterxml.jackson.databind.node.JsonNodeFactory;
import com.fasterxml.jackson.databind.node.ObjectNode; import com.fasterxml.jackson.databind.node.ObjectNode;
import java.time.Duration;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serde;
@ -87,7 +88,7 @@ public class PageViewUntypedDemo {
}) })
.map((user, viewRegion) -> new KeyValue<>(viewRegion.get("region").textValue(), viewRegion)) .map((user, viewRegion) -> new KeyValue<>(viewRegion.get("region").textValue(), viewRegion))
.groupByKey(Serialized.with(Serdes.String(), jsonSerde)) .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() .count()
.toStream() .toStream()
.map((key, value) -> { .map((key, value) -> {

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.examples.temperature; package org.apache.kafka.streams.examples.temperature;
import java.time.Duration;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
@ -30,7 +31,6 @@ import org.apache.kafka.streams.kstream.WindowedSerdes;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
/** /**
* Demonstrates, using the high-level KStream DSL, how to implement an IoT demo application * 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) // to group and reduce them, a key is needed ("temp" has been chosen)
.selectKey((key, value) -> "temp") .selectKey((key, value) -> "temp")
.groupByKey() .groupByKey()
.windowedBy(TimeWindows.of(TimeUnit.SECONDS.toMillis(TEMPERATURE_WINDOW_SIZE))) .windowedBy(TimeWindows.of(Duration.ofSeconds(TEMPERATURE_WINDOW_SIZE)))
.reduce((value1, value2) -> { .reduce((value1, value2) -> {
if (Integer.parseInt(value1) > Integer.parseInt(value2)) if (Integer.parseInt(value1) > Integer.parseInt(value2))
return value1; return value1;

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.examples.wordcount; package org.apache.kafka.streams.examples.wordcount;
import java.time.Duration;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KafkaStreams;
@ -63,7 +64,7 @@ public final class WordCountProcessorDemo {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public void init(final ProcessorContext context) { public void init(final ProcessorContext context) {
this.context = 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()) { try (final KeyValueIterator<String, Integer> iter = kvStore.all()) {
System.out.println("----------- " + timestamp + " ----------- "); System.out.println("----------- " + timestamp + " ----------- ");

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams; package org.apache.kafka.streams;
import java.time.Duration;
import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClient;
import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.KafkaProducer; 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.InvalidStateStoreException;
import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.ProcessorStateException;
import org.apache.kafka.streams.errors.StreamsException; 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.KStream;
import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KTable;
import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.kstream.Produced;
@ -827,7 +829,9 @@ public class KafkaStreams {
* @return {@code true} if all threads were successfully stopped&mdash;{@code false} if the timeout was reached * @return {@code true} if all threads were successfully stopped&mdash;{@code false} if the timeout was reached
* before all threads stopped * before all threads stopped
* Note that this method must not be called in the {@code onChange} callback of {@link StateListener}. * 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) { public synchronized boolean close(final long timeout, final TimeUnit timeUnit) {
log.debug("Stopping Streams client with timeoutMillis = {} ms.", timeUnit.toMillis(timeout)); 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&mdash;{@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 * 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}. * data with regard to the {@link StreamsConfig#APPLICATION_ID_CONFIG application ID}.

View File

@ -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);
}
}
}

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.kstream; 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.processor.TimestampExtractor;
import java.time.Duration; import java.time.Duration;
@ -110,42 +111,93 @@ public final class JoinWindows extends Windows<Window> {
* *
* @param timeDifferenceMs join window interval in milliseconds * @param timeDifferenceMs join window interval in milliseconds
* @throws IllegalArgumentException if {@code timeDifferenceMs} is negative * @throws IllegalArgumentException if {@code timeDifferenceMs} is negative
* @deprecated Use {@link #of(Duration)} instead.
*/ */
@Deprecated
public static JoinWindows of(final long timeDifferenceMs) throws IllegalArgumentException { public static JoinWindows of(final long timeDifferenceMs) throws IllegalArgumentException {
// This is a static factory method, so we initialize grace and retention to the defaults. // This is a static factory method, so we initialize grace and retention to the defaults.
return new JoinWindows(timeDifferenceMs, timeDifferenceMs, null, DEFAULT_RETENTION_MS); 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. * 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 * 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} 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). * value (which would result in a negative window size).
* *
* @param timeDifferenceMs relative window start time in milliseconds * @param timeDifferenceMs relative window start time in milliseconds
* @throws IllegalArgumentException if the resulting window size is negative * @throws IllegalArgumentException if the resulting window size is negative
* @deprecated Use {@link #before(Duration)} instead.
*/ */
@SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this @SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this
@Deprecated
public JoinWindows before(final long timeDifferenceMs) throws IllegalArgumentException { public JoinWindows before(final long timeDifferenceMs) throws IllegalArgumentException {
return new JoinWindows(timeDifferenceMs, afterMs, grace, maintainDurationMs, segments); 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. * 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 * 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} 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). * value (which would result in a negative window size).
* *
* @param timeDifferenceMs relative window end time in milliseconds * @param timeDifferenceMs relative window end time in milliseconds
* @throws IllegalArgumentException if the resulting window size is negative * @throws IllegalArgumentException if the resulting window size is negative
* @deprecated Use {@link #after(Duration)} instead
*/ */
@SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this @SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this
@Deprecated
public JoinWindows after(final long timeDifferenceMs) throws IllegalArgumentException { public JoinWindows after(final long timeDifferenceMs) throws IllegalArgumentException {
return new JoinWindows(beforeMs, timeDifferenceMs, grace, maintainDurationMs, segments); 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}. * Not supported by {@code JoinWindows}.
* Throws {@link UnsupportedOperationException}. * 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. * after the end of its window.
* *
* Lateness is defined as (stream_time - record_timestamp). * 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 * @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 @SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this
public JoinWindows grace(final long millisAfterWindowEnd) { public JoinWindows grace(final Duration afterWindowEnd) throws IllegalArgumentException {
if (millisAfterWindowEnd < 0) { ApiUtils.validateMillisecondDuration(afterWindowEnd, "afterWindowEnd");
if (afterWindowEnd.toMillis() < 0) {
throw new IllegalArgumentException("Grace period must not be negative."); 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 @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 * @param durationMs the window retention time in milliseconds
* @return itself * @return itself
* @throws IllegalArgumentException if {@code durationMs} is smaller than the window size * @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") @SuppressWarnings("deprecation")
@Override @Override

View File

@ -19,6 +19,7 @@ package org.apache.kafka.streams.kstream;
import org.apache.kafka.common.internals.Topic; import org.apache.kafka.common.internals.Topic;
import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.utils.Bytes; 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.processor.StateStore;
import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier;
import org.apache.kafka.streams.state.KeyValueStore; 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. * from window-start through window-end, and for the entire grace period.
* *
* @return itself * @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) { public Materialized<K, V, S> withRetention(final Duration retention) throws IllegalArgumentException {
if (retentionMs < 0) { ApiUtils.validateMillisecondDuration(retention, "retention");
if (retention.toMillis() < 0) {
throw new IllegalArgumentException("Retention must not be negative."); throw new IllegalArgumentException("Retention must not be negative.");
} }
retention = Duration.ofMillis(retentionMs); this.retention = retention;
return this; return this;
} }
} }

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.kstream; 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.processor.TimestampExtractor;
import org.apache.kafka.streams.state.SessionBytesStoreSupplier; 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 * @return a new window specification with default maintain duration of 1 day
* *
* @throws IllegalArgumentException if {@code inactivityGapMs} is zero or negative * @throws IllegalArgumentException if {@code inactivityGapMs} is zero or negative
* @deprecated User {@link #with(Duration)} instead.
*/ */
@Deprecated
public static SessionWindows with(final long inactivityGapMs) { public static SessionWindows with(final long inactivityGapMs) {
if (inactivityGapMs <= 0) { if (inactivityGapMs <= 0) {
throw new IllegalArgumentException("Gap time (inactivityGapMs) cannot be zero or negative."); 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); 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. * 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. * 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 * 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. * 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 * @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) { public SessionWindows grace(final Duration afterWindowEnd) throws IllegalArgumentException {
if (millisAfterWindowEnd < 0) { ApiUtils.validateMillisecondDuration(afterWindowEnd, "afterWindowEnd");
if (afterWindowEnd.toMillis() < 0) {
throw new IllegalArgumentException("Grace period must not be negative."); throw new IllegalArgumentException("Grace period must not be negative.");
} }
return new SessionWindows( return new SessionWindows(
gapMs, gapMs,
maintainDurationMs, maintainDurationMs,
Duration.ofMillis(millisAfterWindowEnd) afterWindowEnd
); );
} }

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.kstream; 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.kstream.internals.TimeWindow;
import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.TimestampExtractor;
import org.apache.kafka.streams.state.WindowBytesStoreSupplier; 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 * @param sizeMs The size of the window in milliseconds
* @return a new window definition with default maintain duration of 1 day * @return a new window definition with default maintain duration of 1 day
* @throws IllegalArgumentException if the specified window size is zero or negative * @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 { public static TimeWindows of(final long sizeMs) throws IllegalArgumentException {
if (sizeMs <= 0) { if (sizeMs <= 0) {
throw new IllegalArgumentException("Window size (sizeMs) must be larger than zero."); 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 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 * 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. * 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> * <p>
* This provides the semantics of hopping windows, which are fixed-sized, overlapping windows. * 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 * @param advanceMs The advance interval ("hop") in milliseconds of the window, with the requirement that {@code 0 < advanceMs <= sizeMs}.
* {@code 0 < advanceMs &le; sizeMs}.
* @return a new window definition with default maintain duration of 1 day * @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 * @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 @SuppressWarnings("deprecation") // will be fixed when we remove segments from Windows
@Deprecated
public TimeWindows advanceBy(final long advanceMs) { public TimeWindows advanceBy(final long advanceMs) {
if (advanceMs <= 0 || advanceMs > sizeMs) { if (advanceMs <= 0 || advanceMs > sizeMs) {
throw new IllegalArgumentException(String.format("AdvanceMs must lie within interval (0, %d].", 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 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 @Override
public Map<Long, TimeWindow> windowsFor(final long timestamp) { public Map<Long, TimeWindow> windowsFor(final long timestamp) {
long windowStart = (Math.max(0, timestamp - sizeMs + advanceMs) / advanceMs) * advanceMs; 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). * 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 * @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 @SuppressWarnings("deprecation") // will be fixed when we remove segments from Windows
public TimeWindows grace(final long millisAfterWindowEnd) { public TimeWindows grace(final Duration afterWindowEnd) throws IllegalArgumentException {
if (millisAfterWindowEnd < 0) { ApiUtils.validateMillisecondDuration(afterWindowEnd, "afterWindowEnd");
if (afterWindowEnd.toMillis() < 0) {
throw new IllegalArgumentException("Grace period must not be negative."); 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 @SuppressWarnings("deprecation") // continuing to support Windows#maintainMs/segmentInterval in fallback mode

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.kstream; package org.apache.kafka.streams.kstream;
import java.time.Duration;
import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.PunctuationType; 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 * 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 * 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). * {@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. * a method to be {@link Punctuator#punctuate(long) called periodically} with the provided context.
* <p> * <p>
* Use {@link TransformerSupplier} to provide new instances of {@code Transformer} to Kafka Stream's runtime. * 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. * framework may later re-use the transformer by calling {@link #init(ProcessorContext)} again.
* <p> * <p>
* The provided {@link ProcessorContext context} can be used to access topology and record meta data, to * 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. * {@link Punctuator#punctuate(long) called periodically} and to access attached {@link StateStore}s.
* <p> * <p>
* Note, that {@link ProcessorContext} is updated in the background with the current record's meta data. * Note, that {@link ProcessorContext} is updated in the background with the current record's meta data.

View File

@ -16,6 +16,8 @@
*/ */
package org.apache.kafka.streams.kstream; 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.kstream.internals.UnlimitedWindow;
import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.TimestampExtractor;
@ -62,7 +64,9 @@ public final class UnlimitedWindows extends Windows<UnlimitedWindow> {
* @param startMs the window start time * @param startMs the window start time
* @return a new unlimited window that starts at {@code startMs} * @return a new unlimited window that starts at {@code startMs}
* @throws IllegalArgumentException if the start time is negative * @throws IllegalArgumentException if the start time is negative
* @deprecated Use {@link #startOn(Instant)} instead
*/ */
@Deprecated
public UnlimitedWindows startOn(final long startMs) throws IllegalArgumentException { public UnlimitedWindows startOn(final long startMs) throws IllegalArgumentException {
if (startMs < 0) { if (startMs < 0) {
throw new IllegalArgumentException("Window start time (startMs) cannot be negative."); 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 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 @Override
public Map<Long, UnlimitedWindow> windowsFor(final long timestamp) { public Map<Long, UnlimitedWindow> windowsFor(final long timestamp) {
// always return the single unlimited window // always return the single unlimited window

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.kstream; package org.apache.kafka.streams.kstream;
import java.time.Duration;
import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.ProcessorContext; 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 * 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 * 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). * {@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. * 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. * If {@code ValueTransformer} is applied to a {@link KeyValue} pair record the record's key is preserved.
* <p> * <p>
@ -54,7 +55,7 @@ public interface ValueTransformer<V, VR> {
* framework may later re-use the transformer by calling {@link #init(ProcessorContext)} again. * framework may later re-use the transformer by calling {@link #init(ProcessorContext)} again.
* <p> * <p>
* The provided {@link ProcessorContext context} can be used to access topology and record meta data, to * 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. * {@link Punctuator#punctuate(long) called periodically} and to access attached {@link StateStore}s.
* <p> * <p>
* Note that {@link ProcessorContext} is updated in the background with the current record's meta data. * Note that {@link ProcessorContext} is updated in the background with the current record's meta data.

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.kstream; package org.apache.kafka.streams.kstream;
import java.time.Duration;
import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.ProcessorContext; 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 * 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). * {@link #transform(Object, Object)} (cf. {@link ValueMapper} for stateless value transformation).
* Additionally, this {@code ValueTransformerWithKey} can * 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. * {@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. * 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. * 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. * This is called once per instance when the topology gets initialized.
* <p> * <p>
* The provided {@link ProcessorContext context} can be used to access topology and record meta data, to * 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. * {@link Punctuator#punctuate(long) called periodically} and to access attached {@link StateStore}s.
* <p> * <p>
* Note that {@link ProcessorContext} is updated in the background with the current record's meta data. * Note that {@link ProcessorContext} is updated in the background with the current record's meta data.

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.kstream; package org.apache.kafka.streams.kstream;
import java.time.Instant;
import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.TimestampExtractor;
/** /**
@ -36,6 +37,9 @@ public abstract class Window {
protected final long startMs; protected final long startMs;
protected final long endMs; protected final long endMs;
private final Instant startTime;
private final Instant endTime;
/** /**
* Create a new window for the given start and end time. * Create a new window for the given start and end time.
@ -53,6 +57,9 @@ public abstract class Window {
} }
this.startMs = startMs; this.startMs = startMs;
this.endMs = endMs; this.endMs = endMs;
this.startTime = Instant.ofEpochMilli(startMs);
this.endTime = Instant.ofEpochMilli(endMs);
} }
/** /**
@ -69,6 +76,24 @@ public abstract class Window {
return endMs; 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. * 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 * Should throw an {@link IllegalArgumentException} if the {@code other} window has a different type than {@code

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.kstream.internals; package org.apache.kafka.streams.kstream.internals;
import java.time.Duration;
import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.kstream.ForeachAction; 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( return Stores.windowStoreBuilder(
Stores.persistentWindowStore( Stores.persistentWindowStore(
joinName + "-store", joinName + "-store",
windows.size() + windows.gracePeriodMs(), Duration.ofMillis(windows.size() + windows.gracePeriodMs()),
windows.size(), Duration.ofMillis(windows.size()),
true true
), ),
keySerde, keySerde,

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.kstream.internals; package org.apache.kafka.streams.kstream.internals;
import java.time.Duration;
import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Bytes;
@ -170,8 +171,8 @@ public class TimeWindowedKStreamImpl<K, V, W extends Window> extends AbstractStr
supplier = Stores.persistentWindowStore( supplier = Stores.persistentWindowStore(
materialized.storeName(), materialized.storeName(),
retentionPeriod, Duration.ofMillis(retentionPeriod),
windows.size(), Duration.ofMillis(windows.size()),
false false
); );

View File

@ -16,8 +16,10 @@
*/ */
package org.apache.kafka.streams.processor; 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 * @see Punctuator
*/ */

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.processor; package org.apache.kafka.streams.processor;
import java.time.Duration;
import org.apache.kafka.common.annotation.InterfaceStability; 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. * framework may later re-use the processor by calling {@link #init()} again.
* <p> * <p>
* The provided {@link ProcessorContext context} can be used to access topology and record meta data, to * 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. * {@link Punctuator#punctuate(long) called periodically} and to access attached {@link StateStore}s.
* *
* @param context the context; may not be null * @param context the context; may not be null

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.processor; package org.apache.kafka.streams.processor;
import java.time.Duration;
import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.serialization.Serde; 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 * Schedules a periodic operation for processors. A processor may call this method during
* {@link Processor#init(ProcessorContext) initialization} or * {@link Processor#init(ProcessorContext) initialization} or
* {@link Processor#process(Object, Object) processing} to * {@link Processor#process(Object, Object) processing} to
* schedule a periodic callback - called a punctuation - to {@link Punctuator#punctuate(long)}. * schedule a periodic callback &mdash; called a punctuation &mdash; to {@link Punctuator#punctuate(long)}.
* The type parameter controls what notion of time is used for punctuation: * The type parameter controls what notion of time is used for punctuation:
* <ul> * <ul>
* <li>{@link PunctuationType#STREAM_TIME} - uses "stream time", which is advanced by the processing of messages * <li>{@link PunctuationType#STREAM_TIME} &mdash; uses "stream time", which is advanced by the processing of messages
* in accordance with the timestamp as extracted by the {@link TimestampExtractor} in use. * 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. * The first punctuation will be triggered by the first record that is processed.
* <b>NOTE:</b> Only advanced if messages arrive</li> * <b>NOTE:</b> Only advanced if messages arrive</li>
* <li>{@link PunctuationType#WALL_CLOCK_TIME} - uses system time (the wall-clock time), * <li>{@link PunctuationType#WALL_CLOCK_TIME} &mdash; uses system time (the wall-clock time),
* which is advanced at the polling interval ({@link org.apache.kafka.streams.StreamsConfig#POLL_MS_CONFIG}) * which is advanced independent of whether new messages arrive.
* independent of whether new messages arrive.
* The first punctuation will be triggered after interval has elapsed. * 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 * <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> * 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 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 * @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 * @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, Cancellable schedule(final long intervalMs,
final PunctuationType type, final PunctuationType type,
final Punctuator callback); 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 &mdash; called a punctuation &mdash; to {@link Punctuator#punctuate(long)}.
* The type parameter controls what notion of time is used for punctuation:
* <ul>
* <li>{@link PunctuationType#STREAM_TIME} &mdash; 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} &mdash; 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. * Forwards a key/value pair to all downstream processors.
* Used the input record's timestamp as timestamp for the output record. * Used the input record's timestamp as timestamp for the output record.

View File

@ -16,8 +16,10 @@
*/ */
package org.apache.kafka.streams.processor; 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> * <ul>
* <li>STREAM_TIME - uses "stream time", which is advanced by the processing of messages * <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. * in accordance with the timestamp as extracted by the {@link TimestampExtractor} in use.

View File

@ -16,8 +16,10 @@
*/ */
package org.apache.kafka.streams.processor; 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 * @see Cancellable
*/ */

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.processor.internals; package org.apache.kafka.streams.processor.internals;
import java.time.Duration;
import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.streams.StreamsMetrics; import org.apache.kafka.streams.StreamsMetrics;
@ -85,12 +86,20 @@ public final class ForwardingDisabledProcessorContext implements ProcessorContex
} }
@Override @Override
@Deprecated
public Cancellable schedule(final long intervalMs, public Cancellable schedule(final long intervalMs,
final PunctuationType type, final PunctuationType type,
final Punctuator callback) { final Punctuator callback) {
return delegate.schedule(intervalMs, type, 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 @Override
public <K, V> void forward(final K key, final V value) { public <K, V> void forward(final K key, final V value) {
throw new StreamsException("ProcessorContext#forward() not supported."); throw new StreamsException("ProcessorContext#forward() not supported.");

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.processor.internals; package org.apache.kafka.streams.processor.internals;
import java.time.Duration;
import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.processor.Cancellable; import org.apache.kafka.streams.processor.Cancellable;
import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.PunctuationType;
@ -92,10 +93,19 @@ public class GlobalProcessorContextImpl extends AbstractProcessorContext {
* @throws UnsupportedOperationException on every invocation * @throws UnsupportedOperationException on every invocation
*/ */
@Override @Override
@Deprecated
public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) { 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."); 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 @Override
public long streamTime() { public long streamTime() {
throw new RuntimeException("Stream time is not implemented for the global processor context."); throw new RuntimeException("Stream time is not implemented for the global processor context.");

View File

@ -16,6 +16,8 @@
*/ */
package org.apache.kafka.streams.processor.internals; 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.StreamsConfig;
import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.Cancellable; import org.apache.kafka.streams.processor.Cancellable;
@ -150,10 +152,19 @@ public class ProcessorContextImpl extends AbstractProcessorContext implements Re
} }
@Override @Override
@Deprecated
public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) { public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) {
return task.schedule(interval, type, 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) { void setStreamTimeSupplier(final TimestampSupplier streamTimeSupplier) {
this.streamTimeSupplier = streamTimeSupplier; this.streamTimeSupplier = streamTimeSupplier;
} }

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.processor.internals; package org.apache.kafka.streams.processor.internals;
import java.time.Duration;
import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.Headers;
@ -189,10 +190,19 @@ class StandbyContextImpl extends AbstractProcessorContext implements RecordColle
* @throws UnsupportedOperationException on every invocation * @throws UnsupportedOperationException on every invocation
*/ */
@Override @Override
@SuppressWarnings("deprecation")
public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) { 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."); 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 * @throws UnsupportedOperationException on every invocation
*/ */

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.state; package org.apache.kafka.streams.state;
import java.time.Instant;
import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.errors.InvalidStateStoreException;
import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.Windowed;
@ -40,9 +41,8 @@ public interface ReadOnlyWindowStore<K, V> {
V fetch(K key, long time); V fetch(K key, long time);
/** /**
* Get all the key-value pairs with the given key and the time range from all * Get all the key-value pairs with the given key and the time range from all the existing windows.
* the existing windows. * <p>
*
* This iterator must be closed after use. * This iterator must be closed after use.
* <p> * <p>
* The time range is inclusive and applies to the starting timestamp of the window. * 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>} * @return an iterator over key-value pairs {@code <timestamp, value>}
* @throws InvalidStateStoreException if the store is not initialized * @throws InvalidStateStoreException if the store is not initialized
* @throws NullPointerException If {@code null} is used for key. * @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); 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 * Get all the key-value pairs with the given key and the time range from all the existing windows.
* 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. * This iterator must be closed after use.
* *
* @param from the first key in the range * @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>} * @return an iterator over windowed key-value pairs {@code <Windowed<K>, value>}
* @throws InvalidStateStoreException if the store is not initialized * @throws InvalidStateStoreException if the store is not initialized
* @throws NullPointerException If {@code null} is used for any key. * @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); 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. * 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>} * @return an iterator over windowed key-value pairs {@code <Windowed<K>, value>}
* @throws InvalidStateStoreException if the store is not initialized * @throws InvalidStateStoreException if the store is not initialized
* @throws NullPointerException if {@code null} is used for any key * @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); 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;
} }

View File

@ -16,11 +16,13 @@
*/ */
package org.apache.kafka.streams.state; package org.apache.kafka.streams.state;
import java.time.Duration;
import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.common.utils.Time; 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.InMemoryKeyValueStore;
import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
import org.apache.kafka.streams.state.internals.MemoryNavigableLRUCache; 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 windowSize size of the windows (cannot be negative)
* @param retainDuplicates whether or not to retain duplicates. * @param retainDuplicates whether or not to retain duplicates.
* @return an instance of {@link WindowBytesStoreSupplier} * @return an instance of {@link WindowBytesStoreSupplier}
* @deprecated Use {@link #persistentWindowStore(String, Duration, Duration, boolean)} instead
*/ */
@Deprecated
public static WindowBytesStoreSupplier persistentWindowStore(final String name, public static WindowBytesStoreSupplier persistentWindowStore(final String name,
final long retentionPeriod, final long retentionPeriod,
final long windowSize, final long windowSize,
@ -196,6 +200,29 @@ public class Stores {
return persistentWindowStore(name, retentionPeriod, windowSize, retainDuplicates, defaultSegmentInterval); 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}. * Create a persistent {@link WindowBytesStoreSupplier}.
* @param name name of the store (cannot be {@code null}) * @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 windowSize size of the windows (cannot be negative)
* @param retainDuplicates whether or not to retain duplicates. * @param retainDuplicates whether or not to retain duplicates.
* @return an instance of {@link WindowBytesStoreSupplier} * @return an instance of {@link WindowBytesStoreSupplier}
* @deprecated Use {@link #persistentWindowStore(String, Duration, Duration, boolean)} instead
*/ */
@Deprecated
public static WindowBytesStoreSupplier persistentWindowStore(final String name, public static WindowBytesStoreSupplier persistentWindowStore(final String name,
final long retentionPeriod, final long retentionPeriod,
final long windowSize, final long windowSize,
@ -250,6 +279,21 @@ public class Stores {
return new RocksDbSessionBytesStoreSupplier(name, retentionPeriod); 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}. * Creates a {@link StoreBuilder} that can be used to build a {@link WindowStore}.

View File

@ -16,6 +16,8 @@
*/ */
package org.apache.kafka.streams.state; 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; 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. * @throws NullPointerException If null is used for key.
*/ */
void put(K key, V value, long windowStartTimestamp); 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);
} }

View File

@ -16,12 +16,14 @@
*/ */
package org.apache.kafka.streams.state; package org.apache.kafka.streams.state;
import java.time.Instant;
import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.KeyValue;
import java.io.Closeable; 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, * 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. * or use try-with-resources statement (available since JDK7) for this {@link Closeable} class.

View File

@ -16,9 +16,11 @@
*/ */
package org.apache.kafka.streams.state.internals; package org.apache.kafka.streams.state.internals;
import java.time.Instant;
import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.Bytes; 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.Windowed;
import org.apache.kafka.streams.kstream.internals.CacheFlushListener; import org.apache.kafka.streams.kstream.internals.CacheFlushListener;
import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorContext;
@ -203,6 +205,13 @@ class CachingWindowStore<K, V> extends WrappedStateStore.AbstractStateStore impl
return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator); 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 @Override
public KeyValueIterator<Windowed<Bytes>, byte[]> fetch(final Bytes from, final Bytes to, final long timeFrom, final long timeTo) { 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 // 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) { private V fetchPrevious(final Bytes key, final long timestamp) {
final byte[] value = underlying.fetch(key, timestamp); final byte[] value = underlying.fetch(key, timestamp);
if (value != null) { if (value != null) {
@ -275,4 +294,11 @@ class CachingWindowStore<K, V> extends WrappedStateStore.AbstractStateStore impl
cacheFunction 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());
}
} }

View File

@ -16,8 +16,10 @@
*/ */
package org.apache.kafka.streams.state.internals; package org.apache.kafka.streams.state.internals;
import java.time.Instant;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.streams.internals.ApiUtils;
import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStore;
@ -56,11 +58,28 @@ class ChangeLoggingWindowBytesStore extends WrappedStateStore.AbstractStateStore
return bytesStore.fetch(key, from, to); 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 @Override
public KeyValueIterator<Windowed<Bytes>, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) { 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); 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 @Override
public KeyValueIterator<Windowed<Bytes>, byte[]> all() { public KeyValueIterator<Windowed<Bytes>, byte[]> all() {
return bytesStore.all(); return bytesStore.all();
@ -71,6 +90,13 @@ class ChangeLoggingWindowBytesStore extends WrappedStateStore.AbstractStateStore
return bytesStore.fetchAll(timeFrom, timeTo); 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 @Override
public void put(final Bytes key, final byte[] value) { public void put(final Bytes key, final byte[] value) {
put(key, value, context.timestamp()); put(key, value, context.timestamp());

View File

@ -16,6 +16,8 @@
*/ */
package org.apache.kafka.streams.state.internals; 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.errors.InvalidStateStoreException;
import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueIterator;
@ -64,6 +66,7 @@ public class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K
} }
@Override @Override
@Deprecated
public WindowStoreIterator<V> fetch(final K key, final long timeFrom, final long timeTo) { public WindowStoreIterator<V> fetch(final K key, final long timeFrom, final long timeTo) {
Objects.requireNonNull(key, "key can't be null"); Objects.requireNonNull(key, "key can't be null");
final List<ReadOnlyWindowStore<K, V>> stores = provider.stores(storeName, windowStoreType); 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(); 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 @Override
public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { 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"); Objects.requireNonNull(from, "from can't be null");
@ -100,6 +110,13 @@ public class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K
nextIteratorFunction)); 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 @Override
public KeyValueIterator<Windowed<K>, V> all() { public KeyValueIterator<Windowed<K>, V> all() {
final NextIteratorFunction<Windowed<K>, V, ReadOnlyWindowStore<K, V>> nextIteratorFunction = new NextIteratorFunction<Windowed<K>, V, ReadOnlyWindowStore<K, V>>() { 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 @Override
@Deprecated
public KeyValueIterator<Windowed<K>, V> fetchAll(final long timeFrom, final long timeTo) { 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>>() { final NextIteratorFunction<Windowed<K>, V, ReadOnlyWindowStore<K, V>> nextIteratorFunction = new NextIteratorFunction<Windowed<K>, V, ReadOnlyWindowStore<K, V>>() {
@Override @Override
@ -127,4 +145,11 @@ public class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K
provider.stores(storeName, windowStoreType).iterator(), provider.stores(storeName, windowStoreType).iterator(),
nextIteratorFunction)); 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());
}
} }

View File

@ -16,10 +16,12 @@
*/ */
package org.apache.kafka.streams.state.internals; package org.apache.kafka.streams.state.internals;
import java.time.Instant;
import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.common.utils.Time; 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.errors.ProcessorStateException;
import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorContext;
@ -147,6 +149,13 @@ public class MeteredWindowStore<K, V> extends WrappedStateStore.AbstractStateSto
time); 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 @Override
public KeyValueIterator<Windowed<K>, V> all() { public KeyValueIterator<Windowed<K>, V> all() {
return new MeteredWindowedKeyValueIterator<>(inner.all(), fetchTime, metrics, serdes, time); return new MeteredWindowedKeyValueIterator<>(inner.all(), fetchTime, metrics, serdes, time);
@ -161,6 +170,13 @@ public class MeteredWindowStore<K, V> extends WrappedStateStore.AbstractStateSto
time); 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 @Override
public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { 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), return new MeteredWindowedKeyValueIterator<>(inner.fetch(keyBytes(from), keyBytes(to), timeFrom, timeTo),
@ -170,6 +186,13 @@ public class MeteredWindowStore<K, V> extends WrappedStateStore.AbstractStateSto
time); 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 @Override
public void flush() { public void flush() {
final long startNs = time.nanoseconds(); final long startNs = time.nanoseconds();

View File

@ -16,8 +16,10 @@
*/ */
package org.apache.kafka.streams.state.internals; package org.apache.kafka.streams.state.internals;
import java.time.Instant;
import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.utils.Bytes; 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.Windowed;
import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStore; 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(); 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 @Override
public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { 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); 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(); 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 @Override
public KeyValueIterator<Windowed<K>, V> all() { public KeyValueIterator<Windowed<K>, V> all() {
final KeyValueIterator<Bytes, byte[]> bytesIterator = bytesStore.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(); 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() { private void maybeUpdateSeqnumForDups() {
if (retainDuplicates) { if (retainDuplicates) {
seqnum = (seqnum + 1) & 0x7FFFFFFF; seqnum = (seqnum + 1) & 0x7FFFFFFF;

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams; package org.apache.kafka.streams;
import java.time.Duration;
import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.clients.producer.MockProducer;
@ -463,7 +464,7 @@ public class KafkaStreamsTest {
System.currentTimeMillis()); System.currentTimeMillis());
assertTrue("Timed out waiting to receive single message", latch.await(30, TimeUnit.SECONDS)); 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 { } finally {
// stop the thread so we don't interfere with other tests etc // stop the thread so we don't interfere with other tests etc
keepRunning.set(false); keepRunning.set(false);

View File

@ -47,6 +47,7 @@ import java.util.Properties;
import java.util.Set; import java.util.Set;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import static java.time.Duration.ofMillis;
import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -737,7 +738,7 @@ public class TopologyTest {
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
builder.stream("input-topic") builder.stream("input-topic")
.groupByKey() .groupByKey()
.windowedBy(TimeWindows.of(1)) .windowedBy(TimeWindows.of(ofMillis(1)))
.count(); .count();
final TopologyDescription describe = builder.build().describe(); final TopologyDescription describe = builder.build().describe();
assertEquals( assertEquals(
@ -757,7 +758,7 @@ public class TopologyTest {
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
builder.stream("input-topic") builder.stream("input-topic")
.groupByKey() .groupByKey()
.windowedBy(TimeWindows.of(1)) .windowedBy(TimeWindows.of(ofMillis(1)))
.count(Materialized.as("count-store")); .count(Materialized.as("count-store"));
final TopologyDescription describe = builder.build().describe(); final TopologyDescription describe = builder.build().describe();
assertEquals( assertEquals(
@ -777,7 +778,7 @@ public class TopologyTest {
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
builder.stream("input-topic") builder.stream("input-topic")
.groupByKey() .groupByKey()
.windowedBy(TimeWindows.of(1)) .windowedBy(TimeWindows.of(ofMillis(1)))
.count(Materialized.with(null, Serdes.Long())); .count(Materialized.with(null, Serdes.Long()));
final TopologyDescription describe = builder.build().describe(); final TopologyDescription describe = builder.build().describe();
assertEquals( assertEquals(
@ -797,7 +798,7 @@ public class TopologyTest {
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
builder.stream("input-topic") builder.stream("input-topic")
.groupByKey() .groupByKey()
.windowedBy(SessionWindows.with(1)) .windowedBy(SessionWindows.with(ofMillis(1)))
.count(); .count();
final TopologyDescription describe = builder.build().describe(); final TopologyDescription describe = builder.build().describe();
assertEquals( assertEquals(
@ -817,7 +818,7 @@ public class TopologyTest {
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
builder.stream("input-topic") builder.stream("input-topic")
.groupByKey() .groupByKey()
.windowedBy(SessionWindows.with(1)) .windowedBy(SessionWindows.with(ofMillis(1)))
.count(Materialized.as("count-store")); .count(Materialized.as("count-store"));
final TopologyDescription describe = builder.build().describe(); final TopologyDescription describe = builder.build().describe();
assertEquals( assertEquals(
@ -837,7 +838,7 @@ public class TopologyTest {
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
builder.stream("input-topic") builder.stream("input-topic")
.groupByKey() .groupByKey()
.windowedBy(SessionWindows.with(1)) .windowedBy(SessionWindows.with(ofMillis(1)))
.count(Materialized.with(null, Serdes.Long())); .count(Materialized.with(null, Serdes.Long()));
final TopologyDescription describe = builder.build().describe(); final TopologyDescription describe = builder.build().describe();
assertEquals( assertEquals(

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.integration; package org.apache.kafka.streams.integration;
import java.time.Duration;
import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClient;
import org.apache.kafka.clients.admin.ConsumerGroupDescription; import org.apache.kafka.clients.admin.ConsumerGroupDescription;
@ -66,6 +67,7 @@ import java.util.concurrent.TimeUnit;
import kafka.tools.StreamsResetter; import kafka.tools.StreamsResetter;
import static java.time.Duration.ofMillis;
import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
@ -199,7 +201,7 @@ public abstract class AbstractResetIntegrationTest {
void cleanupTest() throws Exception { void cleanupTest() throws Exception {
if (streams != null) { if (streams != null) {
streams.close(30, TimeUnit.SECONDS); streams.close(Duration.ofSeconds(30));
} }
IntegrationTestUtils.purgeLocalStreamsState(streamsConfig); IntegrationTestUtils.purgeLocalStreamsState(streamsConfig);
} }
@ -526,7 +528,7 @@ public abstract class AbstractResetIntegrationTest {
input.through(INTERMEDIATE_USER_TOPIC) input.through(INTERMEDIATE_USER_TOPIC)
.groupByKey() .groupByKey()
.windowedBy(TimeWindows.of(35).advanceBy(10)) .windowedBy(TimeWindows.of(ofMillis(35)).advanceBy(ofMillis(10)))
.count() .count()
.toStream() .toStream()
.map(new KeyValueMapper<Windowed<Long>, Long, KeyValue<Long, Long>>() { .map(new KeyValueMapper<Windowed<Long>, Long, KeyValue<Long, Long>>() {

View File

@ -17,6 +17,7 @@
package org.apache.kafka.streams.integration; package org.apache.kafka.streams.integration;
import java.time.Duration;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.LongSerializer;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
@ -52,7 +53,6 @@ import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.TimeUnit;
import kafka.utils.MockTime; import kafka.utils.MockTime;
@ -148,7 +148,7 @@ public class GlobalThreadShutDownOrderTest {
} }
}, 10000L, "Has not processed record within 10 seconds"); }, 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); final List<Long> expectedRetrievedValues = Arrays.asList(1L, 2L, 3L, 4L);
assertEquals(expectedRetrievedValues, retrievedValuesList); assertEquals(expectedRetrievedValues, retrievedValuesList);

View File

@ -57,6 +57,8 @@ import java.util.Properties;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit; 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.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForCompletion;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
@ -187,8 +189,8 @@ public class InternalTopicIntegrationTest {
textLines.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"))) textLines.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+")))
.groupBy(MockMapper.selectValueMapper()) .groupBy(MockMapper.selectValueMapper())
.windowedBy(TimeWindows.of(1000).grace(0L)) .windowedBy(TimeWindows.of(ofSeconds(1L)).grace(ofMillis(0L)))
.count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("CountWindows").withRetention(2_000L)); .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("CountWindows").withRetention(ofSeconds(2L)));
final KafkaStreams streams = new KafkaStreams(builder.build(), streamsProp); final KafkaStreams streams = new KafkaStreams(builder.build(), streamsProp);
streams.start(); streams.start();

View File

@ -53,6 +53,8 @@ import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Properties; import java.util.Properties;
import static java.time.Duration.ofMillis;
/** /**
* Similar to KStreamAggregationIntegrationTest but with dedupping enabled * Similar to KStreamAggregationIntegrationTest but with dedupping enabled
* by virtue of having a large commit interval * by virtue of having a large commit interval
@ -144,7 +146,7 @@ public class KStreamAggregationDedupIntegrationTest {
produceMessages(secondBatchTimestamp); produceMessages(secondBatchTimestamp);
groupedStream groupedStream
.windowedBy(TimeWindows.of(500L)) .windowedBy(TimeWindows.of(ofMillis(500L)))
.reduce(reducer, Materialized.as("reduce-time-windows")) .reduce(reducer, Materialized.as("reduce-time-windows"))
.toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start()) .toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start())
.to(outputTopic, Produced.with(Serdes.String(), Serdes.String())); .to(outputTopic, Produced.with(Serdes.String(), Serdes.String()));
@ -179,7 +181,7 @@ public class KStreamAggregationDedupIntegrationTest {
produceMessages(timestamp); produceMessages(timestamp);
stream.groupByKey(Serialized.with(Serdes.Integer(), Serdes.String())) stream.groupByKey(Serialized.with(Serdes.Integer(), Serdes.String()))
.windowedBy(TimeWindows.of(500L)) .windowedBy(TimeWindows.of(ofMillis(500L)))
.count(Materialized.as("count-windows")) .count(Materialized.as("count-windows"))
.toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start()) .toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start())
.to(outputTopic, Produced.with(Serdes.String(), Serdes.Long())); .to(outputTopic, Produced.with(Serdes.String(), Serdes.Long()));

View File

@ -85,6 +85,8 @@ import java.util.Set;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit; 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.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.core.Is.is; import static org.hamcrest.core.Is.is;
@ -202,7 +204,7 @@ public class KStreamAggregationIntegrationTest {
final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class); final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
groupedStream groupedStream
.windowedBy(TimeWindows.of(500L)) .windowedBy(TimeWindows.of(ofMillis(500L)))
.reduce(reducer) .reduce(reducer)
.toStream() .toStream()
.to(outputTopic, Produced.with(windowedSerde, Serdes.String())); .to(outputTopic, Produced.with(windowedSerde, Serdes.String()));
@ -307,7 +309,7 @@ public class KStreamAggregationIntegrationTest {
produceMessages(secondTimestamp); produceMessages(secondTimestamp);
final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class); final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
groupedStream.windowedBy(TimeWindows.of(500L)) groupedStream.windowedBy(TimeWindows.of(ofMillis(500L)))
.aggregate( .aggregate(
initializer, initializer,
aggregator, aggregator,
@ -427,7 +429,7 @@ public class KStreamAggregationIntegrationTest {
produceMessages(timestamp); produceMessages(timestamp);
stream.groupByKey(Serialized.with(Serdes.Integer(), Serdes.String())) stream.groupByKey(Serialized.with(Serdes.Integer(), Serdes.String()))
.windowedBy(TimeWindows.of(500L)) .windowedBy(TimeWindows.of(ofMillis(500L)))
.count() .count()
.toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start()).to(outputTopic, Produced.with(Serdes.String(), Serdes.Long())); .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())) builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey(Serialized.with(Serdes.String(), Serdes.String())) .groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
.windowedBy(SessionWindows.with(sessionGap)) .windowedBy(SessionWindows.with(ofMillis(sessionGap)))
.count() .count()
.toStream() .toStream()
.transform(() -> new Transformer<Windowed<String>, Long, KeyValue<Object, Object>>() { .transform(() -> new Transformer<Windowed<String>, Long, KeyValue<Object, Object>>() {
@ -618,7 +620,7 @@ public class KStreamAggregationIntegrationTest {
final String userSessionsStore = "UserSessionsStore"; final String userSessionsStore = "UserSessionsStore";
builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String())) builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey(Serialized.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)) .reduce((value1, value2) -> value1 + ":" + value2, Materialized.as(userSessionsStore))
.toStream() .toStream()
.foreach((key, value) -> { .foreach((key, value) -> {
@ -705,7 +707,7 @@ public class KStreamAggregationIntegrationTest {
builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String())) builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey(Serialized.with(Serdes.String(), Serdes.String())) .groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
.windowedBy(UnlimitedWindows.of().startOn(startTime)) .windowedBy(UnlimitedWindows.of().startOn(ofEpochMilli(startTime)))
.count() .count()
.toStream() .toStream()
.transform(() -> new Transformer<Windowed<String>, Long, KeyValue<Object, Object>>() { .transform(() -> new Transformer<Windowed<String>, Long, KeyValue<Object, Object>>() {

View File

@ -18,6 +18,7 @@
package org.apache.kafka.streams.integration; package org.apache.kafka.streams.integration;
import java.time.Duration;
import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
@ -51,7 +52,6 @@ import java.util.Map;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
@Category({IntegrationTest.class}) @Category({IntegrationTest.class})
public class KTableSourceTopicRestartIntegrationTest { public class KTableSourceTopicRestartIntegrationTest {
@ -130,7 +130,7 @@ public class KTableSourceTopicRestartIntegrationTest {
assertNumberValuesRead(readKeyValues, expectedResultsWithDataWrittenDuringRestoreMap, "Table did not get all values after restart"); assertNumberValuesRead(readKeyValues, expectedResultsWithDataWrittenDuringRestoreMap, "Table did not get all values after restart");
} finally { } 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"); assertNumberValuesRead(readKeyValues, expectedResultsWithDataWrittenDuringRestoreMap, "Table did not get all values after restart");
} finally { } 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"); assertNumberValuesRead(readKeyValues, expectedValues, "Table did not get all values after restart");
} finally { } finally {
streamsOne.close(5, TimeUnit.SECONDS); streamsOne.close(Duration.ofSeconds(5));
} }
} }

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.integration; package org.apache.kafka.streams.integration;
import java.time.Duration;
import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClient;
import org.apache.kafka.clients.admin.Config; import org.apache.kafka.clients.admin.Config;
import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerConfig;
@ -51,7 +52,6 @@ import java.util.List;
import java.util.Properties; import java.util.Properties;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
@Category({IntegrationTest.class}) @Category({IntegrationTest.class})
public class PurgeRepartitionTopicIntegrationTest { public class PurgeRepartitionTopicIntegrationTest {
@ -173,7 +173,7 @@ public class PurgeRepartitionTopicIntegrationTest {
@After @After
public void shutdown() throws IOException { public void shutdown() throws IOException {
if (kafkaStreams != null) { if (kafkaStreams != null) {
kafkaStreams.close(30, TimeUnit.SECONDS); kafkaStreams.close(Duration.ofSeconds(30));
} }
} }

View File

@ -86,6 +86,9 @@ import java.util.TreeSet;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; 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.MatcherAssert.assertThat;
import static org.hamcrest.core.IsEqual.equalTo; import static org.hamcrest.core.IsEqual.equalTo;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -220,7 +223,7 @@ public class QueryableStateIntegrationTest {
@After @After
public void shutdown() throws IOException { public void shutdown() throws IOException {
if (kafkaStreams != null) { if (kafkaStreams != null) {
kafkaStreams.close(30, TimeUnit.SECONDS); kafkaStreams.close(ofSeconds(30));
} }
IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration); IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration);
} }
@ -256,7 +259,7 @@ public class QueryableStateIntegrationTest {
// Create a Windowed State Store that contains the word count for every 1 minute // Create a Windowed State Store that contains the word count for every 1 minute
groupedByWord groupedByWord
.windowedBy(TimeWindows.of(WINDOW_SIZE)) .windowedBy(TimeWindows.of(ofMillis(WINDOW_SIZE)))
.count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as(windowStoreName + "-" + inputTopic)) .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as(windowStoreName + "-" + inputTopic))
.toStream(new KeyValueMapper<Windowed<String>, Long, String>() { .toStream(new KeyValueMapper<Windowed<String>, Long, String>() {
@Override @Override
@ -361,7 +364,7 @@ public class QueryableStateIntegrationTest {
final int index = metadata.hostInfo().port(); final int index = metadata.hostInfo().port();
final KafkaStreams streamsWithKey = streamRunnables[index].getStream(); final KafkaStreams streamsWithKey = streamRunnables[index].getStream();
final ReadOnlyWindowStore<String, Long> store = streamsWithKey.store(storeName, QueryableStoreTypes.<String, Long>windowStore()); 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) { } catch (final IllegalStateException e) {
// Kafka Streams instance may have closed but rebalance hasn't happened // Kafka Streams instance may have closed but rebalance hasn't happened
return false; return false;
@ -695,7 +698,7 @@ public class QueryableStateIntegrationTest {
final String windowStoreName = "windowed-count"; final String windowStoreName = "windowed-count";
s1.groupByKey() s1.groupByKey()
.windowedBy(TimeWindows.of(WINDOW_SIZE)) .windowedBy(TimeWindows.of(ofMillis(WINDOW_SIZE)))
.count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as(windowStoreName)); .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as(windowStoreName));
kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration); kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration);
kafkaStreams.start(); kafkaStreams.start();
@ -1017,7 +1020,7 @@ public class QueryableStateIntegrationTest {
private Set<KeyValue<String, Long>> fetch(final ReadOnlyWindowStore<String, Long> store, private Set<KeyValue<String, Long>> fetch(final ReadOnlyWindowStore<String, Long> store,
final String key) { 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()) { if (fetch.hasNext()) {
final KeyValue<Long, Long> next = fetch.next(); final KeyValue<Long, Long> next = fetch.next();
return Collections.singleton(KeyValue.pair(key, next.value)); 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, private Map<String, Long> fetchMap(final ReadOnlyWindowStore<String, Long> store,
final String key) { 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()) { if (fetch.hasNext()) {
final KeyValue<Long, Long> next = fetch.next(); final KeyValue<Long, Long> next = fetch.next();
return Collections.singletonMap(key, next.value); return Collections.singletonMap(key, next.value);

View File

@ -55,10 +55,11 @@ import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Locale; import java.util.Locale;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher; import java.util.regex.Matcher;
import java.util.regex.Pattern; 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.hamcrest.CoreMatchers.equalTo;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat; import static org.junit.Assert.assertThat;
@ -161,7 +162,7 @@ public class RepartitionOptimizingIntegrationTest {
mappedStream.filter((k, v) -> k.equals("A")) mappedStream.filter((k, v) -> k.equals("A"))
.join(countStream, (v1, v2) -> v1 + ":" + v2.toString(), .join(countStream, (v1, v2) -> v1 + ":" + v2.toString(),
JoinWindows.of(5000), JoinWindows.of(ofMillis(5000)),
Joined.with(Serdes.String(), Serdes.String(), Serdes.Long())) Joined.with(Serdes.String(), Serdes.String(), Serdes.Long()))
.to(JOINED_TOPIC); .to(JOINED_TOPIC);
@ -211,7 +212,7 @@ public class RepartitionOptimizingIntegrationTest {
assertThat(3, equalTo(processorValueCollector.size())); assertThat(3, equalTo(processorValueCollector.size()));
assertThat(processorValueCollector, equalTo(expectedCollectedProcessorValues)); assertThat(processorValueCollector, equalTo(expectedCollectedProcessorValues));
streams.close(5, TimeUnit.SECONDS); streams.close(ofSeconds(5));
} }

View File

@ -18,6 +18,7 @@
package org.apache.kafka.streams.integration; package org.apache.kafka.streams.integration;
import java.time.Duration;
import kafka.utils.MockTime; import kafka.utils.MockTime;
import org.apache.kafka.common.serialization.LongDeserializer; import org.apache.kafka.common.serialization.LongDeserializer;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
@ -46,7 +47,6 @@ import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher; import java.util.regex.Matcher;
import java.util.regex.Pattern; 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")); 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); IntegrationTestUtils.waitUntilFinalKeyValueRecordsReceived(consumerConfig2, COUNT_STRING_TOPIC, expectedStringCountKeyValues);
streams.close(5, TimeUnit.SECONDS); streams.close(Duration.ofSeconds(5));
} }

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.integration; package org.apache.kafka.streams.integration;
import java.time.Duration;
import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.KafkaConsumer;
@ -113,7 +114,7 @@ public class RestoreIntegrationTest {
@After @After
public void shutdown() { public void shutdown() {
if (kafkaStreams != null) { if (kafkaStreams != null) {
kafkaStreams.close(30, TimeUnit.SECONDS); kafkaStreams.close(Duration.ofSeconds(30));
} }
} }

View File

@ -32,6 +32,7 @@ import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import static java.time.Duration.ofSeconds;
/** /**
* Tests all available joins of Kafka Streams DSL. * 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") 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); runTest(expectedResult);
} }
@ -109,7 +110,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
leftStream.map(MockMapper.<Long, String>noOpKeyValueMapper()) leftStream.map(MockMapper.<Long, String>noOpKeyValueMapper())
.join(rightStream.flatMap(MockMapper.<Long, String>noOpFlatKeyValueMapper()) .join(rightStream.flatMap(MockMapper.<Long, String>noOpFlatKeyValueMapper())
.selectKey(MockMapper.<Long, String>selectKeyKeyValueMapper()), .selectKey(MockMapper.<Long, String>selectKeyKeyValueMapper()),
valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC);
runTest(expectedResult); runTest(expectedResult);
} }
@ -136,7 +137,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
Arrays.asList("D-a", "D-b", "D-c", "D-d") 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); runTest(expectedResult);
} }
@ -166,7 +167,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
leftStream.map(MockMapper.<Long, String>noOpKeyValueMapper()) leftStream.map(MockMapper.<Long, String>noOpKeyValueMapper())
.leftJoin(rightStream.flatMap(MockMapper.<Long, String>noOpFlatKeyValueMapper()) .leftJoin(rightStream.flatMap(MockMapper.<Long, String>noOpFlatKeyValueMapper())
.selectKey(MockMapper.<Long, String>selectKeyKeyValueMapper()), .selectKey(MockMapper.<Long, String>selectKeyKeyValueMapper()),
valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC);
runTest(expectedResult); runTest(expectedResult);
} }
@ -193,7 +194,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
Arrays.asList("D-a", "D-b", "D-c", "D-d") 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); runTest(expectedResult);
} }
@ -223,7 +224,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
leftStream.map(MockMapper.<Long, String>noOpKeyValueMapper()) leftStream.map(MockMapper.<Long, String>noOpKeyValueMapper())
.outerJoin(rightStream.flatMap(MockMapper.<Long, String>noOpFlatKeyValueMapper()) .outerJoin(rightStream.flatMap(MockMapper.<Long, String>noOpFlatKeyValueMapper())
.selectKey(MockMapper.<Long, String>selectKeyKeyValueMapper()), .selectKey(MockMapper.<Long, String>selectKeyKeyValueMapper()),
valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC);
runTest(expectedResult); 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") "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)) leftStream.join(rightStream, valueJoiner, JoinWindows.of(ofSeconds(10)))
.join(rightStream, valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); .join(rightStream, valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC);
runTest(expectedResult); runTest(expectedResult);
} }

View File

@ -434,7 +434,7 @@ public class SuppressionIntegrationTest {
Consumed.with(STRING_SERDE, STRING_SERDE) Consumed.with(STRING_SERDE, STRING_SERDE)
) )
.groupBy((String k1, String v1) -> k1, Grouped.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()); .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("counts").withCachingDisabled().withLoggingDisabled());
valueCounts valueCounts

View File

@ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream;
import org.junit.Test; 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.verifyEquality;
import static org.apache.kafka.streams.EqualityCheck.verifyInEquality; import static org.apache.kafka.streams.EqualityCheck.verifyInEquality;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -31,29 +32,29 @@ public class JoinWindowsTest {
@Test @Test
public void validWindows() { public void validWindows() {
JoinWindows.of(ANY_OTHER_SIZE) // [ -anyOtherSize ; anyOtherSize ] JoinWindows.of(ofMillis(ANY_OTHER_SIZE)) // [ -anyOtherSize ; anyOtherSize ]
.before(ANY_SIZE) // [ -anySize ; anyOtherSize ] .before(ofMillis(ANY_SIZE)) // [ -anySize ; anyOtherSize ]
.before(0) // [ 0 ; anyOtherSize ] .before(ofMillis(0)) // [ 0 ; anyOtherSize ]
.before(-ANY_SIZE) // [ anySize ; anyOtherSize ] .before(ofMillis(-ANY_SIZE)) // [ anySize ; anyOtherSize ]
.before(-ANY_OTHER_SIZE); // [ anyOtherSize ; anyOtherSize ] .before(ofMillis(-ANY_OTHER_SIZE)); // [ anyOtherSize ; anyOtherSize ]
JoinWindows.of(ANY_OTHER_SIZE) // [ -anyOtherSize ; anyOtherSize ] JoinWindows.of(ofMillis(ANY_OTHER_SIZE)) // [ -anyOtherSize ; anyOtherSize ]
.after(ANY_SIZE) // [ -anyOtherSize ; anySize ] .after(ofMillis(ANY_SIZE)) // [ -anyOtherSize ; anySize ]
.after(0) // [ -anyOtherSize ; 0 ] .after(ofMillis(0)) // [ -anyOtherSize ; 0 ]
.after(-ANY_SIZE) // [ -anyOtherSize ; -anySize ] .after(ofMillis(-ANY_SIZE)) // [ -anyOtherSize ; -anySize ]
.after(-ANY_OTHER_SIZE); // [ -anyOtherSize ; -anyOtherSize ] .after(ofMillis(-ANY_OTHER_SIZE)); // [ -anyOtherSize ; -anyOtherSize ]
} }
@Test(expected = IllegalArgumentException.class) @Test(expected = IllegalArgumentException.class)
public void timeDifferenceMustNotBeNegative() { public void timeDifferenceMustNotBeNegative() {
JoinWindows.of(-1); JoinWindows.of(ofMillis(-1));
} }
@Test @Test
public void endTimeShouldNotBeBeforeStart() { public void endTimeShouldNotBeBeforeStart() {
final JoinWindows windowSpec = JoinWindows.of(ANY_SIZE); final JoinWindows windowSpec = JoinWindows.of(ofMillis(ANY_SIZE));
try { try {
windowSpec.after(-ANY_SIZE - 1); windowSpec.after(ofMillis(-ANY_SIZE - 1));
fail("window end time should not be before window start time"); fail("window end time should not be before window start time");
} catch (final IllegalArgumentException e) { } catch (final IllegalArgumentException e) {
// expected // expected
@ -62,9 +63,9 @@ public class JoinWindowsTest {
@Test @Test
public void startTimeShouldNotBeAfterEnd() { public void startTimeShouldNotBeAfterEnd() {
final JoinWindows windowSpec = JoinWindows.of(ANY_SIZE); final JoinWindows windowSpec = JoinWindows.of(ofMillis(ANY_SIZE));
try { try {
windowSpec.before(-ANY_SIZE - 1); windowSpec.before(ofMillis(-ANY_SIZE - 1));
fail("window start time should not be after window end time"); fail("window start time should not be after window end time");
} catch (final IllegalArgumentException e) { } catch (final IllegalArgumentException e) {
// expected // expected
@ -74,7 +75,7 @@ public class JoinWindowsTest {
@Deprecated @Deprecated
@Test @Test
public void untilShouldSetMaintainDuration() { public void untilShouldSetMaintainDuration() {
final JoinWindows windowSpec = JoinWindows.of(ANY_SIZE); final JoinWindows windowSpec = JoinWindows.of(ofMillis(ANY_SIZE));
final long windowSize = windowSpec.size(); final long windowSize = windowSpec.size();
assertEquals(windowSize, windowSpec.until(windowSize).maintainMs()); assertEquals(windowSize, windowSpec.until(windowSize).maintainMs());
} }
@ -82,7 +83,7 @@ public class JoinWindowsTest {
@Deprecated @Deprecated
@Test @Test
public void retentionTimeMustNoBeSmallerThanWindowSize() { public void retentionTimeMustNoBeSmallerThanWindowSize() {
final JoinWindows windowSpec = JoinWindows.of(ANY_SIZE); final JoinWindows windowSpec = JoinWindows.of(ofMillis(ANY_SIZE));
final long windowSize = windowSpec.size(); final long windowSize = windowSpec.size();
try { try {
windowSpec.until(windowSize - 1); windowSpec.until(windowSize - 1);
@ -94,10 +95,10 @@ public class JoinWindowsTest {
@Test @Test
public void gracePeriodShouldEnforceBoundaries() { public void gracePeriodShouldEnforceBoundaries() {
JoinWindows.of(3L).grace(0L); JoinWindows.of(ofMillis(3L)).grace(ofMillis(0L));
try { try {
JoinWindows.of(3L).grace(-1L); JoinWindows.of(ofMillis(3L)).grace(ofMillis(-1L));
fail("should not accept negatives"); fail("should not accept negatives");
} catch (final IllegalArgumentException e) { } catch (final IllegalArgumentException e) {
//expected //expected
@ -106,58 +107,58 @@ public class JoinWindowsTest {
@Test @Test
public void equalsAndHashcodeShouldBeValidForPositiveCases() { 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( verifyEquality(
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(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 is a little weird in that before and after set the same fields as of. // JoinWindows is a little weird in that before and after set the same fields as of.
verifyEquality( verifyEquality(
JoinWindows.of(9).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(3).before(1).after(2).grace(3).until(60) JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60)
); );
} }
@Test @Test
public void equalsAndHashcodeShouldBeValidForNegativeCases() { 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( verifyInEquality(
JoinWindows.of(3).before(9).after(2).grace(3).until(60), JoinWindows.of(ofMillis(3)).before(ofMillis(9)).after(ofMillis(2)).grace(ofMillis(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)
); );
verifyInEquality( verifyInEquality(
JoinWindows.of(3).before(1).after(9).grace(3).until(60), JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(9)).grace(ofMillis(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)
); );
verifyInEquality( verifyInEquality(
JoinWindows.of(3).before(1).after(2).grace(9).until(60), JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(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(3)).until(60)
); );
verifyInEquality( verifyInEquality(
JoinWindows.of(3).before(1).after(2).grace(3).until(90), JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(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(60)
); );
} }
} }

View File

@ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream;
import org.junit.Test; 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.verifyEquality;
import static org.apache.kafka.streams.EqualityCheck.verifyInEquality; import static org.apache.kafka.streams.EqualityCheck.verifyInEquality;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -28,23 +29,23 @@ public class SessionWindowsTest {
@Test @Test
public void shouldSetWindowGap() { public void shouldSetWindowGap() {
final long anyGap = 42L; final long anyGap = 42L;
assertEquals(anyGap, SessionWindows.with(anyGap).inactivityGap()); assertEquals(anyGap, SessionWindows.with(ofMillis(anyGap)).inactivityGap());
} }
@Deprecated @Deprecated
@Test @Test
public void shouldSetWindowRetentionTime() { public void shouldSetWindowRetentionTime() {
final long anyRetentionTime = 42L; final long anyRetentionTime = 42L;
assertEquals(anyRetentionTime, SessionWindows.with(1).until(anyRetentionTime).maintainMs()); assertEquals(anyRetentionTime, SessionWindows.with(ofMillis(1)).until(anyRetentionTime).maintainMs());
} }
@Test @Test
public void gracePeriodShouldEnforceBoundaries() { public void gracePeriodShouldEnforceBoundaries() {
SessionWindows.with(3L).grace(0L); SessionWindows.with(ofMillis(3L)).grace(ofMillis(0));
try { try {
SessionWindows.with(3L).grace(-1L); SessionWindows.with(ofMillis(3L)).grace(ofMillis(-1L));
fail("should not accept negatives"); fail("should not accept negatives");
} catch (final IllegalArgumentException e) { } catch (final IllegalArgumentException e) {
//expected //expected
@ -53,25 +54,25 @@ public class SessionWindowsTest {
@Test(expected = IllegalArgumentException.class) @Test(expected = IllegalArgumentException.class)
public void windowSizeMustNotBeNegative() { public void windowSizeMustNotBeNegative() {
SessionWindows.with(-1); SessionWindows.with(ofMillis(-1));
} }
@Test(expected = IllegalArgumentException.class) @Test(expected = IllegalArgumentException.class)
public void windowSizeMustNotBeZero() { public void windowSizeMustNotBeZero() {
SessionWindows.with(0); SessionWindows.with(ofMillis(0));
} }
@SuppressWarnings("deprecation") // specifically testing deprecated apis @SuppressWarnings("deprecation") // specifically testing deprecated apis
@Test @Test
public void retentionTimeShouldBeGapIfGapIsLargerThanDefaultRetentionTime() { public void retentionTimeShouldBeGapIfGapIsLargerThanDefaultRetentionTime() {
final long windowGap = 2 * SessionWindows.with(1).maintainMs(); final long windowGap = 2 * SessionWindows.with(ofMillis(1)).maintainMs();
assertEquals(windowGap, SessionWindows.with(windowGap).maintainMs()); assertEquals(windowGap, SessionWindows.with(ofMillis(windowGap)).maintainMs());
} }
@Deprecated @Deprecated
@Test @Test
public void retentionTimeMustNotBeNegative() { public void retentionTimeMustNotBeNegative() {
final SessionWindows windowSpec = SessionWindows.with(42); final SessionWindows windowSpec = SessionWindows.with(ofMillis(42));
try { try {
windowSpec.until(41); windowSpec.until(41);
fail("should not accept retention time smaller than gap"); fail("should not accept retention time smaller than gap");
@ -82,27 +83,27 @@ public class SessionWindowsTest {
@Test @Test
public void equalsAndHashcodeShouldBeValidForPositiveCases() { 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 @Test
public void equalsAndHashcodeShouldBeValidForNegativeCases() { 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));
} }
} }

View File

@ -21,6 +21,7 @@ import org.junit.Test;
import java.util.Map; 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.verifyEquality;
import static org.apache.kafka.streams.EqualityCheck.verifyInEquality; import static org.apache.kafka.streams.EqualityCheck.verifyInEquality;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -33,43 +34,43 @@ public class TimeWindowsTest {
@Test @Test
public void shouldSetWindowSize() { public void shouldSetWindowSize() {
assertEquals(ANY_SIZE, TimeWindows.of(ANY_SIZE).sizeMs); assertEquals(ANY_SIZE, TimeWindows.of(ofMillis(ANY_SIZE)).sizeMs);
} }
@Test @Test
public void shouldSetWindowAdvance() { public void shouldSetWindowAdvance() {
final long anyAdvance = 4; 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 @SuppressWarnings("deprecation") // specifically testing deprecated APIs
@Test @Test
public void shouldSetWindowRetentionTime() { 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 @SuppressWarnings("deprecation") // specifically testing deprecated APIs
@Test @Test
public void shouldUseWindowSizeAsRentitionTimeIfWindowSizeIsLargerThanDefaultRetentionTime() { public void shouldUseWindowSizeAsRentitionTimeIfWindowSizeIsLargerThanDefaultRetentionTime() {
final long windowSize = 2 * TimeWindows.of(1).maintainMs(); final long windowSize = 2 * TimeWindows.of(ofMillis(1)).maintainMs();
assertEquals(windowSize, TimeWindows.of(windowSize).maintainMs()); assertEquals(windowSize, TimeWindows.of(ofMillis(windowSize)).maintainMs());
} }
@Test(expected = IllegalArgumentException.class) @Test(expected = IllegalArgumentException.class)
public void windowSizeMustNotBeZero() { public void windowSizeMustNotBeZero() {
TimeWindows.of(0); TimeWindows.of(ofMillis(0));
} }
@Test(expected = IllegalArgumentException.class) @Test(expected = IllegalArgumentException.class)
public void windowSizeMustNotBeNegative() { public void windowSizeMustNotBeNegative() {
TimeWindows.of(-1); TimeWindows.of(ofMillis(-1));
} }
@Test @Test
public void advanceIntervalMustNotBeZero() { public void advanceIntervalMustNotBeZero() {
final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); final TimeWindows windowSpec = TimeWindows.of(ofMillis(ANY_SIZE));
try { try {
windowSpec.advanceBy(0); windowSpec.advanceBy(ofMillis(0));
fail("should not accept zero advance parameter"); fail("should not accept zero advance parameter");
} catch (final IllegalArgumentException e) { } catch (final IllegalArgumentException e) {
// expected // expected
@ -78,9 +79,9 @@ public class TimeWindowsTest {
@Test @Test
public void advanceIntervalMustNotBeNegative() { public void advanceIntervalMustNotBeNegative() {
final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); final TimeWindows windowSpec = TimeWindows.of(ofMillis(ANY_SIZE));
try { try {
windowSpec.advanceBy(-1); windowSpec.advanceBy(ofMillis(-1));
fail("should not accept negative advance parameter"); fail("should not accept negative advance parameter");
} catch (final IllegalArgumentException e) { } catch (final IllegalArgumentException e) {
// expected // expected
@ -90,9 +91,9 @@ public class TimeWindowsTest {
@Deprecated @Deprecated
@Test @Test
public void advanceIntervalMustNotBeLargerThanWindowSize() { public void advanceIntervalMustNotBeLargerThanWindowSize() {
final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); final TimeWindows windowSpec = TimeWindows.of(ofMillis(ANY_SIZE));
try { try {
windowSpec.advanceBy(ANY_SIZE + 1); windowSpec.advanceBy(ofMillis(ANY_SIZE + 1));
fail("should not accept advance greater than window size"); fail("should not accept advance greater than window size");
} catch (final IllegalArgumentException e) { } catch (final IllegalArgumentException e) {
// expected // expected
@ -102,7 +103,7 @@ public class TimeWindowsTest {
@Deprecated @Deprecated
@Test @Test
public void retentionTimeMustNoBeSmallerThanWindowSize() { public void retentionTimeMustNoBeSmallerThanWindowSize() {
final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); final TimeWindows windowSpec = TimeWindows.of(ofMillis(ANY_SIZE));
try { try {
windowSpec.until(ANY_SIZE - 1); windowSpec.until(ANY_SIZE - 1);
fail("should not accept retention time smaller than window size"); fail("should not accept retention time smaller than window size");
@ -113,10 +114,10 @@ public class TimeWindowsTest {
@Test @Test
public void gracePeriodShouldEnforceBoundaries() { public void gracePeriodShouldEnforceBoundaries() {
TimeWindows.of(3L).grace(0L); TimeWindows.of(ofMillis(3L)).grace(ofMillis(0L));
try { try {
TimeWindows.of(3L).grace(-1L); TimeWindows.of(ofMillis(3L)).grace(ofMillis(-1L));
fail("should not accept negatives"); fail("should not accept negatives");
} catch (final IllegalArgumentException e) { } catch (final IllegalArgumentException e) {
//expected //expected
@ -125,7 +126,7 @@ public class TimeWindowsTest {
@Test @Test
public void shouldComputeWindowsForHoppingWindows() { 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); final Map<Long, TimeWindow> matched = windows.windowsFor(21L);
assertEquals(12L / 5L + 1, matched.size()); assertEquals(12L / 5L + 1, matched.size());
assertEquals(new TimeWindow(10L, 22L), matched.get(10L)); assertEquals(new TimeWindow(10L, 22L), matched.get(10L));
@ -135,7 +136,7 @@ public class TimeWindowsTest {
@Test @Test
public void shouldComputeWindowsForBarelyOverlappingHoppingWindows() { 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); final Map<Long, TimeWindow> matched = windows.windowsFor(7L);
assertEquals(1, matched.size()); assertEquals(1, matched.size());
assertEquals(new TimeWindow(5L, 11L), matched.get(5L)); assertEquals(new TimeWindow(5L, 11L), matched.get(5L));
@ -143,7 +144,7 @@ public class TimeWindowsTest {
@Test @Test
public void shouldComputeWindowsForTumblingWindows() { public void shouldComputeWindowsForTumblingWindows() {
final TimeWindows windows = TimeWindows.of(12L); final TimeWindows windows = TimeWindows.of(ofMillis(12L));
final Map<Long, TimeWindow> matched = windows.windowsFor(21L); final Map<Long, TimeWindow> matched = windows.windowsFor(21L);
assertEquals(1, matched.size()); assertEquals(1, matched.size());
assertEquals(new TimeWindow(12L, 24L), matched.get(12L)); assertEquals(new TimeWindow(12L, 24L), matched.get(12L));
@ -152,49 +153,49 @@ public class TimeWindowsTest {
@Test @Test
public void equalsAndHashcodeShouldBeValidForPositiveCases() { 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( verifyEquality(
TimeWindows.of(3).advanceBy(1).grace(1).until(4), TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1)).grace(ofMillis(1)).until(4),
TimeWindows.of(3).advanceBy(1).grace(1).until(4) TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1)).grace(ofMillis(1)).until(4)
); );
} }
@Test @Test
public void equalsAndHashcodeShouldBeValidForNegativeCases() { 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( verifyInEquality(
TimeWindows.of(4).advanceBy(2).grace(2).until(4), TimeWindows.of(ofMillis(4)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4),
TimeWindows.of(3).advanceBy(2).grace(2).until(4) TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4)
); );
verifyInEquality( verifyInEquality(
TimeWindows.of(3).advanceBy(1).grace(2).until(4), TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1)).grace(ofMillis(2)).until(4),
TimeWindows.of(3).advanceBy(2).grace(2).until(4) TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4)
); );
assertNotEquals( assertNotEquals(
TimeWindows.of(3).advanceBy(2).grace(1).until(4), TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(1)).until(4),
TimeWindows.of(3).advanceBy(2).grace(2).until(4) TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4)
); );
assertNotEquals( assertNotEquals(
TimeWindows.of(3).advanceBy(2).grace(2).until(9), TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(9),
TimeWindows.of(3).advanceBy(2).grace(2).until(4) TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4)
); );
} }
} }

View File

@ -21,6 +21,7 @@ import org.junit.Test;
import java.util.Map; 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.verifyEquality;
import static org.apache.kafka.streams.EqualityCheck.verifyInEquality; import static org.apache.kafka.streams.EqualityCheck.verifyInEquality;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -33,12 +34,12 @@ public class UnlimitedWindowsTest {
@Test @Test
public void shouldSetWindowStartTime() { public void shouldSetWindowStartTime() {
assertEquals(anyStartTime, UnlimitedWindows.of().startOn(anyStartTime).startMs); assertEquals(anyStartTime, UnlimitedWindows.of().startOn(ofEpochMilli(anyStartTime)).startMs);
} }
@Test(expected = IllegalArgumentException.class) @Test(expected = IllegalArgumentException.class)
public void startTimeMustNotBeNegative() { public void startTimeMustNotBeNegative() {
UnlimitedWindows.of().startOn(-1); UnlimitedWindows.of().startOn(ofEpochMilli(-1));
} }
@Test @Test
@ -54,7 +55,7 @@ public class UnlimitedWindowsTest {
@Test @Test
public void shouldIncludeRecordsThatHappenedOnWindowStart() { 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); final Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(w.startMs);
assertEquals(1, matchedWindows.size()); assertEquals(1, matchedWindows.size());
assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime)); assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime));
@ -62,7 +63,7 @@ public class UnlimitedWindowsTest {
@Test @Test
public void shouldIncludeRecordsThatHappenedAfterWindowStart() { 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 long timestamp = w.startMs + 1;
final Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(timestamp); final Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(timestamp);
assertEquals(1, matchedWindows.size()); assertEquals(1, matchedWindows.size());
@ -71,7 +72,7 @@ public class UnlimitedWindowsTest {
@Test @Test
public void shouldExcludeRecordsThatHappenedBeforeWindowStart() { 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 long timestamp = w.startMs - 1;
final Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(timestamp); final Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(timestamp);
assertTrue(matchedWindows.isEmpty()); assertTrue(matchedWindows.isEmpty());
@ -81,13 +82,13 @@ public class UnlimitedWindowsTest {
public void equalsAndHashcodeShouldBeValidForPositiveCases() { public void equalsAndHashcodeShouldBeValidForPositiveCases() {
verifyEquality(UnlimitedWindows.of(), UnlimitedWindows.of()); 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 @Test
public void equalsAndHashcodeShouldBeValidForNegativeCases() { public void equalsAndHashcodeShouldBeValidForNegativeCases() {
verifyInEquality(UnlimitedWindows.of().startOn(9), UnlimitedWindows.of().startOn(1)); verifyInEquality(UnlimitedWindows.of().startOn(ofEpochMilli(9)), UnlimitedWindows.of().startOn(ofEpochMilli(1)));
} }
} }

View File

@ -59,6 +59,7 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Properties; import java.util.Properties;
import static java.time.Duration.ofMillis;
import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; import static org.apache.kafka.test.StreamsTestUtils.getMetricByName;
import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.CoreMatchers.hasItem;
@ -96,7 +97,7 @@ public class KGroupedStreamImplTest {
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldNotHaveNullReducerWithWindowedReduce() { 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) @Test(expected = NullPointerException.class)
@ -106,7 +107,7 @@ public class KGroupedStreamImplTest {
@Test(expected = InvalidTopicException.class) @Test(expected = InvalidTopicException.class)
public void shouldNotHaveInvalidStoreNameWithWindowedReduce() { 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) @Test(expected = NullPointerException.class)
@ -126,12 +127,12 @@ public class KGroupedStreamImplTest {
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldNotHaveNullInitializerOnWindowedAggregate() { 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) @Test(expected = NullPointerException.class)
public void shouldNotHaveNullAdderOnWindowedAggregate() { 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) @Test(expected = NullPointerException.class)
@ -141,7 +142,7 @@ public class KGroupedStreamImplTest {
@Test(expected = InvalidTopicException.class) @Test(expected = InvalidTopicException.class)
public void shouldNotHaveInvalidStoreNameOnWindowedAggregate() { 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) { private void doAggregateSessionWindows(final Map<Windowed<String>, Integer> results) {
@ -161,7 +162,7 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldAggregateSessionWindows() { public void shouldAggregateSessionWindows() {
final Map<Windowed<String>, Integer> results = new HashMap<>(); 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 @Override
public Integer apply() { public Integer apply() {
return 0; return 0;
@ -191,7 +192,7 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldAggregateSessionWindowsWithInternalStoreName() { public void shouldAggregateSessionWindowsWithInternalStoreName() {
final Map<Windowed<String>, Integer> results = new HashMap<>(); 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 @Override
public Integer apply() { public Integer apply() {
return 0; return 0;
@ -234,7 +235,7 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldCountSessionWindows() { public void shouldCountSessionWindows() {
final Map<Windowed<String>, Long> results = new HashMap<>(); 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")); .count(Materialized.<String, Long, SessionStore<Bytes, byte[]>>as("session-store"));
table.toStream().foreach(new ForeachAction<Windowed<String>, Long>() { table.toStream().foreach(new ForeachAction<Windowed<String>, Long>() {
@Override @Override
@ -249,7 +250,7 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldCountSessionWindowsWithInternalStoreName() { public void shouldCountSessionWindowsWithInternalStoreName() {
final Map<Windowed<String>, Long> results = new HashMap<>(); 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>() { table.toStream().foreach(new ForeachAction<Windowed<String>, Long>() {
@Override @Override
public void apply(final Windowed<String> key, final Long value) { public void apply(final Windowed<String> key, final Long value) {
@ -277,7 +278,7 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldReduceSessionWindows() { public void shouldReduceSessionWindows() {
final Map<Windowed<String>, String> results = new HashMap<>(); 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>() { .reduce(new Reducer<String>() {
@Override @Override
public String apply(final String value1, final String value2) { public String apply(final String value1, final String value2) {
@ -297,7 +298,7 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldReduceSessionWindowsWithInternalStoreName() { public void shouldReduceSessionWindowsWithInternalStoreName() {
final Map<Windowed<String>, String> results = new HashMap<>(); 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>() { .reduce(new Reducer<String>() {
@Override @Override
public String apply(final String value1, final String value2) { public String apply(final String value1, final String value2) {
@ -316,7 +317,7 @@ public class KGroupedStreamImplTest {
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldNotAcceptNullReducerWhenReducingSessionWindows() { 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) @Test(expected = NullPointerException.class)
@ -326,17 +327,17 @@ public class KGroupedStreamImplTest {
@Test(expected = InvalidTopicException.class) @Test(expected = InvalidTopicException.class)
public void shouldNotAcceptInvalidStoreNameWhenReducingSessionWindows() { 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) @Test(expected = NullPointerException.class)
public void shouldNotAcceptNullStateStoreSupplierWhenReducingSessionWindows() { 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) @Test(expected = NullPointerException.class)
public void shouldNotAcceptNullInitializerWhenAggregatingSessionWindows() { 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 @Override
public String apply(final String aggKey, final String aggOne, final String aggTwo) { public String apply(final String aggKey, final String aggOne, final String aggTwo) {
return null; return null;
@ -346,7 +347,7 @@ public class KGroupedStreamImplTest {
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldNotAcceptNullAggregatorWhenAggregatingSessionWindows() { 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 @Override
public String apply(final String aggKey, final String aggOne, final String aggTwo) { public String apply(final String aggKey, final String aggOne, final String aggTwo) {
return null; return null;
@ -356,7 +357,7 @@ public class KGroupedStreamImplTest {
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldNotAcceptNullSessionMergerWhenAggregatingSessionWindows() { 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, null,
Materialized.<String, String, SessionStore<Bytes, byte[]>>as("storeName")); Materialized.<String, String, SessionStore<Bytes, byte[]>>as("storeName"));
} }
@ -368,7 +369,7 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldAcceptNullStoreNameWhenAggregatingSessionWindows() { 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>() { .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, new Merger<String, String>() {
@Override @Override
public String apply(final String aggKey, final String aggOne, final String aggTwo) { public String apply(final String aggKey, final String aggOne, final String aggTwo) {
@ -379,7 +380,7 @@ public class KGroupedStreamImplTest {
@Test(expected = InvalidTopicException.class) @Test(expected = InvalidTopicException.class)
public void shouldNotAcceptInvalidStoreNameWhenAggregatingSessionWindows() { 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>() { .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, new Merger<String, String>() {
@Override @Override
public String apply(final String aggKey, final String aggOne, final String aggTwo) { public String apply(final String aggKey, final String aggOne, final String aggTwo) {
@ -558,7 +559,7 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldCountWindowed() { public void shouldCountWindowed() {
final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>(); 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() .toStream()
.foreach(new ForeachAction<Windowed<String>, Long>() { .foreach(new ForeachAction<Windowed<String>, Long>() {
@Override @Override
@ -573,7 +574,7 @@ public class KGroupedStreamImplTest {
@Test @Test
public void shouldCountWindowedWithInternalStoreName() { public void shouldCountWindowedWithInternalStoreName() {
final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>(); final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>();
groupedStream.windowedBy(TimeWindows.of(500L)).count() groupedStream.windowedBy(TimeWindows.of(ofMillis(500L))).count()
.toStream() .toStream()
.foreach(new ForeachAction<Windowed<String>, Long>() { .foreach(new ForeachAction<Windowed<String>, Long>() {
@Override @Override

View File

@ -63,6 +63,7 @@ import java.util.Properties;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import static java.time.Duration.ofMillis;
import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.hamcrest.core.IsInstanceOf.instanceOf;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -162,14 +163,14 @@ public class KStreamImplTest {
public Integer apply(final Integer value1, final Integer value2) { public Integer apply(final Integer value1, final Integer value2) {
return value1 + value2; return value1 + value2;
} }
}, JoinWindows.of(anyWindowSize), joined); }, JoinWindows.of(ofMillis(anyWindowSize)), joined);
streams2[1].join(streams3[1], new ValueJoiner<Integer, Integer, Integer>() { streams2[1].join(streams3[1], new ValueJoiner<Integer, Integer, Integer>() {
@Override @Override
public Integer apply(final Integer value1, final Integer value2) { public Integer apply(final Integer value1, final Integer value2) {
return value1 + value2; return value1 + value2;
} }
}, JoinWindows.of(anyWindowSize), joined); }, JoinWindows.of(ofMillis(anyWindowSize)), joined);
stream4.to("topic-5"); stream4.to("topic-5");
@ -383,7 +384,7 @@ public class KStreamImplTest {
}); });
stream.join(kStream, stream.join(kStream,
valueJoiner, valueJoiner,
JoinWindows.of(windowSize).until(3 * windowSize), JoinWindows.of(ofMillis(windowSize)).until(3 * windowSize),
Joined.with(Serdes.String(), Joined.with(Serdes.String(),
Serdes.String(), Serdes.String(),
Serdes.String())) Serdes.String()))
@ -418,7 +419,7 @@ public class KStreamImplTest {
stream.join( stream.join(
kStream, kStream,
valueJoiner, valueJoiner,
JoinWindows.of(windowSize).grace(3L * windowSize), JoinWindows.of(ofMillis(windowSize)).grace(ofMillis(3L * windowSize)),
Joined.with(Serdes.String(), Serdes.String(), Serdes.String()) Joined.with(Serdes.String(), Serdes.String(), Serdes.String())
) )
.to("output-topic", Produced.with(Serdes.String(), Serdes.String())); .to("output-topic", Produced.with(Serdes.String(), Serdes.String()));
@ -537,12 +538,12 @@ public class KStreamImplTest {
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldNotAllowNullOtherStreamOnJoin() { 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) @Test(expected = NullPointerException.class)
public void shouldNotAllowNullValueJoinerOnJoin() { public void shouldNotAllowNullValueJoinerOnJoin() {
testStream.join(testStream, null, JoinWindows.of(10)); testStream.join(testStream, null, JoinWindows.of(ofMillis(10)));
} }
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
@ -655,12 +656,12 @@ public class KStreamImplTest {
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldThrowNullPointerOnJoinWithStreamWhenJoinedIsNull() { 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) @Test(expected = NullPointerException.class)
public void shouldThrowNullPointerOnOuterJoinJoinedIsNull() { 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 @Test

View File

@ -41,6 +41,7 @@ import java.util.HashSet;
import java.util.Properties; import java.util.Properties;
import java.util.Set; import java.util.Set;
import static java.time.Duration.ofMillis;
import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; import static org.apache.kafka.test.StreamsTestUtils.getMetricByName;
import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.CoreMatchers.hasItem;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -71,7 +72,7 @@ public class KStreamKStreamJoinTest {
return value1 + value2; return value1 + value2;
} }
}, },
JoinWindows.of(100), JoinWindows.of(ofMillis(100)),
Joined.with(Serdes.String(), Serdes.Integer(), Serdes.Integer()) Joined.with(Serdes.String(), Serdes.Integer(), Serdes.Integer())
); );
@ -101,7 +102,7 @@ public class KStreamKStreamJoinTest {
joined = stream1.join( joined = stream1.join(
stream2, stream2,
MockValueJoiner.TOSTRING_JOINER, MockValueJoiner.TOSTRING_JOINER,
JoinWindows.of(100), JoinWindows.of(ofMillis(100)),
Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String()));
joined.process(supplier); joined.process(supplier);
@ -204,7 +205,7 @@ public class KStreamKStreamJoinTest {
joined = stream1.outerJoin( joined = stream1.outerJoin(
stream2, stream2,
MockValueJoiner.TOSTRING_JOINER, MockValueJoiner.TOSTRING_JOINER,
JoinWindows.of(100), JoinWindows.of(ofMillis(100)),
Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String()));
joined.process(supplier); joined.process(supplier);
final Collection<Set<String>> copartitionGroups = TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups(); final Collection<Set<String>> copartitionGroups = TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
@ -308,7 +309,7 @@ public class KStreamKStreamJoinTest {
joined = stream1.join( joined = stream1.join(
stream2, stream2,
MockValueJoiner.TOSTRING_JOINER, MockValueJoiner.TOSTRING_JOINER,
JoinWindows.of(100), JoinWindows.of(ofMillis(100)),
Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String()));
joined.process(supplier); joined.process(supplier);
@ -529,7 +530,7 @@ public class KStreamKStreamJoinTest {
joined = stream1.join( joined = stream1.join(
stream2, stream2,
MockValueJoiner.TOSTRING_JOINER, MockValueJoiner.TOSTRING_JOINER,
JoinWindows.of(0).after(100), JoinWindows.of(ofMillis(0)).after(ofMillis(100)),
Joined.with(Serdes.Integer(), Joined.with(Serdes.Integer(),
Serdes.String(), Serdes.String(),
Serdes.String())); Serdes.String()));
@ -640,7 +641,7 @@ public class KStreamKStreamJoinTest {
joined = stream1.join( joined = stream1.join(
stream2, stream2,
MockValueJoiner.TOSTRING_JOINER, MockValueJoiner.TOSTRING_JOINER,
JoinWindows.of(0).before(100), JoinWindows.of(ofMillis(0)).before(ofMillis(100)),
Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String()));
joined.process(supplier); joined.process(supplier);

View File

@ -39,6 +39,7 @@ import java.util.HashSet;
import java.util.Properties; import java.util.Properties;
import java.util.Set; import java.util.Set;
import static java.time.Duration.ofMillis;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
public class KStreamKStreamLeftJoinTest { public class KStreamKStreamLeftJoinTest {
@ -65,7 +66,7 @@ public class KStreamKStreamLeftJoinTest {
joined = stream1.leftJoin(stream2, joined = stream1.leftJoin(stream2,
MockValueJoiner.TOSTRING_JOINER, MockValueJoiner.TOSTRING_JOINER,
JoinWindows.of(100), JoinWindows.of(ofMillis(100)),
Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String()));
joined.process(supplier); joined.process(supplier);
@ -151,7 +152,7 @@ public class KStreamKStreamLeftJoinTest {
joined = stream1.leftJoin(stream2, joined = stream1.leftJoin(stream2,
MockValueJoiner.TOSTRING_JOINER, MockValueJoiner.TOSTRING_JOINER,
JoinWindows.of(100), JoinWindows.of(ofMillis(100)),
Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String()));
joined.process(supplier); joined.process(supplier);

View File

@ -50,6 +50,7 @@ import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; 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.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; 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 Merger<String, Long> sessionMerger = (aggKey, aggOne, aggTwo) -> aggOne + aggTwo;
private final KStreamSessionWindowAggregate<String, String, Long> sessionAggregator = private final KStreamSessionWindowAggregate<String, String, Long> sessionAggregator =
new KStreamSessionWindowAggregate<>( new KStreamSessionWindowAggregate<>(
SessionWindows.with(GAP_MS), SessionWindows.with(ofMillis(GAP_MS)),
STORE_NAME, STORE_NAME,
initializer, initializer,
aggregator, aggregator,
@ -106,7 +107,7 @@ public class KStreamSessionWindowAggregateProcessorTest {
} }
private void initStore(final boolean enableCaching) { 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.String(),
Serdes.Long()) Serdes.Long())
.withLoggingDisabled(); .withLoggingDisabled();
@ -322,7 +323,7 @@ public class KStreamSessionWindowAggregateProcessorTest {
LogCaptureAppender.setClassLoggerToDebug(KStreamSessionWindowAggregate.class); LogCaptureAppender.setClassLoggerToDebug(KStreamSessionWindowAggregate.class);
final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(); final LogCaptureAppender appender = LogCaptureAppender.createAndRegister();
final Processor<String, String> processor = new KStreamSessionWindowAggregate<>( final Processor<String, String> processor = new KStreamSessionWindowAggregate<>(
SessionWindows.with(10L).grace(10L), SessionWindows.with(ofMillis(10L)).grace(ofMillis(10L)),
STORE_NAME, STORE_NAME,
initializer, initializer,
aggregator, aggregator,

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.kstream.internals; package org.apache.kafka.streams.kstream.internals;
import java.time.Duration;
import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.IntegerSerializer;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.Consumed;
@ -112,7 +113,7 @@ public class KStreamTransformTest {
@Override @Override
public void init(final ProcessorContext context) { 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 @Override
public void punctuate(final long timestamp) { public void punctuate(final long timestamp) {
context.forward(-1, (int) timestamp); context.forward(-1, (int) timestamp);

View File

@ -47,6 +47,7 @@ import org.junit.Test;
import java.util.List; import java.util.List;
import java.util.Properties; 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.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; import static org.apache.kafka.test.StreamsTestUtils.getMetricByName;
@ -70,7 +71,7 @@ public class KStreamWindowAggregateTest {
final KTable<Windowed<String>, String> table2 = builder final KTable<Windowed<String>, String> table2 = builder
.stream(topic1, Consumed.with(Serdes.String(), Serdes.String())) .stream(topic1, Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey(Serialized.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())); .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<>(); final MockProcessorSupplier<Windowed<String>, String> supplier = new MockProcessorSupplier<>();
@ -128,7 +129,7 @@ public class KStreamWindowAggregateTest {
final KTable<Windowed<String>, String> table1 = builder final KTable<Windowed<String>, String> table1 = builder
.stream(topic1, Consumed.with(Serdes.String(), Serdes.String())) .stream(topic1, Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey(Serialized.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())); .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<>(); final MockProcessorSupplier<Windowed<String>, String> supplier = new MockProcessorSupplier<>();
@ -137,7 +138,7 @@ public class KStreamWindowAggregateTest {
final KTable<Windowed<String>, String> table2 = builder final KTable<Windowed<String>, String> table2 = builder
.stream(topic2, Consumed.with(Serdes.String(), Serdes.String())) .stream(topic2, Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey(Serialized.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())); .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("topic2-Canonized").withValueSerde(Serdes.String()));
table2.toStream().process(supplier); 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())); final KStream<String, String> stream1 = builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String()));
stream1.groupByKey(Serialized.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( .aggregate(
MockInitializer.STRING_INIT, MockInitializer.STRING_INIT,
MockAggregator.toStringInstance("+"), MockAggregator.toStringInstance("+"),
@ -257,7 +258,7 @@ public class KStreamWindowAggregateTest {
final KStream<String, String> stream1 = builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String())); final KStream<String, String> stream1 = builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String()));
stream1.groupByKey(Serialized.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( .aggregate(
() -> "", () -> "",
MockAggregator.toStringInstance("+"), MockAggregator.toStringInstance("+"),
@ -316,7 +317,7 @@ public class KStreamWindowAggregateTest {
final KStream<String, String> stream1 = builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String())); final KStream<String, String> stream1 = builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String()));
stream1.groupByKey(Serialized.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( .aggregate(
() -> "", () -> "",
MockAggregator.toStringInstance("+"), MockAggregator.toStringInstance("+"),

View File

@ -36,6 +36,7 @@ import org.junit.Test;
import java.util.Properties; 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.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; import static org.apache.kafka.test.StreamsTestUtils.getMetricByName;
@ -58,7 +59,7 @@ public class KStreamWindowReduceTest {
builder builder
.stream("TOPIC", Consumed.with(Serdes.String(), Serdes.String())) .stream("TOPIC", Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey(Serialized.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); .reduce((value1, value2) -> value1 + "+" + value2);
@ -80,7 +81,7 @@ public class KStreamWindowReduceTest {
builder builder
.stream("TOPIC", Consumed.with(Serdes.String(), Serdes.String())) .stream("TOPIC", Consumed.with(Serdes.String(), Serdes.String()))
.groupByKey(Serialized.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) .reduce((value1, value2) -> value1 + "+" + value2)
.toStream() .toStream()
.map((key, value) -> new KeyValue<>(key.toString(), value)) .map((key, value) -> new KeyValue<>(key.toString(), value))

View File

@ -47,6 +47,7 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Properties; import java.util.Properties;
import static java.time.Duration.ofMillis;
import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
@ -69,7 +70,7 @@ public class SessionWindowedKStreamImplTest {
public void before() { public void before() {
final KStream<String, String> stream = builder.stream(TOPIC, Consumed.with(Serdes.String(), Serdes.String())); final KStream<String, String> stream = builder.stream(TOPIC, Consumed.with(Serdes.String(), Serdes.String()));
this.stream = stream.groupByKey(Serialized.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 @Test

View File

@ -368,7 +368,7 @@ public class SuppressScenarioTest {
final KTable<Windowed<String>, Long> valueCounts = builder final KTable<Windowed<String>, Long> valueCounts = builder
.stream("input", Consumed.with(STRING_SERDE, STRING_SERDE)) .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE))
.groupBy((String k, String v) -> k, Grouped.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()); .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("counts").withCachingDisabled());
valueCounts valueCounts
.suppress(untilWindowCloses(unbounded())) .suppress(untilWindowCloses(unbounded()))
@ -422,7 +422,7 @@ public class SuppressScenarioTest {
final KTable<Windowed<String>, Long> valueCounts = builder final KTable<Windowed<String>, Long> valueCounts = builder
.stream("input", Consumed.with(STRING_SERDE, STRING_SERDE)) .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE))
.groupBy((String k, String v) -> k, Grouped.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)); .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("counts").withCachingDisabled().withKeySerde(STRING_SERDE));
valueCounts valueCounts
.suppress(untilWindowCloses(unbounded())) .suppress(untilWindowCloses(unbounded()))
@ -481,7 +481,7 @@ public class SuppressScenarioTest {
final KTable<Windowed<String>, Long> valueCounts = builder final KTable<Windowed<String>, Long> valueCounts = builder
.stream("input", Consumed.with(STRING_SERDE, STRING_SERDE)) .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE))
.groupBy((String k, String v) -> k, Grouped.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()); .count(Materialized.<String, Long, SessionStore<Bytes, byte[]>>as("counts").withCachingDisabled());
valueCounts valueCounts
.suppress(untilWindowCloses(unbounded())) .suppress(untilWindowCloses(unbounded()))

View File

@ -32,6 +32,7 @@ import org.junit.Test;
import java.time.Duration; 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.BufferConfig.unbounded;
import static org.apache.kafka.streams.kstream.Suppressed.untilTimeLimit; import static org.apache.kafka.streams.kstream.Suppressed.untilTimeLimit;
import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses;
@ -146,7 +147,7 @@ public class SuppressTopologyTest {
anonymousNodeBuilder anonymousNodeBuilder
.stream("input", Consumed.with(STRING_SERDE, STRING_SERDE)) .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE))
.groupBy((String k, String v) -> k, Grouped.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()) .count(Materialized.<String, Long, SessionStore<Bytes, byte[]>>as("counts").withCachingDisabled())
.suppress(untilWindowCloses(unbounded())) .suppress(untilWindowCloses(unbounded()))
.toStream() .toStream()
@ -164,7 +165,7 @@ public class SuppressTopologyTest {
namedNodeBuilder namedNodeBuilder
.stream("input", Consumed.with(STRING_SERDE, STRING_SERDE)) .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE))
.groupBy((String k, String v) -> k, Grouped.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()) .count(Materialized.<String, Long, SessionStore<Bytes, byte[]>>as("counts").withCachingDisabled())
.suppress(untilWindowCloses(unbounded()).withName("myname")) .suppress(untilWindowCloses(unbounded()).withName("myname"))
.toStream() .toStream()

View File

@ -21,6 +21,7 @@ import org.junit.Test;
import java.util.Map; import java.util.Map;
import static java.time.Duration.ofMillis;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
@ -124,7 +125,7 @@ public class TimeWindowTest {
@Test @Test
public void shouldReturnMatchedWindowsOrderedByTimestamp() { 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 Map<Long, TimeWindow> matched = windows.windowsFor(21L);
final Long[] expected = matched.keySet().toArray(new Long[matched.size()]); final Long[] expected = matched.keySet().toArray(new Long[matched.size()]);

View File

@ -46,6 +46,8 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Properties; 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.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
@ -61,7 +63,7 @@ public class TimeWindowedKStreamImplTest {
public void before() { public void before() {
final KStream<String, String> stream = builder.stream(TOPIC, Consumed.with(Serdes.String(), Serdes.String())); final KStream<String, String> stream = builder.stream(TOPIC, Consumed.with(Serdes.String(), Serdes.String()));
windowedStream = stream.groupByKey(Serialized.with(Serdes.String(), Serdes.String())) windowedStream = stream.groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
.windowedBy(TimeWindows.of(500L)); .windowedBy(TimeWindows.of(ofMillis(500L)));
} }
@Test @Test
@ -136,7 +138,7 @@ public class TimeWindowedKStreamImplTest {
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) {
processData(driver); processData(driver);
final WindowStore<String, Long> windowStore = driver.getWindowStore("count-store"); final WindowStore<String, Long> windowStore = driver.getWindowStore("count-store");
final List<KeyValue<Windowed<String>, Long>> data = 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( assertThat(data, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), 2L), 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)) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) {
processData(driver); processData(driver);
final WindowStore<String, String> windowStore = driver.getWindowStore("reduced"); 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( assertThat(data, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), "1+2"), 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)) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) {
processData(driver); processData(driver);
final WindowStore<String, String> windowStore = driver.getWindowStore("aggregated"); 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( assertThat(data, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), "0+1+2"), KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), "0+1+2"),
KeyValue.pair(new Windowed<>("1", new TimeWindow(500, 1000)), "0+3"), KeyValue.pair(new Windowed<>("1", new TimeWindow(500, 1000)), "0+3"),

View File

@ -26,6 +26,7 @@ import org.apache.kafka.streams.processor.Processor;
import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorContext;
import org.junit.Test; import org.junit.Test;
import static java.time.Duration.ofMillis;
import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertThat; import static org.junit.Assert.assertThat;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
@ -78,7 +79,7 @@ public class GraphGraceSearchUtilTest {
@Test @Test
public void shouldExtractGraceFromKStreamWindowAggregateNode() { 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<>( final StatefulProcessorNode<String, Long> node = new StatefulProcessorNode<>(
"asdf", "asdf",
new ProcessorParameters<>( new ProcessorParameters<>(
@ -101,7 +102,7 @@ public class GraphGraceSearchUtilTest {
@Test @Test
public void shouldExtractGraceFromKStreamSessionWindowAggregateNode() { 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<>( final StatefulProcessorNode<String, Long> node = new StatefulProcessorNode<>(
"asdf", "asdf",
@ -126,7 +127,7 @@ public class GraphGraceSearchUtilTest {
@Test @Test
public void shouldExtractGraceFromAncestorThroughStatefulParent() { 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<>( final StatefulProcessorNode<String, Long> graceGrandparent = new StatefulProcessorNode<>(
"asdf", "asdf",
new ProcessorParameters<>(new KStreamSessionWindowAggregate<String, Long, Integer>( new ProcessorParameters<>(new KStreamSessionWindowAggregate<String, Long, Integer>(
@ -167,7 +168,7 @@ public class GraphGraceSearchUtilTest {
@Test @Test
public void shouldExtractGraceFromAncestorThroughStatelessParent() { 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<>( final StatefulProcessorNode<String, Long> graceGrandparent = new StatefulProcessorNode<>(
"asdf", "asdf",
new ProcessorParameters<>( new ProcessorParameters<>(
@ -201,7 +202,7 @@ public class GraphGraceSearchUtilTest {
"asdf", "asdf",
new ProcessorParameters<>( new ProcessorParameters<>(
new KStreamSessionWindowAggregate<String, Long, Integer>( new KStreamSessionWindowAggregate<String, Long, Integer>(
SessionWindows.with(10L).grace(1234L), SessionWindows.with(ofMillis(10L)).grace(ofMillis(1234L)),
"asdf", "asdf",
null, null,
null, null,
@ -218,7 +219,7 @@ public class GraphGraceSearchUtilTest {
"asdf", "asdf",
new ProcessorParameters<>( new ProcessorParameters<>(
new KStreamWindowAggregate<String, Long, Integer, TimeWindow>( new KStreamWindowAggregate<String, Long, Integer, TimeWindow>(
TimeWindows.of(10L).grace(4321L), TimeWindows.of(ofMillis(10L)).grace(ofMillis(4321L)),
"asdf", "asdf",
null, null,
null null

View File

@ -34,6 +34,7 @@ import java.util.Properties;
import java.util.regex.Matcher; import java.util.regex.Matcher;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import static java.time.Duration.ofMillis;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
public class StreamsGraphTest { public class StreamsGraphTest {
@ -51,7 +52,7 @@ public class StreamsGraphTest {
final ValueJoiner<String, String, String> valueJoiner = (v, v2) -> v + v2; 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 // build step one
assertEquals(expectedJoinedTopology, builder.build().describe().toString()); assertEquals(expectedJoinedTopology, builder.build().describe().toString());
@ -100,7 +101,7 @@ public class StreamsGraphTest {
final KStream<String, String> mappedKeyStream = inputStream.selectKey((k, v) -> k + v); final KStream<String, String> mappedKeyStream = inputStream.selectKey((k, v) -> k + v);
mappedKeyStream.mapValues(v -> v.toUpperCase(Locale.getDefault())).groupByKey().count().toStream().to("output"); 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); 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"); final KStream<String, String> mappedKeyStream = inputStream.selectKey((k, v) -> k + v).through("through-topic");
mappedKeyStream.groupByKey().count().toStream().to("output"); 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); return builder.build(properties);

View File

@ -54,7 +54,6 @@ import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStore;
import java.io.IOException; import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Locale; import java.util.Locale;
@ -63,6 +62,10 @@ import java.util.Random;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit; 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 * Class that provides support for a series of benchmarks. It is usually driven by
* tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py. * tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py.
@ -332,7 +335,7 @@ public class SimpleBenchmark {
consumer.seekToBeginning(partitions); consumer.seekToBeginning(partitions);
while (true) { 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 (records.isEmpty()) {
if (processedRecords == numRecords) { if (processedRecords == numRecords) {
break; break;
@ -370,7 +373,7 @@ public class SimpleBenchmark {
consumer.seekToBeginning(partitions); consumer.seekToBeginning(partitions);
while (true) { 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 (records.isEmpty()) {
if (processedRecords == numRecords) { if (processedRecords == numRecords) {
break; break;
@ -498,7 +501,7 @@ public class SimpleBenchmark {
@Override @Override
public void process(final Integer key, final byte[] value) { public void process(final Integer key, final byte[] value) {
final long timestamp = context().timestamp(); 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()) { while (iter.hasNext()) {
iter.next(); iter.next();
} }
@ -550,7 +553,7 @@ public class SimpleBenchmark {
input.peek(new CountDownAction(latch)) input.peek(new CountDownAction(latch))
.groupByKey() .groupByKey()
.windowedBy(TimeWindows.of(AGGREGATE_WINDOW_SIZE).advanceBy(AGGREGATE_WINDOW_ADVANCE)) .windowedBy(TimeWindows.of(ofMillis(AGGREGATE_WINDOW_SIZE)).advanceBy(ofMillis(AGGREGATE_WINDOW_ADVANCE)))
.count(); .count();
final KafkaStreams streams = createKafkaStreamsWithExceptionHandler(builder, props); 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[]> input1 = builder.stream(kStreamTopic1);
final KStream<Integer, byte[]> input2 = builder.stream(kStreamTopic2); 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); final KafkaStreams streams = createKafkaStreamsWithExceptionHandler(builder, props);
@ -677,7 +680,7 @@ public class SimpleBenchmark {
public void uncaughtException(final Thread t, final Throwable e) { public void uncaughtException(final Thread t, final Throwable e) {
System.out.println("FATAL: An unexpected exception is encountered on thread " + t + ": " + e); System.out.println("FATAL: An unexpected exception is encountered on thread " + t + ": " + e);
streamsClient.close(30, TimeUnit.SECONDS); streamsClient.close(ofSeconds(30));
} }
}); });

View File

@ -17,6 +17,7 @@
package org.apache.kafka.streams.perf; package org.apache.kafka.streams.perf;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import java.time.Duration;
import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.ProducerRecord;
@ -334,7 +335,7 @@ public class YahooBenchmark {
// calculate windowed counts // calculate windowed counts
keyedByCampaign keyedByCampaign
.groupByKey(Serialized.with(Serdes.String(), Serdes.String())) .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")); .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("time-windows"));
return new KafkaStreams(builder.build(), streamConfig); return new KafkaStreams(builder.build(), streamConfig);

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.processor.internals; package org.apache.kafka.streams.processor.internals;
import java.time.Duration;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.header.internals.RecordHeader;
@ -202,6 +203,13 @@ public class AbstractProcessorContextTest {
return null; return null;
} }
@Override
public Cancellable schedule(final Duration interval,
final PunctuationType type,
final Punctuator callback) throws IllegalArgumentException {
return null;
}
@Override @Override
public <K, V> void forward(final K key, final V value) {} public <K, V> void forward(final K key, final V value) {}

View File

@ -47,6 +47,7 @@ import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.regex.Pattern; 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.mkList;
import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.common.utils.Utils.mkSet;
import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.hamcrest.core.IsInstanceOf.instanceOf;
@ -588,7 +589,7 @@ public class InternalTopologyBuilderTest {
builder.addProcessor("processor", new MockProcessorSupplier(), "source"); builder.addProcessor("processor", new MockProcessorSupplier(), "source");
builder.addStateStore( builder.addStateStore(
Stores.windowStoreBuilder( Stores.windowStoreBuilder(
Stores.persistentWindowStore("store1", 30_000L, 10_000L, false), Stores.persistentWindowStore("store1", ofSeconds(30L), ofSeconds(10L), false),
Serdes.String(), Serdes.String(),
Serdes.String() Serdes.String()
), ),
@ -596,7 +597,7 @@ public class InternalTopologyBuilderTest {
); );
builder.addStateStore( builder.addStateStore(
Stores.sessionStoreBuilder( Stores.sessionStoreBuilder(
Stores.persistentSessionStore("store2", 30000), Serdes.String(), Serdes.String() Stores.persistentSessionStore("store2", ofSeconds(30)), Serdes.String(), Serdes.String()
), ),
"processor" "processor"
); );

View File

@ -72,6 +72,7 @@ import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import static java.time.Duration.ofMillis;
import static java.util.Collections.emptyList; import static java.util.Collections.emptyList;
import static java.util.Collections.emptySet; import static java.util.Collections.emptySet;
import static java.util.Collections.singletonList; import static java.util.Collections.singletonList;
@ -211,7 +212,7 @@ public class StandbyTaskTest {
} }
restoreStateConsumer.seekToBeginning(partition); 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 StandbyContextImpl context = (StandbyContextImpl) task.context();
final MockKeyValueStore store1 = (MockKeyValueStore) context.getStateMgr().getStore(storeName1); final MockKeyValueStore store1 = (MockKeyValueStore) context.getStateMgr().getStore(storeName1);
@ -239,8 +240,8 @@ public class StandbyTaskTest {
builder builder
.stream(Collections.singleton("topic"), new ConsumedInternal<>()) .stream(Collections.singleton("topic"), new ConsumedInternal<>())
.groupByKey() .groupByKey()
.windowedBy(TimeWindows.of(60_000).grace(0L)) .windowedBy(TimeWindows.of(ofMillis(60_000)).grace(ofMillis(0L)))
.count(Materialized.<Object, Long, WindowStore<Bytes, byte[]>>as(storeName).withRetention(120_000L)); .count(Materialized.<Object, Long, WindowStore<Bytes, byte[]>>as(storeName).withRetention(ofMillis(120_000L)));
builder.buildAndOptimizeTopology(); builder.buildAndOptimizeTopology();
@ -484,7 +485,7 @@ public class StandbyTaskTest {
@Test @Test
public void shouldInitializeWindowStoreWithoutException() throws IOException { public void shouldInitializeWindowStoreWithoutException() throws IOException {
final InternalStreamsBuilder builder = new InternalStreamsBuilder(new InternalTopologyBuilder()); 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); initializeStandbyStores(builder);
} }

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.processor.internals; package org.apache.kafka.streams.processor.internals;
import java.time.Duration;
import org.apache.kafka.clients.admin.MockAdminClient; import org.apache.kafka.clients.admin.MockAdminClient;
import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
@ -1040,13 +1041,13 @@ public class StreamThreadTest {
return new Processor<Object, Object>() { return new Processor<Object, Object>() {
@Override @Override
public void init(final ProcessorContext context) { 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 @Override
public void punctuate(final long timestamp) { public void punctuate(final long timestamp) {
punctuatedStreamTime.add(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 @Override
public void punctuate(final long timestamp) { public void punctuate(final long timestamp) {
punctuatedWallClockTime.add(timestamp); punctuatedWallClockTime.add(timestamp);

View File

@ -60,6 +60,7 @@ import java.util.Set;
import java.util.UUID; import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import static java.time.Duration.ofMillis;
import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.CoreMatchers.not;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -961,7 +962,7 @@ public class StreamsPartitionAssignorTest {
return null; return null;
} }
}, },
JoinWindows.of(0) JoinWindows.of(ofMillis(0))
); );
final UUID uuid = UUID.randomUUID(); final UUID uuid = UUID.randomUUID();

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.state; package org.apache.kafka.streams.state;
import java.time.Instant;
import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStore;
@ -88,22 +89,43 @@ public class NoOpWindowStore implements ReadOnlyWindowStore, StateStore {
} }
@Override @Override
@SuppressWarnings("deprecation")
public WindowStoreIterator fetch(final Object key, final long timeFrom, final long timeTo) { public WindowStoreIterator fetch(final Object key, final long timeFrom, final long timeTo) {
return EMPTY_WINDOW_STORE_ITERATOR; return EMPTY_WINDOW_STORE_ITERATOR;
} }
@Override @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) { public WindowStoreIterator<KeyValue> fetch(final Object from, final Object to, final long timeFrom, final long timeTo) {
return EMPTY_WINDOW_STORE_ITERATOR; 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 @Override
public WindowStoreIterator<KeyValue> all() { public WindowStoreIterator<KeyValue> all() {
return EMPTY_WINDOW_STORE_ITERATOR; return EMPTY_WINDOW_STORE_ITERATOR;
} }
@Override @Override
@SuppressWarnings("deprecation")
public WindowStoreIterator<KeyValue> fetchAll(final long timeFrom, final long timeTo) { public WindowStoreIterator<KeyValue> fetchAll(final long timeFrom, final long timeTo) {
return EMPTY_WINDOW_STORE_ITERATOR; return EMPTY_WINDOW_STORE_ITERATOR;
} }
@Override
public KeyValueIterator fetchAll(final Instant from, final Instant to) {
return EMPTY_WINDOW_STORE_ITERATOR;
}
} }

View File

@ -24,6 +24,7 @@ import org.apache.kafka.streams.state.internals.RocksDBStore;
import org.apache.kafka.streams.state.internals.RocksDBWindowStore; import org.apache.kafka.streams.state.internals.RocksDBWindowStore;
import org.junit.Test; import org.junit.Test;
import static java.time.Duration.ofMillis;
import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.CoreMatchers.nullValue;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.hamcrest.core.IsInstanceOf.instanceOf;
@ -70,7 +71,7 @@ public class StoresTest {
@Test(expected = IllegalArgumentException.class) @Test(expected = IllegalArgumentException.class)
public void shouldThrowIfIPersistentWindowStoreIfWindowSizeIsNegative() { public void shouldThrowIfIPersistentWindowStoreIfWindowSizeIsNegative() {
Stores.persistentWindowStore("anyName", 0L, -1L, false); Stores.persistentWindowStore("anyName", ofMillis(0L), ofMillis(-1L), false);
} }
@Test(expected = IllegalArgumentException.class) @Test(expected = IllegalArgumentException.class)
@ -80,12 +81,13 @@ public class StoresTest {
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldThrowIfIPersistentSessionStoreStoreNameIsNull() { public void shouldThrowIfIPersistentSessionStoreStoreNameIsNull() {
Stores.persistentSessionStore(null, 0); Stores.persistentSessionStore(null, ofMillis(0));
} }
@Test(expected = IllegalArgumentException.class) @Test(expected = IllegalArgumentException.class)
public void shouldThrowIfIPersistentSessionStoreRetentionPeriodIsNegative() { public void shouldThrowIfIPersistentSessionStoreRetentionPeriodIsNegative() {
Stores.persistentSessionStore("anyName", -1); Stores.persistentSessionStore("anyName", ofMillis(-1));
} }
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
@ -120,18 +122,18 @@ public class StoresTest {
@Test @Test
public void shouldCreateRocksDbWindowStore() { 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 @Test
public void shouldCreateRocksDbSessionStore() { public void shouldCreateRocksDbSessionStore() {
assertThat(Stores.persistentSessionStore("store", 1).get(), instanceOf(RocksDBSessionStore.class)); assertThat(Stores.persistentSessionStore("store", ofMillis(1)).get(), instanceOf(RocksDBSessionStore.class));
} }
@Test @Test
public void shouldBuildWindowStore() { public void shouldBuildWindowStore() {
final WindowStore<String, String> store = Stores.windowStoreBuilder( 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(),
Serdes.String() Serdes.String()
).build(); ).build();
@ -151,7 +153,7 @@ public class StoresTest {
@Test @Test
public void shouldBuildSessionStore() { public void shouldBuildSessionStore() {
final SessionStore<String, String> store = Stores.sessionStoreBuilder( final SessionStore<String, String> store = Stores.sessionStoreBuilder(
Stores.persistentSessionStore("name", 10), Stores.persistentSessionStore("name", ofMillis(10)),
Serdes.String(), Serdes.String(),
Serdes.String() Serdes.String()
).build(); ).build();

View File

@ -49,6 +49,9 @@ import java.util.List;
import java.util.Properties; import java.util.Properties;
import java.util.UUID; 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.common.utils.Utils.mkList;
import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize; import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize;
import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.toList;
@ -101,7 +104,7 @@ public class CachingWindowStoreTest {
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
final StoreBuilder<WindowStore<String, String>> storeBuilder = Stores.windowStoreBuilder( 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(),
Serdes.String()) Serdes.String())
.withCachingEnabled(); .withCachingEnabled();
@ -197,8 +200,8 @@ public class CachingWindowStoreTest {
assertThat(cachingStore.fetch(bytesKey("c"), 10), equalTo(null)); assertThat(cachingStore.fetch(bytesKey("c"), 10), equalTo(null));
assertThat(cachingStore.fetch(bytesKey("a"), 0), equalTo(null)); assertThat(cachingStore.fetch(bytesKey("a"), 0), equalTo(null));
final WindowStoreIterator<byte[]> a = cachingStore.fetch(bytesKey("a"), 10, 10); final WindowStoreIterator<byte[]> a = cachingStore.fetch(bytesKey("a"), ofEpochMilli(10), ofEpochMilli(10));
final WindowStoreIterator<byte[]> b = cachingStore.fetch(bytesKey("b"), 10, 10); final WindowStoreIterator<byte[]> b = cachingStore.fetch(bytesKey("b"), ofEpochMilli(10), ofEpochMilli(10));
verifyKeyValue(a.next(), DEFAULT_TIMESTAMP, "a"); verifyKeyValue(a.next(), DEFAULT_TIMESTAMP, "a");
verifyKeyValue(b.next(), DEFAULT_TIMESTAMP, "b"); verifyKeyValue(b.next(), DEFAULT_TIMESTAMP, "b");
assertFalse(a.hasNext()); assertFalse(a.hasNext());
@ -224,7 +227,7 @@ public class CachingWindowStoreTest {
cachingStore.put(bytesKey("a"), bytesValue("a")); cachingStore.put(bytesKey("a"), bytesValue("a"));
cachingStore.put(bytesKey("b"), bytesValue("b")); 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("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"); verifyWindowedKeyValue(iterator.next(), new Windowed<>(bytesKey("b"), new TimeWindow(DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE)), "b");
assertFalse(iterator.hasNext()); assertFalse(iterator.hasNext());
@ -258,21 +261,21 @@ public class CachingWindowStoreTest {
cachingStore.put(bytesKey(array[i]), bytesValue(array[i])); 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++) { for (int i = 0; i < array.length; i++) {
final String str = array[i]; final String str = array[i];
verifyWindowedKeyValue(iterator.next(), new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), str); verifyWindowedKeyValue(iterator.next(), new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), str);
} }
assertFalse(iterator.hasNext()); 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++) { for (int i = 2; i <= 4; i++) {
final String str = array[i]; final String str = array[i];
verifyWindowedKeyValue(iterator1.next(), new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), str); verifyWindowedKeyValue(iterator1.next(), new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), str);
} }
assertFalse(iterator1.hasNext()); 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++) { for (int i = 5; i <= 7; i++) {
final String str = array[i]; final String str = array[i];
verifyWindowedKeyValue(iterator2.next(), new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), str); verifyWindowedKeyValue(iterator2.next(), new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), str);
@ -336,7 +339,7 @@ public class CachingWindowStoreTest {
cachingStore.flush(); cachingStore.flush();
cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP); 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"); verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP, "b");
assertFalse(fetch.hasNext()); assertFalse(fetch.hasNext());
} }
@ -346,7 +349,7 @@ public class CachingWindowStoreTest {
cachingStore.put(bytesKey("1"), bytesValue("a"), DEFAULT_TIMESTAMP); cachingStore.put(bytesKey("1"), bytesValue("a"), DEFAULT_TIMESTAMP);
cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE); 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, "a");
verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP + WINDOW_SIZE, "b"); verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP + WINDOW_SIZE, "b");
assertFalse(fetch.hasNext()); assertFalse(fetch.hasNext());
@ -357,7 +360,7 @@ public class CachingWindowStoreTest {
final Bytes key = Bytes.wrap("1".getBytes()); final Bytes key = Bytes.wrap("1".getBytes());
underlying.put(WindowKeySchema.toStoreKeyBinary(key, DEFAULT_TIMESTAMP, 0), "a".getBytes()); underlying.put(WindowKeySchema.toStoreKeyBinary(key, DEFAULT_TIMESTAMP, 0), "a".getBytes());
cachingStore.put(key, bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE); 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, "a");
verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP + WINDOW_SIZE, "b"); verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP + WINDOW_SIZE, "b");
assertFalse(fetch.hasNext()); assertFalse(fetch.hasNext());
@ -370,7 +373,7 @@ public class CachingWindowStoreTest {
cachingStore.put(key, bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE); cachingStore.put(key, bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE);
final KeyValueIterator<Windowed<Bytes>, byte[]> fetchRange = 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, DEFAULT_TIMESTAMP + WINDOW_SIZE)), "a");
verifyWindowedKeyValue(fetchRange.next(), new Windowed<>(key, new TimeWindow(DEFAULT_TIMESTAMP + WINDOW_SIZE, DEFAULT_TIMESTAMP + WINDOW_SIZE + WINDOW_SIZE)), "b"); verifyWindowedKeyValue(fetchRange.next(), new Windowed<>(key, new TimeWindow(DEFAULT_TIMESTAMP + WINDOW_SIZE, DEFAULT_TIMESTAMP + WINDOW_SIZE + WINDOW_SIZE)), "b");
assertFalse(fetchRange.hasNext()); assertFalse(fetchRange.hasNext());
@ -387,13 +390,13 @@ public class CachingWindowStoreTest {
@Test(expected = InvalidStateStoreException.class) @Test(expected = InvalidStateStoreException.class)
public void shouldThrowIfTryingToFetchFromClosedCachingStore() { public void shouldThrowIfTryingToFetchFromClosedCachingStore() {
cachingStore.close(); cachingStore.close();
cachingStore.fetch(bytesKey("a"), 0, 10); cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(10));
} }
@Test(expected = InvalidStateStoreException.class) @Test(expected = InvalidStateStoreException.class)
public void shouldThrowIfTryingToFetchRangeFromClosedCachingStore() { public void shouldThrowIfTryingToFetchRangeFromClosedCachingStore() {
cachingStore.close(); cachingStore.close();
cachingStore.fetch(bytesKey("a"), bytesKey("b"), 0, 10); cachingStore.fetch(bytesKey("a"), bytesKey("b"), ofEpochMilli(0), ofEpochMilli(10));
} }
@Test(expected = InvalidStateStoreException.class) @Test(expected = InvalidStateStoreException.class)
@ -415,7 +418,7 @@ public class CachingWindowStoreTest {
KeyValue.pair(1L, bytesValue("0003")), KeyValue.pair(1L, bytesValue("0003")),
KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")) 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); verifyKeyValueList(expected, actual);
} }
@ -433,12 +436,12 @@ public class CachingWindowStoreTest {
windowedPair("a", "0003", 1), windowedPair("a", "0003", 1),
windowedPair("a", "0005", SEGMENT_INTERVAL) windowedPair("a", "0005", SEGMENT_INTERVAL)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("a"), 0, Long.MAX_VALUE)) toList(cachingStore.fetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
); );
verifyKeyValueList( verifyKeyValueList(
mkList(windowedPair("aa", "0002", 0), windowedPair("aa", "0004", 1)), 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( verifyKeyValueList(
@ -449,7 +452,7 @@ public class CachingWindowStoreTest {
windowedPair("aa", "0004", 1), windowedPair("aa", "0004", 1),
windowedPair("a", "0005", SEGMENT_INTERVAL) windowedPair("a", "0005", SEGMENT_INTERVAL)
), ),
toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), 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) @Test(expected = NullPointerException.class)
public void shouldThrowNullPointerExceptionOnFetchNullKey() { public void shouldThrowNullPointerExceptionOnFetchNullKey() {
cachingStore.fetch(null, 1L, 2L); cachingStore.fetch(null, ofEpochMilli(1L), ofEpochMilli(2L));
} }
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldThrowNullPointerExceptionOnRangeNullFromKey() { public void shouldThrowNullPointerExceptionOnRangeNullFromKey() {
cachingStore.fetch(null, bytesKey("anyTo"), 1L, 2L); cachingStore.fetch(null, bytesKey("anyTo"), ofEpochMilli(1L), ofEpochMilli(2L));
} }
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldThrowNullPointerExceptionOnRangeNullToKey() { 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) { private static KeyValue<Windowed<Bytes>, byte[]> windowedPair(final String key, final String value, final long timestamp) {

View File

@ -36,6 +36,7 @@ import org.junit.runner.RunWith;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import static java.time.Instant.ofEpochMilli;
import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertArrayEquals;
@RunWith(EasyMockRunner.class) @RunWith(EasyMockRunner.class)
@ -101,7 +102,7 @@ public class ChangeLoggingWindowBytesStoreTest {
init(); init();
store.fetch(bytesKey, 0, 10); store.fetch(bytesKey, ofEpochMilli(0), ofEpochMilli(10));
EasyMock.verify(inner); EasyMock.verify(inner);
} }
@ -111,7 +112,7 @@ public class ChangeLoggingWindowBytesStoreTest {
init(); init();
store.fetch(bytesKey, bytesKey, 0, 1); store.fetch(bytesKey, bytesKey, ofEpochMilli(0), ofEpochMilli(1));
EasyMock.verify(inner); EasyMock.verify(inner);
} }

View File

@ -35,6 +35,7 @@ import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.NoSuchElementException; import java.util.NoSuchElementException;
import static java.time.Instant.ofEpochMilli;
import static java.util.Arrays.asList; import static java.util.Arrays.asList;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.core.IsEqual.equalTo; import static org.hamcrest.core.IsEqual.equalTo;
@ -77,7 +78,7 @@ public class CompositeReadOnlyWindowStoreTest {
underlyingWindowStore.put("my-key", "my-value", 0L); underlyingWindowStore.put("my-key", "my-value", 0L);
underlyingWindowStore.put("my-key", "my-later-value", 10L); 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); final List<KeyValue<Long, String>> results = StreamsTestUtils.toList(iterator);
assertEquals(asList(new KeyValue<>(0L, "my-value"), assertEquals(asList(new KeyValue<>(0L, "my-value"),
@ -87,7 +88,7 @@ public class CompositeReadOnlyWindowStoreTest {
@Test @Test
public void shouldReturnEmptyIteratorIfNoData() { 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()); assertEquals(false, iterator.hasNext());
} }
@ -100,10 +101,10 @@ public class CompositeReadOnlyWindowStoreTest {
underlyingWindowStore.put("key-one", "value-one", 0L); underlyingWindowStore.put("key-one", "value-one", 0L);
secondUnderlying.put("key-two", "value-two", 10L); secondUnderlying.put("key-two", "value-two", 10L);
final List<KeyValue<Long, String>> keyOneResults = StreamsTestUtils.toList(windowStore.fetch("key-one", 0L, final List<KeyValue<Long, String>> keyOneResults = StreamsTestUtils.toList(windowStore.fetch("key-one", ofEpochMilli(0L),
1L)); ofEpochMilli(1L)));
final List<KeyValue<Long, String>> keyTwoResults = StreamsTestUtils.toList(windowStore.fetch("key-two", 10L, final List<KeyValue<Long, String>> keyTwoResults = StreamsTestUtils.toList(windowStore.fetch("key-two", ofEpochMilli(10L),
11L)); ofEpochMilli(11L)));
assertEquals(Collections.singletonList(KeyValue.pair(0L, "value-one")), keyOneResults); assertEquals(Collections.singletonList(KeyValue.pair(0L, "value-one")), keyOneResults);
assertEquals(Collections.singletonList(KeyValue.pair(10L, "value-two")), keyTwoResults); assertEquals(Collections.singletonList(KeyValue.pair(10L, "value-two")), keyTwoResults);
@ -114,14 +115,14 @@ public class CompositeReadOnlyWindowStoreTest {
otherUnderlyingStore.put("some-key", "some-value", 0L); otherUnderlyingStore.put("some-key", "some-value", 0L);
underlyingWindowStore.put("some-key", "my-value", 1L); 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); assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results);
} }
@Test(expected = InvalidStateStoreException.class) @Test(expected = InvalidStateStoreException.class)
public void shouldThrowInvalidStateStoreExceptionOnRebalance() { public void shouldThrowInvalidStateStoreExceptionOnRebalance() {
final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new StateStoreProviderStub(true), QueryableStoreTypes.windowStore(), "foo"); 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 @Test
@ -130,7 +131,7 @@ public class CompositeReadOnlyWindowStoreTest {
final CompositeReadOnlyWindowStore<Object, Object> store = final CompositeReadOnlyWindowStore<Object, Object> store =
new CompositeReadOnlyWindowStore<>(stubProviderOne, QueryableStoreTypes.windowStore(), "window-store"); new CompositeReadOnlyWindowStore<>(stubProviderOne, QueryableStoreTypes.windowStore(), "window-store");
try { try {
store.fetch("key", 1, 10); store.fetch("key", ofEpochMilli(1), ofEpochMilli(10));
Assert.fail("InvalidStateStoreException was expected"); Assert.fail("InvalidStateStoreException was expected");
} catch (final InvalidStateStoreException e) { } catch (final InvalidStateStoreException e) {
Assert.assertEquals("State store is not available anymore and may have been migrated to another instance; " + 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() { public void emptyIteratorAlwaysReturnsFalse() {
final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new
StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo"); 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()); Assert.assertFalse(windowStoreIterator.hasNext());
} }
@ -151,7 +152,7 @@ public class CompositeReadOnlyWindowStoreTest {
public void emptyIteratorPeekNextKeyShouldThrowNoSuchElementException() { public void emptyIteratorPeekNextKeyShouldThrowNoSuchElementException() {
final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new
StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo"); 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); windowStoreIteratorException.expect(NoSuchElementException.class);
windowStoreIterator.peekNextKey(); windowStoreIterator.peekNextKey();
@ -161,7 +162,7 @@ public class CompositeReadOnlyWindowStoreTest {
public void emptyIteratorNextShouldThrowNoSuchElementException() { public void emptyIteratorNextShouldThrowNoSuchElementException() {
final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new
StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo"); 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); windowStoreIteratorException.expect(NoSuchElementException.class);
windowStoreIterator.next(); windowStoreIterator.next();
@ -173,7 +174,7 @@ public class CompositeReadOnlyWindowStoreTest {
stubProviderTwo.addStore(storeName, secondUnderlying); stubProviderTwo.addStore(storeName, secondUnderlying);
underlyingWindowStore.put("a", "a", 0L); underlyingWindowStore.put("a", "a", 0L);
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.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( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"),
KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
@ -212,7 +213,7 @@ public class CompositeReadOnlyWindowStoreTest {
stubProviderTwo.addStore(storeName, secondUnderlying); stubProviderTwo.addStore(storeName, secondUnderlying);
underlyingWindowStore.put("a", "a", 0L); underlyingWindowStore.put("a", "a", 0L);
secondUnderlying.put("b", "b", 10L); secondUnderlying.put("b", "b", 10L);
final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.fetchAll(0, 10)); final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(0), ofEpochMilli(10)));
assertThat(results, equalTo(Arrays.asList( assertThat(results, equalTo(Arrays.asList(
KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"),
KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
@ -220,17 +221,17 @@ public class CompositeReadOnlyWindowStoreTest {
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldThrowNPEIfKeyIsNull() { public void shouldThrowNPEIfKeyIsNull() {
windowStore.fetch(null, 0, 0); windowStore.fetch(null, ofEpochMilli(0), ofEpochMilli(0));
} }
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldThrowNPEIfFromKeyIsNull() { public void shouldThrowNPEIfFromKeyIsNull() {
windowStore.fetch(null, "a", 0, 0); windowStore.fetch(null, "a", ofEpochMilli(0), ofEpochMilli(0));
} }
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldThrowNPEIfToKeyIsNull() { public void shouldThrowNPEIfToKeyIsNull() {
windowStore.fetch("a", null, 0, 0); windowStore.fetch("a", null, ofEpochMilli(0), ofEpochMilli(0));
} }
} }

View File

@ -40,6 +40,7 @@ import org.junit.Test;
import java.util.Map; import java.util.Map;
import static java.time.Instant.ofEpochMilli;
import static java.util.Collections.singletonMap; import static java.util.Collections.singletonMap;
import static org.apache.kafka.test.StreamsTestUtils.getMetricByNameFilterByTags; import static org.apache.kafka.test.StreamsTestUtils.getMetricByNameFilterByTags;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -114,7 +115,7 @@ public class MeteredWindowStoreTest {
EasyMock.replay(innerStoreMock); EasyMock.replay(innerStoreMock);
store.init(context, store); 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(); 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", "all")).metricValue());
assertEquals(1.0, getMetricByNameFilterByTags(metrics, "fetch-total", "stream-scope-metrics", singletonMap("scope-id", "mocked-store")).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); EasyMock.replay(innerStoreMock);
store.init(context, store); 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(); 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", "all")).metricValue());
assertEquals(1.0, getMetricByNameFilterByTags(metrics, "fetch-total", "stream-scope-metrics", singletonMap("scope-id", "mocked-store")).metricValue()); assertEquals(1.0, getMetricByNameFilterByTags(metrics, "fetch-total", "stream-scope-metrics", singletonMap("scope-id", "mocked-store")).metricValue());

View File

@ -16,6 +16,8 @@
*/ */
package org.apache.kafka.streams.state.internals; 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.KeyValue;
import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.errors.InvalidStateStoreException;
import org.apache.kafka.streams.kstream.Windowed; 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()); 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 @Override
public KeyValueIterator<Windowed<K>, V> all() { public KeyValueIterator<Windowed<K>, V> all() {
if (!open) { 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 @Override
public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) {
if (!open) { 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) { public void put(final K key, final V value, final long timestamp) {
if (!data.containsKey(timestamp)) { if (!data.containsKey(timestamp)) {
data.put(timestamp, new TreeMap<K, V>()); data.put(timestamp, new TreeMap<K, V>());

View File

@ -54,6 +54,7 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import static java.time.Instant.ofEpochMilli;
import static java.util.Objects.requireNonNull; import static java.util.Objects.requireNonNull;
import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
@ -142,7 +143,7 @@ public class RocksDBWindowStoreTest {
setCurrentTime(currentTime); setCurrentTime(currentTime);
windowStore.put(1, "three"); 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 // roll to the next segment that will close the first
currentTime = currentTime + segmentInterval; currentTime = currentTime + segmentInterval;
@ -177,12 +178,12 @@ public class RocksDBWindowStoreTest {
assertEquals("four", windowStore.fetch(4, startTime + 4L)); assertEquals("four", windowStore.fetch(4, startTime + 4L));
assertEquals("five", windowStore.fetch(5, startTime + 5L)); assertEquals("five", windowStore.fetch(5, startTime + 5L));
assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime + 0L - windowSize, startTime + 0L + 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, startTime + 1L - windowSize, startTime + 1L + 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, startTime + 2L - windowSize, startTime + 2L + windowSize))); assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + 3L - windowSize, startTime + 3L + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L + windowSize))));
assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + 4L - windowSize, startTime + 4L + 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, startTime + 5L - windowSize, startTime + 5L + windowSize))); assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L + windowSize))));
putSecondBatch(windowStore, startTime, context); putSecondBatch(windowStore, startTime, context);
@ -193,21 +194,21 @@ public class RocksDBWindowStoreTest {
assertEquals("two+5", windowStore.fetch(2, startTime + 7L)); assertEquals("two+5", windowStore.fetch(2, startTime + 7L));
assertEquals("two+6", windowStore.fetch(2, startTime + 8L)); assertEquals("two+6", windowStore.fetch(2, startTime + 8L));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime - 2L - windowSize, startTime - 2L + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 2L - windowSize), ofEpochMilli(startTime - 2L + windowSize))));
assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime - 1L - windowSize, startTime - 1L + 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, startTime - windowSize, startTime + 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, startTime + 1L - windowSize, startTime + 1L + 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, startTime + 2L - windowSize, startTime + 2L + 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, startTime + 3L - windowSize, startTime + 3L + 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, startTime + 4L - windowSize, startTime + 4L + 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, startTime + 5L - windowSize, startTime + 5L + 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, startTime + 6L - windowSize, startTime + 6L + 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, startTime + 7L - windowSize, startTime + 7L + 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, startTime + 8L - windowSize, startTime + 8L + 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, startTime + 9L - windowSize, startTime + 9L + 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, startTime + 10L - windowSize, startTime + 10L + 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, startTime + 11L - windowSize, startTime + 11L + 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, startTime + 12L - windowSize, startTime + 12L + 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 ... // Flush the store and verify all current entries were properly flushed ...
windowStore.flush(); windowStore.flush();
@ -257,17 +258,17 @@ public class RocksDBWindowStoreTest {
assertEquals( assertEquals(
Utils.mkList(one, two, four), Utils.mkList(one, two, four),
StreamsTestUtils.toList(windowStore.fetchAll(startTime + 1, startTime + 4)) StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4)))
); );
assertEquals( assertEquals(
Utils.mkList(zero, one, two), Utils.mkList(zero, one, two),
StreamsTestUtils.toList(windowStore.fetchAll(startTime + 0, startTime + 3)) StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3)))
); );
assertEquals( assertEquals(
Utils.mkList(one, two, four, five), 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( assertEquals(
Utils.mkList(zero, one), 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( assertEquals(
Utils.mkList(one), 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( assertEquals(
Utils.mkList(one, two), 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( assertEquals(
Utils.mkList(zero, one, two), 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( assertEquals(
Utils.mkList(zero, one, two, Utils.mkList(zero, one, two,
four, five), 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( assertEquals(
Utils.mkList(two, four, five), 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( assertEquals(
Utils.mkList(), 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( assertEquals(
Utils.mkList(), 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); putFirstBatch(windowStore, startTime, context);
assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime + 0L - windowSize, startTime + 0L))); assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L))));
assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + 1L - windowSize, startTime + 1L))); assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L))));
assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L))); assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + 3L - windowSize, startTime + 3L))); assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L))));
assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + 4L - windowSize, startTime + 4L))); assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L))));
assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + 5L - windowSize, startTime + 5L))); assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L))));
putSecondBatch(windowStore, startTime, context); putSecondBatch(windowStore, startTime, context);
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime - 1L - windowSize, startTime - 1L))); assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 1L - windowSize), ofEpochMilli(startTime - 1L))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 0L - windowSize, startTime + 0L))); assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 1L - windowSize, startTime + 1L))); assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L))));
assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L))); 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, startTime + 3L - windowSize, startTime + 3L))); 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, startTime + 4L - windowSize, startTime + 4L))); 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, startTime + 5L - windowSize, startTime + 5L))); 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, startTime + 6L - windowSize, startTime + 6L))); 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, startTime + 7L - windowSize, startTime + 7L))); 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, startTime + 8L - windowSize, startTime + 8L))); 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, startTime + 9L - windowSize, startTime + 9L))); 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, startTime + 10L - windowSize, startTime + 10L))); 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, startTime + 11L - windowSize, startTime + 11L))); assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L - windowSize), ofEpochMilli(startTime + 11L))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 12L - windowSize, startTime + 12L))); assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 12L - windowSize), ofEpochMilli(startTime + 12L))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 13L - windowSize, startTime + 13L))); 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 ... // Flush the store and verify all current entries were properly flushed ...
windowStore.flush(); windowStore.flush();
@ -372,30 +373,30 @@ public class RocksDBWindowStoreTest {
putFirstBatch(windowStore, startTime, context); putFirstBatch(windowStore, startTime, context);
assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime + 0L, startTime + 0L + windowSize))); assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L), ofEpochMilli(startTime + 0L + windowSize))));
assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + 1L, startTime + 1L + windowSize))); assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L), ofEpochMilli(startTime + 1L + windowSize))));
assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L, startTime + 2L + windowSize))); assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + 2L + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + 3L, startTime + 3L + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L), ofEpochMilli(startTime + 3L + windowSize))));
assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + 4L, startTime + 4L + windowSize))); assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L), ofEpochMilli(startTime + 4L + windowSize))));
assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + 5L, startTime + 5L + windowSize))); assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L), ofEpochMilli(startTime + 5L + windowSize))));
putSecondBatch(windowStore, startTime, context); putSecondBatch(windowStore, startTime, context);
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime - 2L, startTime - 2L + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 2L), ofEpochMilli(startTime - 2L + windowSize))));
assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime - 1L, startTime - 1L + 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, startTime, startTime + 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, startTime + 1L, startTime + 1L + 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, startTime + 2L, startTime + 2L + 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, startTime + 3L, startTime + 3L + 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, startTime + 4L, startTime + 4L + 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, startTime + 5L, startTime + 5L + 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, startTime + 6L, startTime + 6L + 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, startTime + 7L, startTime + 7L + 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, startTime + 8L, startTime + 8L + windowSize))); assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 8L), ofEpochMilli(startTime + 8L + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 9L, startTime + 9L + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 9L), ofEpochMilli(startTime + 9L + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 10L, startTime + 10L + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 10L), ofEpochMilli(startTime + 10L + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 11L, startTime + 11L + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L), ofEpochMilli(startTime + 11L + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 12L, startTime + 12L + 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 ... // Flush the store and verify all current entries were properly flushed ...
windowStore.flush(); windowStore.flush();
@ -419,17 +420,17 @@ public class RocksDBWindowStoreTest {
setCurrentTime(startTime); setCurrentTime(startTime);
windowStore.put(0, "zero"); 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+"); 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, ofEpochMilli(startTime - windowSize), ofEpochMilli(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, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(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, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize))));
assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, startTime + 3L - windowSize, startTime + 3L + 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, startTime + 4L - windowSize, startTime + 4L + 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 ... // Flush the store and verify all current entries were properly flushed ...
windowStore.flush(); windowStore.flush();
@ -487,12 +488,12 @@ public class RocksDBWindowStoreTest {
segmentDirs(baseDir) segmentDirs(baseDir)
); );
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))));
assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + 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, startTime + increment * 2 - windowSize, startTime + increment * 2 + 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, startTime + increment * 3 - windowSize, startTime + increment * 3 + 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, startTime + increment * 4 - windowSize, startTime + increment * 4 + 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, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize))); assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize))));
setCurrentTime(startTime + increment * 6); setCurrentTime(startTime + increment * 6);
windowStore.put(6, "six"); 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(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize))));
assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + 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, startTime + increment * 3 - windowSize, startTime + increment * 3 + 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, startTime + increment * 4 - windowSize, startTime + increment * 4 + 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, startTime + increment * 5 - windowSize, startTime + increment * 5 + 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, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize))); assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize))));
setCurrentTime(startTime + increment * 7); setCurrentTime(startTime + increment * 7);
@ -526,14 +527,14 @@ public class RocksDBWindowStoreTest {
segmentDirs(baseDir) segmentDirs(baseDir)
); );
assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize))));
assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + 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, startTime + increment * 3 - windowSize, startTime + increment * 3 + 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, startTime + increment * 4 - windowSize, startTime + increment * 4 + 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, startTime + increment * 5 - windowSize, startTime + increment * 5 + 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, startTime + increment * 6 - windowSize, startTime + increment * 6 + 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, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize))); assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize))));
setCurrentTime(startTime + increment * 8); setCurrentTime(startTime + increment * 8);
windowStore.put(8, "eight"); 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(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + 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, startTime + increment * 3 - windowSize, startTime + increment * 3 + 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, startTime + increment * 4 - windowSize, startTime + increment * 4 + 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, startTime + increment * 5 - windowSize, startTime + increment * 5 + 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, startTime + increment * 6 - windowSize, startTime + increment * 6 + 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, startTime + increment * 7 - windowSize, startTime + increment * 7 + 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, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize))); assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize))));
// check segment directories // check segment directories
windowStore.flush(); windowStore.flush();
@ -603,27 +604,27 @@ public class RocksDBWindowStoreTest {
Utils.delete(baseDir); Utils.delete(baseDir);
windowStore = createWindowStore(context, false); windowStore = createWindowStore(context, false);
assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + 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, startTime + increment * 3 - windowSize, startTime + increment * 3 + 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, startTime + increment * 4 - windowSize, startTime + increment * 4 + 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, startTime + increment * 5 - windowSize, startTime + increment * 5 + 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, startTime + increment * 6 - windowSize, startTime + increment * 6 + 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, startTime + increment * 7 - windowSize, startTime + increment * 7 + 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, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize))));
context.restore(windowName, changeLog); context.restore(windowName, changeLog);
assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize))); assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize))));
assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + 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, startTime + increment * 3 - windowSize, startTime + increment * 3 + 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, startTime + increment * 4 - windowSize, startTime + increment * 4 + 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, startTime + increment * 5 - windowSize, startTime + increment * 5 + 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, startTime + increment * 6 - windowSize, startTime + increment * 6 + 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, startTime + increment * 7 - windowSize, startTime + increment * 7 + 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, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize))); assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize))));
// check segment directories // check segment directories
windowStore.flush(); windowStore.flush();
@ -662,7 +663,7 @@ public class RocksDBWindowStoreTest {
WindowStoreIterator iter; WindowStoreIterator iter;
int fetchedCount; int fetchedCount;
iter = windowStore.fetch(0, 0L, segmentInterval * 4); iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(segmentInterval * 4));
fetchedCount = 0; fetchedCount = 0;
while (iter.hasNext()) { while (iter.hasNext()) {
iter.next(); iter.next();
@ -678,7 +679,7 @@ public class RocksDBWindowStoreTest {
setCurrentTime(segmentInterval * 3); setCurrentTime(segmentInterval * 3);
windowStore.put(0, "v"); windowStore.put(0, "v");
iter = windowStore.fetch(0, 0L, segmentInterval * 4); iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(segmentInterval * 4));
fetchedCount = 0; fetchedCount = 0;
while (iter.hasNext()) { while (iter.hasNext()) {
iter.next(); iter.next();
@ -694,7 +695,7 @@ public class RocksDBWindowStoreTest {
setCurrentTime(segmentInterval * 5); setCurrentTime(segmentInterval * 5);
windowStore.put(0, "v"); windowStore.put(0, "v");
iter = windowStore.fetch(0, segmentInterval * 4, segmentInterval * 10); iter = windowStore.fetch(0, ofEpochMilli(segmentInterval * 4), ofEpochMilli(segmentInterval * 10));
fetchedCount = 0; fetchedCount = 0;
while (iter.hasNext()) { while (iter.hasNext()) {
iter.next(); iter.next();
@ -736,7 +737,7 @@ public class RocksDBWindowStoreTest {
assertEquals(expected, actual); 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()) { while (iter.hasNext()) {
iter.next(); iter.next();
} }
@ -756,7 +757,7 @@ public class RocksDBWindowStoreTest {
windowStore.put(1, "two", 2L); windowStore.put(1, "two", 2L);
windowStore.put(1, "three", 3L); 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()); assertTrue(iterator.hasNext());
windowStore.close(); windowStore.close();
@ -784,16 +785,17 @@ public class RocksDBWindowStoreTest {
final List expected = Utils.mkList("0001", "0003", "0005"); 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( assertThat(list, equalTo(Utils.mkList(
windowedPair("a", "0001", 0, windowSize), windowedPair("a", "0001", 0, windowSize),
windowedPair("a", "0003", 1, windowSize), windowedPair("a", "0003", 1, windowSize),
windowedPair("a", "0005", 0x7a00000000000000L - 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( assertThat(list, equalTo(Utils.mkList(
windowedPair("aa", "0002", 0, windowSize), windowedPair("aa", "0002", 0, windowSize),
windowedPair("aa", "0004", 1, windowSize) windowedPair("aa", "0004", 1, windowSize)
@ -815,19 +817,19 @@ public class RocksDBWindowStoreTest {
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldThrowNullPointerExceptionOnGetNullKey() { public void shouldThrowNullPointerExceptionOnGetNullKey() {
windowStore = createWindowStore(context, false); windowStore = createWindowStore(context, false);
windowStore.fetch(null, 1L, 2L); windowStore.fetch(null, ofEpochMilli(1L), ofEpochMilli(2L));
} }
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldThrowNullPointerExceptionOnRangeNullFromKey() { public void shouldThrowNullPointerExceptionOnRangeNullFromKey() {
windowStore = createWindowStore(context, false); windowStore = createWindowStore(context, false);
windowStore.fetch(null, 2, 1L, 2L); windowStore.fetch(null, 2, ofEpochMilli(1L), ofEpochMilli(2L));
} }
@Test(expected = NullPointerException.class) @Test(expected = NullPointerException.class)
public void shouldThrowNullPointerExceptionOnRangeNullToKey() { public void shouldThrowNullPointerExceptionOnRangeNullToKey() {
windowStore = createWindowStore(context, false); windowStore = createWindowStore(context, false);
windowStore.fetch(1, null, 1L, 2L); windowStore.fetch(1, null, ofEpochMilli(1L), ofEpochMilli(2L));
} }
@Test @Test
@ -866,11 +868,11 @@ public class RocksDBWindowStoreTest {
windowStore.put(key3, "9", 59999); windowStore.put(key3, "9", 59999);
final List expectedKey1 = Utils.mkList("1", "4", "7"); 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"); 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"); 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, private void putFirstBatch(final WindowStore<Integer, String> store,

View File

@ -78,7 +78,7 @@ public class StreamThreadStateStoreProviderTest {
topology.addStateStore(Stores.keyValueStoreBuilder(Stores.inMemoryKeyValueStore("kv-store"), Serdes.String(), Serdes.String()), "the-processor"); topology.addStateStore(Stores.keyValueStoreBuilder(Stores.inMemoryKeyValueStore("kv-store"), Serdes.String(), Serdes.String()), "the-processor");
topology.addStateStore( topology.addStateStore(
Stores.windowStoreBuilder( Stores.windowStoreBuilder(
Stores.persistentWindowStore("window-store", 10L, 2L, false), Stores.persistentWindowStore("window-store", Duration.ofMillis(10L), Duration.ofMillis(2L), false),
Serdes.String(), Serdes.String(),
Serdes.String()), Serdes.String()),
"the-processor" "the-processor"

View File

@ -41,7 +41,6 @@ import java.time.Duration;
import java.util.Collections; import java.util.Collections;
import java.util.Locale; import java.util.Locale;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.TimeUnit;
public class BrokerCompatibilityTest { public class BrokerCompatibilityTest {
@ -108,7 +107,7 @@ public class BrokerCompatibilityTest {
System.err.println("FATAL: An unexpected exception " + cause); System.err.println("FATAL: An unexpected exception " + cause);
e.printStackTrace(System.err); e.printStackTrace(System.err);
System.err.flush(); System.err.flush();
streams.close(30, TimeUnit.SECONDS); streams.close(Duration.ofSeconds(30));
} }
}); });
System.out.println("start Kafka Streams"); System.out.println("start Kafka Streams");

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.tests; package org.apache.kafka.streams.tests;
import java.time.Duration;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KafkaStreams;
@ -56,7 +57,7 @@ public class EosTestClient extends SmokeTestUtil {
@Override @Override
public void run() { public void run() {
isRunning = false; isRunning = false;
streams.close(TimeUnit.SECONDS.toMillis(300), TimeUnit.SECONDS); streams.close(Duration.ofSeconds(300));
// need to wait for callback to avoid race condition // need to wait for callback to avoid race condition
// -> make sure the callback printout to stdout is there as it is expected test output // -> 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(); streams.start();
} }
if (uncaughtException) { if (uncaughtException) {
streams.close(TimeUnit.SECONDS.toMillis(60), TimeUnit.SECONDS); streams.close(Duration.ofSeconds(60_000L));
streams = null; streams = null;
} }
sleep(1000); sleep(1000);

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.tests; package org.apache.kafka.streams.tests;
import java.time.Duration;
import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord; 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 org.apache.kafka.streams.kstream.KStream;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.TimeUnit;
public class ShutdownDeadlockTest { public class ShutdownDeadlockTest {
@ -65,7 +65,7 @@ public class ShutdownDeadlockTest {
Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
@Override @Override
public void run() { public void run() {
streams.close(5, TimeUnit.SECONDS); streams.close(Duration.ofSeconds(5));
} }
})); }));

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams.tests; package org.apache.kafka.streams.tests;
import java.time.Duration;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.serialization.Serdes; 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 org.apache.kafka.streams.state.WindowStore;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.TimeUnit;
public class SmokeTestClient extends SmokeTestUtil { public class SmokeTestClient extends SmokeTestUtil {
@ -82,7 +82,7 @@ public class SmokeTestClient extends SmokeTestUtil {
} }
public void close() { 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 // do not remove these printouts since they are needed for health scripts
if (!uncaughtException) { if (!uncaughtException) {
System.out.println("SMOKE-TEST-CLIENT-CLOSED"); System.out.println("SMOKE-TEST-CLIENT-CLOSED");
@ -129,7 +129,7 @@ public class SmokeTestClient extends SmokeTestUtil {
data.groupByKey(Serialized.with(stringSerde, intSerde)); data.groupByKey(Serialized.with(stringSerde, intSerde));
groupedData groupedData
.windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(1))) .windowedBy(TimeWindows.of(Duration.ofDays(1)))
.aggregate( .aggregate(
new Initializer<Integer>() { new Initializer<Integer>() {
public Integer apply() { public Integer apply() {
@ -154,7 +154,7 @@ public class SmokeTestClient extends SmokeTestUtil {
// max // max
groupedData groupedData
.windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2))) .windowedBy(TimeWindows.of(Duration.ofDays(2)))
.aggregate( .aggregate(
new Initializer<Integer>() { new Initializer<Integer>() {
public Integer apply() { public Integer apply() {
@ -179,7 +179,7 @@ public class SmokeTestClient extends SmokeTestUtil {
// sum // sum
groupedData groupedData
.windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2))) .windowedBy(TimeWindows.of(Duration.ofDays(2)))
.aggregate( .aggregate(
new Initializer<Long>() { new Initializer<Long>() {
public Long apply() { public Long apply() {
@ -202,7 +202,7 @@ public class SmokeTestClient extends SmokeTestUtil {
// cnt // cnt
groupedData groupedData
.windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2))) .windowedBy(TimeWindows.of(Duration.ofDays(2)))
.count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("uwin-cnt")) .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("uwin-cnt"))
.toStream(new Unwindow<String, Long>()) .toStream(new Unwindow<String, Long>())
.to("cnt", Produced.with(stringSerde, longSerde)); .to("cnt", Produced.with(stringSerde, longSerde));
@ -252,7 +252,7 @@ public class SmokeTestClient extends SmokeTestUtil {
@Override @Override
public void uncaughtException(final Thread t, final Throwable e) { public void uncaughtException(final Thread t, final Throwable e) {
System.out.println("FATAL: An unexpected exception is encountered on thread " + t + ": " + e); System.out.println("FATAL: An unexpected exception is encountered on thread " + t + ": " + e);
streamsClient.close(30, TimeUnit.SECONDS); streamsClient.close(Duration.ofSeconds(30));
} }
}); });

View File

@ -17,6 +17,7 @@
package org.apache.kafka.streams.tests; package org.apache.kafka.streams.tests;
import java.time.Duration;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serde;
@ -33,7 +34,6 @@ import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.TimeUnit;
public class StreamsBrokerDownResilienceTest { public class StreamsBrokerDownResilienceTest {
@ -109,7 +109,7 @@ public class StreamsBrokerDownResilienceTest {
public void uncaughtException(final Thread t, final Throwable e) { public void uncaughtException(final Thread t, final Throwable e) {
System.err.println("FATAL: An unexpected exception " + e); System.err.println("FATAL: An unexpected exception " + e);
System.err.flush(); System.err.flush();
streams.close(30, TimeUnit.SECONDS); streams.close(Duration.ofSeconds(30));
} }
}); });
System.out.println("Start Kafka Streams"); System.out.println("Start Kafka Streams");
@ -118,7 +118,7 @@ public class StreamsBrokerDownResilienceTest {
Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
@Override @Override
public void run() { 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.println("Complete shutdown of streams resilience test app now");
System.out.flush(); System.out.flush();
} }

View File

@ -17,6 +17,7 @@
package org.apache.kafka.streams.tests; package org.apache.kafka.streams.tests;
import java.time.Duration;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serde;
@ -38,7 +39,6 @@ import java.io.IOException;
import java.util.Map; import java.util.Map;
import java.util.Properties; import java.util.Properties;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit;
public class StreamsStandByReplicaTest { public class StreamsStandByReplicaTest {
@ -164,7 +164,7 @@ public class StreamsStandByReplicaTest {
} }
private static void shutdown(final KafkaStreams streams) { private static void shutdown(final KafkaStreams streams) {
streams.close(10, TimeUnit.SECONDS); streams.close(Duration.ofSeconds(10));
} }
private static boolean confirmCorrectConfigs(final Properties properties) { private static boolean confirmCorrectConfigs(final Properties properties) {

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.test; package org.apache.kafka.test;
import java.time.Duration;
import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.header.internals.RecordHeaders;
@ -209,6 +210,13 @@ public class InternalMockProcessorContext extends AbstractProcessorContext imple
throw new UnsupportedOperationException("schedule() not supported."); 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 @Override
public void commit() { } public void commit() { }

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.test; package org.apache.kafka.test;
import java.time.Duration;
import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.AbstractProcessor;
import org.apache.kafka.streams.processor.Cancellable; import org.apache.kafka.streams.processor.Cancellable;
import org.apache.kafka.streams.processor.ProcessorContext; 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) { public void init(final ProcessorContext context) {
super.init(context); super.init(context);
if (scheduleInterval > 0L) { if (scheduleInterval > 0L) {
scheduleCancellable = context.schedule(scheduleInterval, punctuationType, new Punctuator() { scheduleCancellable = context.schedule(Duration.ofMillis(scheduleInterval), punctuationType, new Punctuator() {
@Override @Override
public void punctuate(final long timestamp) { public void punctuate(final long timestamp) {
if (punctuationType == PunctuationType.STREAM_TIME) { if (punctuationType == PunctuationType.STREAM_TIME) {

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.test; package org.apache.kafka.test;
import java.time.Duration;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.processor.Cancellable; import org.apache.kafka.streams.processor.Cancellable;
@ -52,7 +53,15 @@ public class NoOpProcessorContext extends AbstractProcessorContext {
return null; 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; return null;
} }

View File

@ -18,6 +18,8 @@
*/ */
package org.apache.kafka.streams.scala.kstream package org.apache.kafka.streams.scala.kstream
import java.time.Duration.ofSeconds
import org.apache.kafka.streams.kstream.JoinWindows import org.apache.kafka.streams.kstream.JoinWindows
import org.apache.kafka.streams.scala.ImplicitConversions._ import org.apache.kafka.streams.scala.ImplicitConversions._
import org.apache.kafka.streams.scala.Serdes._ 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 stream1 = builder.stream[String, String](sourceTopic1)
val stream2 = builder.stream[String, String](sourceTopic2) 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) val testDriver = createTestDriver(builder)

View File

@ -18,6 +18,8 @@
*/ */
package org.apache.kafka.streams.scala.kstream package org.apache.kafka.streams.scala.kstream
import java.time.Duration
import org.apache.kafka.streams.kstream.internals.MaterializedInternal import org.apache.kafka.streams.kstream.internals.MaterializedInternal
import org.apache.kafka.streams.scala.Serdes._ import org.apache.kafka.streams.scala.Serdes._
import org.apache.kafka.streams.scala._ 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 { "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] = val materialized: Materialized[String, Long, ByteArrayWindowStore] =
Materialized.as[String, Long](storeSupplier) 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 { "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] = val materialized: Materialized[String, Long, ByteArraySessionStore] =
Materialized.as[String, Long](storeSupplier) Materialized.as[String, Long](storeSupplier)

View File

@ -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. * {@link ProducerRecord#equals(Object)} can simplify your code as you can ignore attributes you are not interested in.
* <p> * <p>
* Note, that calling {@code pipeInput()} will also trigger {@link PunctuationType#STREAM_TIME event-time} base * 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 * However, you won't trigger {@link PunctuationType#WALL_CLOCK_TIME wall-clock} type punctuations that you must
* trigger manually via {@link #advanceWallClockTime(long)}. * trigger manually via {@link #advanceWallClockTime(long)}.
* <p> * <p>
@ -489,7 +489,7 @@ public class TopologyTestDriver implements Closeable {
/** /**
* Advances the internally mocked wall-clock time. * Advances the internally mocked wall-clock time.
* This might trigger a {@link PunctuationType#WALL_CLOCK_TIME wall-clock} type * 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 * @param advanceMs the amount of time to advance wall-clock time in milliseconds
*/ */

View File

@ -16,10 +16,12 @@
*/ */
package org.apache.kafka.streams.processor; package org.apache.kafka.streams.processor;
import java.time.Duration;
import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.serialization.Serde; 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.KeyValue;
import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.StreamsMetrics; import org.apache.kafka.streams.StreamsMetrics;
@ -378,6 +380,7 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S
} }
@Override @Override
@Deprecated
public Cancellable schedule(final long intervalMs, final PunctuationType type, final Punctuator callback) { public Cancellable schedule(final long intervalMs, final PunctuationType type, final Punctuator callback) {
final CapturedPunctuator capturedPunctuator = new CapturedPunctuator(intervalMs, type, callback); final CapturedPunctuator capturedPunctuator = new CapturedPunctuator(intervalMs, type, callback);
@ -386,6 +389,14 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S
return capturedPunctuator::cancel; 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(...)}. * Get the punctuators scheduled so far. The returned list is not affected by subsequent calls to {@code schedule(...)}.
* *

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams; package org.apache.kafka.streams;
import java.time.Duration;
import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.AbstractProcessor;
import org.apache.kafka.streams.processor.MockProcessorContext; import org.apache.kafka.streams.processor.MockProcessorContext;
@ -345,7 +346,7 @@ public class MockProcessorContextTest {
@Override @Override
public void init(final ProcessorContext context) { public void init(final ProcessorContext context) {
context.schedule( context.schedule(
1000L, Duration.ofSeconds(1L),
PunctuationType.WALL_CLOCK_TIME, PunctuationType.WALL_CLOCK_TIME,
timestamp -> context.commit() timestamp -> context.commit()
); );

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.streams; package org.apache.kafka.streams;
import java.time.Duration;
import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
@ -231,7 +232,7 @@ public class TopologyTestDriverTest {
initialized = true; initialized = true;
this.context = context; this.context = context;
for (final Punctuation punctuation : punctuations) { 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 @Override
public void init(final ProcessorContext context) { public void init(final ProcessorContext context) {
this.context = context; this.context = context;
context.schedule(60000, PunctuationType.WALL_CLOCK_TIME, timestamp -> flushStore()); context.schedule(Duration.ofMinutes(1), PunctuationType.WALL_CLOCK_TIME, timestamp -> flushStore());
context.schedule(10000, PunctuationType.STREAM_TIME, timestamp -> flushStore()); context.schedule(Duration.ofSeconds(10), PunctuationType.STREAM_TIME, timestamp -> flushStore());
store = (KeyValueStore<String, Long>) context.getStateStore("aggStore"); store = (KeyValueStore<String, Long>) context.getStateStore("aggStore");
} }