kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mj...@apache.org
Subject [kafka] branch trunk updated: KAFKA-7277: Migrate Streams API to Duration instead of longMs times (#5682)
Date Thu, 04 Oct 2018 20:52:56 GMT
This is an automated email from the ASF dual-hosted git repository.

mjsax pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new ca641b3  KAFKA-7277: Migrate Streams API to Duration instead of longMs times (#5682)
ca641b3 is described below

commit ca641b3e2e48c14ff308181c775775408f5f35f7
Author: Nikolay <nizhikov@apache.org>
AuthorDate: Thu Oct 4 23:51:39 2018 +0300

    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>
---
 .../examples/pageview/PageViewTypedDemo.java       |   4 +-
 .../examples/pageview/PageViewUntypedDemo.java     |   3 +-
 .../examples/temperature/TemperatureDemo.java      |   4 +-
 .../examples/wordcount/WordCountProcessorDemo.java |   3 +-
 .../org/apache/kafka/streams/KafkaStreams.java     |  20 ++
 .../apache/kafka/streams/internals/ApiUtils.java   |  60 +++++
 .../apache/kafka/streams/kstream/JoinWindows.java  |  70 ++++-
 .../apache/kafka/streams/kstream/Materialized.java |   9 +-
 .../kafka/streams/kstream/SessionWindows.java      |  26 +-
 .../apache/kafka/streams/kstream/TimeWindows.java  |  53 +++-
 .../apache/kafka/streams/kstream/Transformer.java  |   5 +-
 .../kafka/streams/kstream/UnlimitedWindows.java    |  16 ++
 .../kafka/streams/kstream/ValueTransformer.java    |   5 +-
 .../streams/kstream/ValueTransformerWithKey.java   |   7 +-
 .../org/apache/kafka/streams/kstream/Window.java   |  25 ++
 .../streams/kstream/internals/KStreamImpl.java     |   5 +-
 .../kstream/internals/TimeWindowedKStreamImpl.java |   5 +-
 .../kafka/streams/processor/Cancellable.java       |   4 +-
 .../apache/kafka/streams/processor/Processor.java  |   3 +-
 .../kafka/streams/processor/ProcessorContext.java  |  47 +++-
 .../kafka/streams/processor/PunctuationType.java   |   4 +-
 .../apache/kafka/streams/processor/Punctuator.java |   4 +-
 .../ForwardingDisabledProcessorContext.java        |   9 +
 .../internals/GlobalProcessorContextImpl.java      |  10 +
 .../processor/internals/ProcessorContextImpl.java  |  11 +
 .../processor/internals/StandbyContextImpl.java    |  10 +
 .../kafka/streams/state/ReadOnlyWindowStore.java   |  83 +++++-
 .../org/apache/kafka/streams/state/Stores.java     |  44 ++++
 .../apache/kafka/streams/state/WindowStore.java    |  64 +++++
 .../kafka/streams/state/WindowStoreIterator.java   |   4 +-
 .../state/internals/CachingWindowStore.java        |  32 ++-
 .../internals/ChangeLoggingWindowBytesStore.java   |  28 +-
 .../internals/CompositeReadOnlyWindowStore.java    |  27 +-
 .../state/internals/MeteredWindowStore.java        |  23 ++
 .../state/internals/RocksDBWindowStore.java        |  23 ++
 .../org/apache/kafka/streams/KafkaStreamsTest.java |   3 +-
 .../org/apache/kafka/streams/TopologyTest.java     |  13 +-
 .../integration/AbstractResetIntegrationTest.java  |   6 +-
 .../integration/GlobalThreadShutDownOrderTest.java |   4 +-
 .../integration/InternalTopicIntegrationTest.java  |   6 +-
 .../KStreamAggregationDedupIntegrationTest.java    |   6 +-
 .../KStreamAggregationIntegrationTest.java         |  14 +-
 .../KTableSourceTopicRestartIntegrationTest.java   |   8 +-
 .../PurgeRepartitionTopicIntegrationTest.java      |   4 +-
 .../integration/QueryableStateIntegrationTest.java |  15 +-
 .../RepartitionOptimizingIntegrationTest.java      |   9 +-
 ...artitionWithMergeOptimizingIntegrationTest.java |   6 +-
 .../integration/RestoreIntegrationTest.java        |   3 +-
 .../StreamStreamJoinIntegrationTest.java           |  17 +-
 .../integration/SuppressionIntegrationTest.java    |   4 +-
 .../kafka/streams/kstream/JoinWindowsTest.java     |  87 ++++---
 .../kafka/streams/kstream/SessionWindowsTest.java  |  41 +--
 .../kafka/streams/kstream/TimeWindowsTest.java     |  77 +++---
 .../streams/kstream/UnlimitedWindowsTest.java      |  17 +-
 .../kstream/internals/KGroupedStreamImplTest.java  |  45 ++--
 .../streams/kstream/internals/KStreamImplTest.java |  17 +-
 .../kstream/internals/KStreamKStreamJoinTest.java  |  13 +-
 .../internals/KStreamKStreamLeftJoinTest.java      |   5 +-
 ...KStreamSessionWindowAggregateProcessorTest.java |   9 +-
 .../kstream/internals/KStreamTransformTest.java    |   3 +-
 .../internals/KStreamWindowAggregateTest.java      |  13 +-
 .../kstream/internals/KStreamWindowReduceTest.java |   5 +-
 .../internals/SessionWindowedKStreamImplTest.java  |   5 +-
 .../kstream/internals/SuppressScenarioTest.java    |   6 +-
 .../kstream/internals/SuppressTopologyTest.java    |   5 +-
 .../streams/kstream/internals/TimeWindowTest.java  |   5 +-
 .../internals/TimeWindowedKStreamImplTest.java     |  12 +-
 .../internals/graph/GraphGraceSearchUtilTest.java  |  13 +-
 .../kstream/internals/graph/StreamsGraphTest.java  |   7 +-
 .../apache/kafka/streams/perf/SimpleBenchmark.java |  17 +-
 .../apache/kafka/streams/perf/YahooBenchmark.java  |   3 +-
 .../internals/AbstractProcessorContextTest.java    |   8 +
 .../internals/InternalTopologyBuilderTest.java     |   5 +-
 .../processor/internals/StandbyTaskTest.java       |   9 +-
 .../processor/internals/StreamThreadTest.java      |   5 +-
 .../internals/StreamsPartitionAssignorTest.java    |   3 +-
 .../kafka/streams/state/NoOpWindowStore.java       |  24 +-
 .../org/apache/kafka/streams/state/StoresTest.java |  18 +-
 .../state/internals/CachingWindowStoreTest.java    |  43 +--
 .../ChangeLoggingWindowBytesStoreTest.java         |   7 +-
 .../CompositeReadOnlyWindowStoreTest.java          |  37 +--
 .../state/internals/MeteredWindowStoreTest.java    |   7 +-
 .../state/internals/ReadOnlyWindowStoreStub.java   |  25 ++
 .../state/internals/RocksDBWindowStoreTest.java    | 288 +++++++++++----------
 .../StreamThreadStateStoreProviderTest.java        |   2 +-
 .../streams/tests/BrokerCompatibilityTest.java     |   3 +-
 .../apache/kafka/streams/tests/EosTestClient.java  |   5 +-
 .../kafka/streams/tests/ShutdownDeadlockTest.java  |   4 +-
 .../kafka/streams/tests/SmokeTestClient.java       |  14 +-
 .../tests/StreamsBrokerDownResilienceTest.java     |   6 +-
 .../streams/tests/StreamsStandByReplicaTest.java   |   4 +-
 .../kafka/test/InternalMockProcessorContext.java   |   8 +
 .../java/org/apache/kafka/test/MockProcessor.java  |   3 +-
 .../apache/kafka/test/NoOpProcessorContext.java    |  11 +-
 .../kafka/streams/scala/kstream/KStreamTest.scala  |   4 +-
 .../streams/scala/kstream/MaterializedTest.scala   |   6 +-
 .../apache/kafka/streams/TopologyTestDriver.java   |   4 +-
 .../streams/processor/MockProcessorContext.java    |  11 +
 .../kafka/streams/MockProcessorContextTest.java    |   3 +-
 .../kafka/streams/TopologyTestDriverTest.java      |   7 +-
 100 files changed, 1311 insertions(+), 536 deletions(-)

diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java
index 871d836..f8af99f 100644
--- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java
+++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java
@@ -19,6 +19,7 @@ package org.apache.kafka.streams.examples.pageview;
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import java.time.Duration;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.errors.SerializationException;
 import org.apache.kafka.common.serialization.Deserializer;
@@ -39,7 +40,6 @@ import java.io.IOException;
 import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
 
 /**
  * Demonstrates how to perform a join between a KStream and a KTable, i.e. an example of a stateful computation,
@@ -207,7 +207,7 @@ public class PageViewTypedDemo {
             })
             .map((user, viewRegion) -> new KeyValue<>(viewRegion.region, viewRegion))
             .groupByKey(Serialized.with(Serdes.String(), new JSONSerde<>()))
-            .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(7)).advanceBy(TimeUnit.SECONDS.toMillis(1)))
+            .windowedBy(TimeWindows.of(Duration.ofDays(7)).advanceBy(Duration.ofSeconds(1)))
             .count()
             .toStream()
             .map((key, value) -> {
diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java
index e2b7c37..25b10e8 100644
--- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java
+++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java
@@ -19,6 +19,7 @@ package org.apache.kafka.streams.examples.pageview;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.node.JsonNodeFactory;
 import com.fasterxml.jackson.databind.node.ObjectNode;
+import java.time.Duration;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.Serde;
@@ -87,7 +88,7 @@ public class PageViewUntypedDemo {
             })
             .map((user, viewRegion) -> new KeyValue<>(viewRegion.get("region").textValue(), viewRegion))
             .groupByKey(Serialized.with(Serdes.String(), jsonSerde))
-            .windowedBy(TimeWindows.of(7 * 24 * 60 * 60 * 1000L).advanceBy(1000))
+            .windowedBy(TimeWindows.of(Duration.ofDays(7)).advanceBy(Duration.ofSeconds(1)))
             .count()
             .toStream()
             .map((key, value) -> {
diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java
index c1c7a42..93480e4 100644
--- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java
+++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/temperature/TemperatureDemo.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.examples.temperature;
 
+import java.time.Duration;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
@@ -30,7 +31,6 @@ import org.apache.kafka.streams.kstream.WindowedSerdes;
 
 import java.util.Properties;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
 
 /**
  * Demonstrates, using the high-level KStream DSL, how to implement an IoT demo application
@@ -88,7 +88,7 @@ public class TemperatureDemo {
             // to group and reduce them, a key is needed ("temp" has been chosen)
             .selectKey((key, value) -> "temp")
             .groupByKey()
-            .windowedBy(TimeWindows.of(TimeUnit.SECONDS.toMillis(TEMPERATURE_WINDOW_SIZE)))
+            .windowedBy(TimeWindows.of(Duration.ofSeconds(TEMPERATURE_WINDOW_SIZE)))
             .reduce((value1, value2) -> {
                 if (Integer.parseInt(value1) > Integer.parseInt(value2))
                     return value1;
diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java
index 4fa38dc..6b49641 100644
--- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java
+++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.examples.wordcount;
 
+import java.time.Duration;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.streams.KafkaStreams;
@@ -63,7 +64,7 @@ public final class WordCountProcessorDemo {
                 @SuppressWarnings("unchecked")
                 public void init(final ProcessorContext context) {
                     this.context = context;
-                    this.context.schedule(1000, PunctuationType.STREAM_TIME, timestamp -> {
+                    this.context.schedule(Duration.ofSeconds(1), PunctuationType.STREAM_TIME, timestamp -> {
                         try (final KeyValueIterator<String, Integer> iter = kvStore.all()) {
                             System.out.println("----------- " + timestamp + " ----------- ");
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
index da1a1c4..5fb8959 100644
--- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
+++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams;
 
+import java.time.Duration;
 import org.apache.kafka.clients.admin.AdminClient;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.producer.KafkaProducer;
@@ -36,6 +37,7 @@ import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.streams.errors.InvalidStateStoreException;
 import org.apache.kafka.streams.errors.ProcessorStateException;
 import org.apache.kafka.streams.errors.StreamsException;
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.kstream.Produced;
@@ -827,7 +829,9 @@ public class KafkaStreams {
      * @return {@code true} if all threads were successfully stopped&mdash;{@code false} if the timeout was reached
      * before all threads stopped
      * Note that this method must not be called in the {@code onChange} callback of {@link StateListener}.
+     * @deprecated Use {@link #close(Duration)} instead
      */
+    @Deprecated
     public synchronized boolean close(final long timeout, final TimeUnit timeUnit) {
         log.debug("Stopping Streams client with timeoutMillis = {} ms.", timeUnit.toMillis(timeout));
 
@@ -896,6 +900,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
      * data with regard to the {@link StreamsConfig#APPLICATION_ID_CONFIG application ID}.
      * <p>
diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/ApiUtils.java b/streams/src/main/java/org/apache/kafka/streams/internals/ApiUtils.java
new file mode 100644
index 0000000..e888d7a
--- /dev/null
+++ b/streams/src/main/java/org/apache/kafka/streams/internals/ApiUtils.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.internals;
+
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Objects;
+
+public final class ApiUtils {
+    private ApiUtils() {
+    }
+
+    /**
+     * Validates that milliseconds from {@code duration} can be retrieved.
+     * @param duration Duration to check.
+     * @param name Name of params for an error message.
+     * @return Milliseconds from {@code duration}.
+     */
+    public static long validateMillisecondDuration(final Duration duration, final String name) {
+        try {
+            if (duration == null)
+                throw new IllegalArgumentException("[" + Objects.toString(name) + "] shouldn't be null.");
+
+            return duration.toMillis();
+        } catch (final ArithmeticException e) {
+            throw new IllegalArgumentException("[" + name + "] can't be converted to milliseconds. ", e);
+        }
+    }
+
+    /**
+     * Validates that milliseconds from {@code instant} can be retrieved.
+     * @param instant Instant to check.
+     * @param name Name of params for an error message.
+     * @return Milliseconds from {@code instant}.
+     */
+    public static long validateMillisecondInstant(final Instant instant, final String name) {
+        try {
+            if (instant == null)
+                throw new IllegalArgumentException("[" + name + "] shouldn't be null.");
+
+            return instant.toEpochMilli();
+        } catch (final ArithmeticException e) {
+            throw new IllegalArgumentException("[" + name + "] can't be converted to milliseconds. ", e);
+        }
+    }
+}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java
index 5e742e1..62eade4 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream;
 
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.processor.TimestampExtractor;
 
 import java.time.Duration;
@@ -110,43 +111,94 @@ public final class JoinWindows extends Windows<Window> {
      *
      * @param timeDifferenceMs join window interval in milliseconds
      * @throws IllegalArgumentException if {@code timeDifferenceMs} is negative
+     * @deprecated Use {@link #of(Duration)} instead.
      */
+    @Deprecated
     public static JoinWindows of(final long timeDifferenceMs) throws IllegalArgumentException {
         // This is a static factory method, so we initialize grace and retention to the defaults.
         return new JoinWindows(timeDifferenceMs, timeDifferenceMs, null, DEFAULT_RETENTION_MS);
     }
 
     /**
+     * Specifies that records of the same key are joinable if their timestamps are within {@code timeDifference},
+     * i.e., the timestamp of a record from the secondary stream is max {@code timeDifference} earlier or later than
+     * the timestamp of the record from the primary stream.
+     *
+     * @param timeDifference join window interval
+     * @throws IllegalArgumentException if {@code timeDifference} is negative or can't be represented as {@code long milliseconds}
+     */
+    public static JoinWindows of(final Duration timeDifference) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondDuration(timeDifference, "timeDifference");
+        return of(timeDifference.toMillis());
+    }
+
+    /**
      * Changes the start window boundary to {@code timeDifferenceMs} but keep the end window boundary as is.
      * Thus, records of the same key are joinable if the timestamp of a record from the secondary stream is at most
      * {@code timeDifferenceMs} earlier than the timestamp of the record from the primary stream.
-     * {@code timeDifferenceMs} can be negative but it's absolute value must not be larger than current window "after"
+     * {@code timeDifferenceMs} can be negative but its absolute value must not be larger than current window "after"
      * value (which would result in a negative window size).
      *
      * @param timeDifferenceMs relative window start time in milliseconds
      * @throws IllegalArgumentException if the resulting window size is negative
+     * @deprecated Use {@link #before(Duration)} instead.
      */
     @SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this
+    @Deprecated
     public JoinWindows before(final long timeDifferenceMs) throws IllegalArgumentException {
         return new JoinWindows(timeDifferenceMs, afterMs, grace, maintainDurationMs, segments);
     }
 
     /**
+     * Changes the start window boundary to {@code timeDifference} but keep the end window boundary as is.
+     * Thus, records of the same key are joinable if the timestamp of a record from the secondary stream is at most
+     * {@code timeDifference} earlier than the timestamp of the record from the primary stream.
+     * {@code timeDifference} can be negative but its absolute value must not be larger than current window "after"
+     * value (which would result in a negative window size).
+     *
+     * @param timeDifference relative window start time
+     * @throws IllegalArgumentException if the resulting window size is negative or {@code timeDifference} can't be represented as {@code long milliseconds}
+     */
+    @SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this
+    public JoinWindows before(final Duration timeDifference) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondDuration(timeDifference, "timeDifference");
+        return before(timeDifference.toMillis());
+    }
+
+    /**
      * Changes the end window boundary to {@code timeDifferenceMs} but keep the start window boundary as is.
      * Thus, records of the same key are joinable if the timestamp of a record from the secondary stream is at most
      * {@code timeDifferenceMs} later than the timestamp of the record from the primary stream.
-     * {@code timeDifferenceMs} can be negative but it's absolute value must not be larger than current window "before"
+     * {@code timeDifferenceMs} can be negative but its absolute value must not be larger than current window "before"
      * value (which would result in a negative window size).
      *
      * @param timeDifferenceMs relative window end time in milliseconds
      * @throws IllegalArgumentException if the resulting window size is negative
+     * @deprecated Use {@link #after(Duration)} instead
      */
     @SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this
+    @Deprecated
     public JoinWindows after(final long timeDifferenceMs) throws IllegalArgumentException {
         return new JoinWindows(beforeMs, timeDifferenceMs, grace, maintainDurationMs, segments);
     }
 
     /**
+     * Changes the end window boundary to {@code timeDifference} but keep the start window boundary as is.
+     * Thus, records of the same key are joinable if the timestamp of a record from the secondary stream is at most
+     * {@code timeDifference} later than the timestamp of the record from the primary stream.
+     * {@code timeDifference} can be negative but its absolute value must not be larger than current window "before"
+     * value (which would result in a negative window size).
+     *
+     * @param timeDifference relative window end time
+     * @throws IllegalArgumentException if the resulting window size is negative or {@code timeDifference} can't be represented as {@code long milliseconds}
+     */
+    @SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this
+    public JoinWindows after(final Duration timeDifference) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondDuration(timeDifference, "timeDifference");
+        return after(timeDifference.toMillis());
+    }
+
+    /**
      * Not supported by {@code JoinWindows}.
      * Throws {@link UnsupportedOperationException}.
      *
@@ -163,20 +215,22 @@ public final class JoinWindows extends Windows<Window> {
     }
 
     /**
-     * Reject late events that arrive more than {@code millisAfterWindowEnd}
+     * Reject late events that arrive more than {@code afterWindowEnd}
      * after the end of its window.
      *
      * Lateness is defined as (stream_time - record_timestamp).
      *
-     * @param millisAfterWindowEnd The grace period to admit late-arriving events to a window.
+     * @param afterWindowEnd The grace period to admit late-arriving events to a window.
      * @return this updated builder
+     * @throws IllegalArgumentException if the {@code afterWindowEnd} is negative of can't be represented as {@code long milliseconds}
      */
     @SuppressWarnings({"deprecation"}) // removing segments from Windows will fix this
-    public JoinWindows grace(final long millisAfterWindowEnd) {
-        if (millisAfterWindowEnd < 0) {
+    public JoinWindows grace(final Duration afterWindowEnd) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondDuration(afterWindowEnd, "afterWindowEnd");
+        if (afterWindowEnd.toMillis() < 0) {
             throw new IllegalArgumentException("Grace period must not be negative.");
         }
-        return new JoinWindows(beforeMs, afterMs, Duration.ofMillis(millisAfterWindowEnd), maintainDurationMs, segments);
+        return new JoinWindows(beforeMs, afterMs, afterWindowEnd, maintainDurationMs, segments);
     }
 
     @SuppressWarnings("deprecation") // continuing to support Windows#maintainMs/segmentInterval in fallback mode
@@ -192,7 +246,7 @@ public final class JoinWindows extends Windows<Window> {
      * @param durationMs the window retention time in milliseconds
      * @return itself
      * @throws IllegalArgumentException if {@code durationMs} is smaller than the window size
-     * @deprecated since 2.1. Use {@link JoinWindows#grace(long)} instead.
+     * @deprecated since 2.1. Use {@link JoinWindows#grace(Duration)} instead.
      */
     @SuppressWarnings("deprecation")
     @Override
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Materialized.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Materialized.java
index 15ec6ce..a19412d 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/Materialized.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Materialized.java
@@ -19,6 +19,7 @@ package org.apache.kafka.streams.kstream;
 import org.apache.kafka.common.internals.Topic;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.processor.StateStore;
 import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier;
 import org.apache.kafka.streams.state.KeyValueStore;
@@ -243,12 +244,14 @@ public class Materialized<K, V, S extends StateStore> {
      * from window-start through window-end, and for the entire grace period.
      *
      * @return itself
+     * @throws IllegalArgumentException if retention is negative or can't be represented as {@code long milliseconds}
      */
-    public Materialized<K, V, S> withRetention(final long retentionMs) {
-        if (retentionMs < 0) {
+    public Materialized<K, V, S> withRetention(final Duration retention) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondDuration(retention, "retention");
+        if (retention.toMillis() < 0) {
             throw new IllegalArgumentException("Retention must not be negative.");
         }
-        retention = Duration.ofMillis(retentionMs);
+        this.retention = retention;
         return this;
     }
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java
index 9efb78c..02c7cbf 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream;
 
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.processor.TimestampExtractor;
 import org.apache.kafka.streams.state.SessionBytesStoreSupplier;
 
@@ -88,7 +89,9 @@ public final class SessionWindows {
      * @return a new window specification with default maintain duration of 1 day
      *
      * @throws IllegalArgumentException if {@code inactivityGapMs} is zero or negative
+     * @deprecated User {@link #with(Duration)} instead.
      */
+    @Deprecated
     public static SessionWindows with(final long inactivityGapMs) {
         if (inactivityGapMs <= 0) {
             throw new IllegalArgumentException("Gap time (inactivityGapMs) cannot be zero or negative.");
@@ -97,6 +100,19 @@ public final class SessionWindows {
     }
 
     /**
+     * Create a new window specification with the specified inactivity gap.
+     *
+     * @param inactivityGap the gap of inactivity between sessions
+     * @return a new window specification with default maintain duration of 1 day
+     *
+     * @throws IllegalArgumentException if {@code inactivityGap} is zero or negative or can't be represented as {@code long milliseconds}
+     */
+    public static SessionWindows with(final Duration inactivityGap) {
+        ApiUtils.validateMillisecondDuration(inactivityGap, "inactivityGap");
+        return with(inactivityGap.toMillis());
+    }
+
+    /**
      * Set the window maintain duration (retention time) in milliseconds.
      * This retention time is a guaranteed <i>lower bound</i> for how long a window will be maintained.
      *
@@ -124,18 +140,20 @@ public final class SessionWindows {
      * close times can lead to surprising results in which a too-late event is rejected and then
      * a subsequent event moves the window boundary forward.
      *
-     * @param millisAfterWindowEnd The grace period to admit late-arriving events to a window.
+     * @param afterWindowEnd The grace period to admit late-arriving events to a window.
      * @return this updated builder
+     * @throws IllegalArgumentException if the {@code afterWindowEnd} is negative of can't be represented as {@code long milliseconds}
      */
-    public SessionWindows grace(final long millisAfterWindowEnd) {
-        if (millisAfterWindowEnd < 0) {
+    public SessionWindows grace(final Duration afterWindowEnd) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondDuration(afterWindowEnd, "afterWindowEnd");
+        if (afterWindowEnd.toMillis() < 0) {
             throw new IllegalArgumentException("Grace period must not be negative.");
         }
 
         return new SessionWindows(
             gapMs,
             maintainDurationMs,
-            Duration.ofMillis(millisAfterWindowEnd)
+            afterWindowEnd
         );
     }
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java
index 6a58c2c..46485b1 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream;
 
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.kstream.internals.TimeWindow;
 import org.apache.kafka.streams.processor.TimestampExtractor;
 import org.apache.kafka.streams.state.WindowBytesStoreSupplier;
@@ -100,7 +101,9 @@ public final class TimeWindows extends Windows<TimeWindow> {
      * @param sizeMs The size of the window in milliseconds
      * @return a new window definition with default maintain duration of 1 day
      * @throws IllegalArgumentException if the specified window size is zero or negative
+     * @deprecated Use {@link #of(Duration)} instead
      */
+    @Deprecated
     public static TimeWindows of(final long sizeMs) throws IllegalArgumentException {
         if (sizeMs <= 0) {
             throw new IllegalArgumentException("Window size (sizeMs) must be larger than zero.");
@@ -110,18 +113,36 @@ public final class TimeWindows extends Windows<TimeWindow> {
     }
 
     /**
+     * Return a window definition with the given window size, and with the advance interval being equal to the window
+     * size.
+     * The time interval represented by the N-th window is: {@code [N * size, N * size + size)}.
+     * <p>
+     * This provides the semantics of tumbling windows, which are fixed-sized, gap-less, non-overlapping windows.
+     * Tumbling windows are a special case of hopping windows with {@code advance == size}.
+     *
+     * @param size The size of the window
+     * @return a new window definition with default maintain duration of 1 day
+     * @throws IllegalArgumentException if the specified window size is zero or negative or can't be represented as {@code long milliseconds}
+     */
+    public static TimeWindows of(final Duration size) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondDuration(size, "size");
+        return of(size.toMillis());
+    }
+
+    /**
      * Return a window definition with the original size, but advance ("hop") the window by the given interval, which
      * specifies by how much a window moves forward relative to the previous one.
      * 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 advanceMs The advance interval ("hop") in milliseconds of the window, with the requirement that
-     *                  {@code 0 < advanceMs &le; sizeMs}.
+     * @param advanceMs The advance interval ("hop") in milliseconds of the window, with the requirement that {@code 0 < advanceMs <= sizeMs}.
      * @return a new window definition with default maintain duration of 1 day
      * @throws IllegalArgumentException if the advance interval is negative, zero, or larger-or-equal the window size
+     * @deprecated Use {@link #advanceBy(Duration)} instead
      */
     @SuppressWarnings("deprecation") // will be fixed when we remove segments from Windows
+    @Deprecated
     public TimeWindows advanceBy(final long advanceMs) {
         if (advanceMs <= 0 || advanceMs > sizeMs) {
             throw new IllegalArgumentException(String.format("AdvanceMs must lie within interval (0, %d].", sizeMs));
@@ -129,6 +150,23 @@ public final class TimeWindows extends Windows<TimeWindow> {
         return new TimeWindows(sizeMs, advanceMs, grace, maintainDurationMs, segments);
     }
 
+    /**
+     * Return a window definition with the original size, but advance ("hop") the window by the given interval, which
+     * specifies by how much a window moves forward relative to the previous one.
+     * The time interval represented by the N-th window is: {@code [N * advance, N * advance + size)}.
+     * <p>
+     * This provides the semantics of hopping windows, which are fixed-sized, overlapping windows.
+     *
+     * @param advance The advance interval ("hop") of the window, with the requirement that {@code 0 < advance.toMillis() <= sizeMs}.
+     * @return a new window definition with default maintain duration of 1 day
+     * @throws IllegalArgumentException if the advance interval is negative, zero, or larger-or-equal the window size
+     */
+    @SuppressWarnings("deprecation") // will be fixed when we remove segments from Windows
+    public TimeWindows advanceBy(final Duration advance) {
+        ApiUtils.validateMillisecondDuration(advance, "advance");
+        return advanceBy(advance.toMillis());
+    }
+
     @Override
     public Map<Long, TimeWindow> windowsFor(final long timestamp) {
         long windowStart = (Math.max(0, timestamp - sizeMs + advanceMs) / advanceMs) * advanceMs;
@@ -152,15 +190,18 @@ public final class TimeWindows extends Windows<TimeWindow> {
      *
      * Lateness is defined as (stream_time - record_timestamp).
      *
-     * @param millisAfterWindowEnd The grace period to admit late-arriving events to a window.
+     * @param afterWindowEnd The grace period to admit late-arriving events to a window.
      * @return this updated builder
+     * @throws IllegalArgumentException if {@code afterWindowEnd} is negative or can't be represented as {@code long milliseconds}
      */
     @SuppressWarnings("deprecation") // will be fixed when we remove segments from Windows
-    public TimeWindows grace(final long millisAfterWindowEnd) {
-        if (millisAfterWindowEnd < 0) {
+    public TimeWindows grace(final Duration afterWindowEnd) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondDuration(afterWindowEnd, "afterWindowEnd");
+        if (afterWindowEnd.toMillis() < 0) {
             throw new IllegalArgumentException("Grace period must not be negative.");
         }
-        return new TimeWindows(sizeMs, advanceMs, Duration.ofMillis(millisAfterWindowEnd), maintainDurationMs, segments);
+
+        return new TimeWindows(sizeMs, advanceMs, afterWindowEnd, maintainDurationMs, segments);
     }
 
     @SuppressWarnings("deprecation") // continuing to support Windows#maintainMs/segmentInterval in fallback mode
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java
index b7487b8..af8e87e 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream;
 
+import java.time.Duration;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.PunctuationType;
@@ -29,7 +30,7 @@ import org.apache.kafka.streams.processor.To;
  * This is a stateful record-by-record operation, i.e, {@link #transform(Object, Object)} is invoked individually for
  * each record of a stream and can access and modify a state that is available beyond a single call of
  * {@link #transform(Object, Object)} (cf. {@link KeyValueMapper} for stateless record transformation).
- * Additionally, this {@code Transformer} can {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule}
+ * Additionally, this {@code Transformer} can {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule}
  * a method to be {@link Punctuator#punctuate(long) called periodically} with the provided context.
  * <p>
  * Use {@link TransformerSupplier} to provide new instances of {@code Transformer} to Kafka Stream's runtime.
@@ -55,7 +56,7 @@ public interface Transformer<K, V, R> {
      * framework may later re-use the transformer by calling {@link #init(ProcessorContext)} again.
      * <p>
      * The provided {@link ProcessorContext context} can be used to access topology and record meta data, to
-     * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be
+     * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} a method to be
      * {@link Punctuator#punctuate(long) called periodically} and to access attached {@link StateStore}s.
      * <p>
      * Note, that {@link ProcessorContext} is updated in the background with the current record's meta data.
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java
index 73aa9b1..46d7270 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java
@@ -16,6 +16,8 @@
  */
 package org.apache.kafka.streams.kstream;
 
+import java.time.Instant;
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.kstream.internals.UnlimitedWindow;
 import org.apache.kafka.streams.processor.TimestampExtractor;
 
@@ -62,7 +64,9 @@ public final class UnlimitedWindows extends Windows<UnlimitedWindow> {
      * @param startMs the window start time
      * @return a new unlimited window that starts at {@code startMs}
      * @throws IllegalArgumentException if the start time is negative
+     * @deprecated Use {@link #startOn(Instant)} instead
      */
+    @Deprecated
     public UnlimitedWindows startOn(final long startMs) throws IllegalArgumentException {
         if (startMs < 0) {
             throw new IllegalArgumentException("Window start time (startMs) cannot be negative.");
@@ -70,6 +74,18 @@ public final class UnlimitedWindows extends Windows<UnlimitedWindow> {
         return new UnlimitedWindows(startMs);
     }
 
+    /**
+     * Return a new unlimited window for the specified start timestamp.
+     *
+     * @param start the window start time
+     * @return a new unlimited window that starts at {@code start}
+     * @throws IllegalArgumentException if the start time is negative or can't be represented as {@code long milliseconds}
+     */
+    public UnlimitedWindows startOn(final Instant start) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(start, "start");
+        return startOn(start.toEpochMilli());
+    }
+
     @Override
     public Map<Long, UnlimitedWindow> windowsFor(final long timestamp) {
         // always return the single unlimited window
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java
index b02311b..987cae5 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream;
 
+import java.time.Duration;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.errors.StreamsException;
 import org.apache.kafka.streams.processor.ProcessorContext;
@@ -29,7 +30,7 @@ import org.apache.kafka.streams.processor.To;
  * This is a stateful record-by-record operation, i.e, {@link #transform(Object)} is invoked individually for each
  * record of a stream and can access and modify a state that is available beyond a single call of
  * {@link #transform(Object)} (cf. {@link ValueMapper} for stateless value transformation).
- * Additionally, this {@code ValueTransformer} can {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule}
+ * Additionally, this {@code ValueTransformer} can {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule}
  * a method to be {@link Punctuator#punctuate(long) called periodically} with the provided context.
  * If {@code ValueTransformer} is applied to a {@link KeyValue} pair record the record's key is preserved.
  * <p>
@@ -54,7 +55,7 @@ public interface ValueTransformer<V, VR> {
      * framework may later re-use the transformer by calling {@link #init(ProcessorContext)} again.
      * <p>
      * The provided {@link ProcessorContext context} can be used to access topology and record meta data, to
-     * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be
+     * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} a method to be
      * {@link Punctuator#punctuate(long) called periodically} and to access attached {@link StateStore}s.
      * <p>
      * Note that {@link ProcessorContext} is updated in the background with the current record's meta data.
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerWithKey.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerWithKey.java
index 20c8692..be37b0c 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerWithKey.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerWithKey.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream;
 
+import java.time.Duration;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.errors.StreamsException;
 import org.apache.kafka.streams.processor.ProcessorContext;
@@ -30,7 +31,7 @@ import org.apache.kafka.streams.processor.To;
  * record of a stream and can access and modify a state that is available beyond a single call of
  * {@link #transform(Object, Object)} (cf. {@link ValueMapper} for stateless value transformation).
  * Additionally, this {@code ValueTransformerWithKey} can
- * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be
+ * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} a method to be
  * {@link Punctuator#punctuate(long) called periodically} with the provided context.
  * Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning.
  * If {@code ValueTransformerWithKey} is applied to a {@link KeyValue} pair record the record's key is preserved.
@@ -57,7 +58,7 @@ public interface ValueTransformerWithKey<K, V, VR> {
      * This is called once per instance when the topology gets initialized.
      * <p>
      * The provided {@link ProcessorContext context} can be used to access topology and record meta data, to
-     * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be
+     * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} a method to be
      * {@link Punctuator#punctuate(long) called periodically} and to access attached {@link StateStore}s.
      * <p>
      * Note that {@link ProcessorContext} is updated in the background with the current record's meta data.
@@ -97,4 +98,4 @@ public interface ValueTransformerWithKey<K, V, VR> {
      * will result in an {@link StreamsException exception}.
      */
     void close();
-}
\ No newline at end of file
+}
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java
index f625068..ac49174 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream;
 
+import java.time.Instant;
 import org.apache.kafka.streams.processor.TimestampExtractor;
 
 /**
@@ -36,6 +37,9 @@ public abstract class Window {
 
     protected final long startMs;
     protected final long endMs;
+    private final Instant startTime;
+    private final Instant endTime;
+
 
     /**
      * Create a new window for the given start and end time.
@@ -53,6 +57,9 @@ public abstract class Window {
         }
         this.startMs = startMs;
         this.endMs = endMs;
+
+        this.startTime = Instant.ofEpochMilli(startMs);
+        this.endTime = Instant.ofEpochMilli(endMs);
     }
 
     /**
@@ -70,6 +77,24 @@ public abstract class Window {
     }
 
     /**
+     * Return the start time of this window.
+     *
+     * @return The start time of this window.
+     */
+    public Instant startTime() {
+        return startTime;
+    }
+
+    /**
+     * Return the end time of this window.
+     *
+     * @return The end time of this window.
+     */
+    public Instant endTime() {
+        return endTime;
+    }
+
+    /**
      * Check if the given window overlaps with this window.
      * Should throw an {@link IllegalArgumentException} if the {@code other} window has a different type than {@code
      * this} window.
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
index a3884c4..96fa8b9 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream.internals;
 
+import java.time.Duration;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.kstream.ForeachAction;
@@ -852,8 +853,8 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
         return Stores.windowStoreBuilder(
             Stores.persistentWindowStore(
                 joinName + "-store",
-                windows.size() + windows.gracePeriodMs(),
-                windows.size(),
+                Duration.ofMillis(windows.size() + windows.gracePeriodMs()),
+                Duration.ofMillis(windows.size()),
                 true
             ),
             keySerde,
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java
index fcb9c02..8ba02bf 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream.internals;
 
+import java.time.Duration;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.utils.Bytes;
@@ -170,8 +171,8 @@ public class TimeWindowedKStreamImpl<K, V, W extends Window> extends AbstractStr
 
                 supplier = Stores.persistentWindowStore(
                     materialized.storeName(),
-                    retentionPeriod,
-                    windows.size(),
+                    Duration.ofMillis(retentionPeriod),
+                    Duration.ofMillis(windows.size()),
                     false
                 );
 
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Cancellable.java b/streams/src/main/java/org/apache/kafka/streams/processor/Cancellable.java
index 2e56b56..2acb7625 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/Cancellable.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/Cancellable.java
@@ -16,8 +16,10 @@
  */
 package org.apache.kafka.streams.processor;
 
+import java.time.Duration;
+
 /**
- * Cancellable interface returned in {@link ProcessorContext#schedule(long, PunctuationType, Punctuator)}.
+ * Cancellable interface returned in {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator)}.
  *
  * @see Punctuator
  */
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java
index e35337f..f91f22f 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.processor;
 
+import java.time.Duration;
 import org.apache.kafka.common.annotation.InterfaceStability;
 
 /**
@@ -33,7 +34,7 @@ public interface Processor<K, V> {
      * framework may later re-use the processor by calling {@link #init()} again.
      * <p>
      * The provided {@link ProcessorContext context} can be used to access topology and record meta data, to
-     * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be
+     * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} a method to be
      * {@link Punctuator#punctuate(long) called periodically} and to access attached {@link StateStore}s.
      * 
      * @param context the context; may not be null
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java
index d21667f..8ec06d5 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.processor;
 
+import java.time.Duration;
 import org.apache.kafka.common.annotation.InterfaceStability;
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.Serde;
@@ -97,16 +98,15 @@ public interface ProcessorContext {
      * Schedules a periodic operation for processors. A processor may call this method during
      * {@link Processor#init(ProcessorContext) initialization} or
      * {@link Processor#process(Object, Object) processing} to
-     * schedule a periodic callback - called a punctuation - to {@link Punctuator#punctuate(long)}.
+     * schedule a periodic callback &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} - 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.
      *   The first punctuation will be triggered by the first record that is processed.
      *   <b>NOTE:</b> Only advanced if messages arrive</li>
-     *   <li>{@link PunctuationType#WALL_CLOCK_TIME} - uses system time (the wall-clock time),
-     *   which is advanced at the polling interval ({@link org.apache.kafka.streams.StreamsConfig#POLL_MS_CONFIG})
-     *   independent of whether new messages arrive.
+     *   <li>{@link PunctuationType#WALL_CLOCK_TIME} &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>
@@ -124,12 +124,49 @@ public interface ProcessorContext {
      * @param type one of: {@link PunctuationType#STREAM_TIME}, {@link PunctuationType#WALL_CLOCK_TIME}
      * @param callback a function consuming timestamps representing the current stream or system time
      * @return a handle allowing cancellation of the punctuation schedule established by this method
+     * @deprecated Use {@link #schedule(Duration, PunctuationType, Punctuator)} instead
      */
+    @Deprecated
     Cancellable schedule(final long intervalMs,
                          final PunctuationType type,
                          final Punctuator callback);
 
     /**
+     * Schedules a periodic operation for processors. A processor may call this method during
+     * {@link Processor#init(ProcessorContext) initialization} or
+     * {@link Processor#process(Object, Object) processing} to
+     * schedule a periodic callback &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.
      * Used the input record's timestamp as timestamp for the output record.
      *
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/PunctuationType.java b/streams/src/main/java/org/apache/kafka/streams/processor/PunctuationType.java
index bc0003d..32965e8 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/PunctuationType.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/PunctuationType.java
@@ -16,8 +16,10 @@
  */
 package org.apache.kafka.streams.processor;
 
+import java.time.Duration;
+
 /**
- * Controls what notion of time is used for punctuation scheduled via {@link ProcessorContext#schedule(long, PunctuationType, Punctuator)} schedule}:
+ * Controls what notion of time is used for punctuation scheduled via {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator)} schedule}:
  * <ul>
  *   <li>STREAM_TIME - uses "stream time", which is advanced by the processing of messages
  *   in accordance with the timestamp as extracted by the {@link TimestampExtractor} in use.
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Punctuator.java b/streams/src/main/java/org/apache/kafka/streams/processor/Punctuator.java
index 407270f..1886dad 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/Punctuator.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/Punctuator.java
@@ -16,8 +16,10 @@
  */
 package org.apache.kafka.streams.processor;
 
+import java.time.Duration;
+
 /**
- * A functional interface used as an argument to {@link ProcessorContext#schedule(long, PunctuationType, Punctuator)}.
+ * A functional interface used as an argument to {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator)}.
  *
  * @see Cancellable
  */
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java
index 7e2610c..0ef70b7 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.processor.internals;
 
+import java.time.Duration;
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.streams.StreamsMetrics;
@@ -85,6 +86,7 @@ public final class ForwardingDisabledProcessorContext implements ProcessorContex
     }
 
     @Override
+    @Deprecated
     public Cancellable schedule(final long intervalMs,
                                 final PunctuationType type,
                                 final Punctuator callback) {
@@ -92,6 +94,13 @@ public final class ForwardingDisabledProcessorContext implements ProcessorContex
     }
 
     @Override
+    public Cancellable schedule(final Duration interval,
+                                final PunctuationType type,
+                                final Punctuator callback) throws IllegalArgumentException {
+        return delegate.schedule(interval, type, callback);
+    }
+
+    @Override
     public <K, V> void forward(final K key, final V value) {
         throw new StreamsException("ProcessorContext#forward() not supported.");
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java
index 56af6e5..5c5b84f 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.processor.internals;
 
+import java.time.Duration;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.processor.Cancellable;
 import org.apache.kafka.streams.processor.PunctuationType;
@@ -92,10 +93,19 @@ public class GlobalProcessorContextImpl extends AbstractProcessorContext {
      * @throws UnsupportedOperationException on every invocation
      */
     @Override
+    @Deprecated
     public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) {
         throw new UnsupportedOperationException("this should not happen: schedule() not supported in global processor context.");
     }
 
+    /**
+     * @throws UnsupportedOperationException on every invocation
+     */
+    @Override
+    public Cancellable schedule(final Duration interval, final PunctuationType type, final Punctuator callback) {
+        throw new UnsupportedOperationException("this should not happen: schedule() not supported in global processor context.");
+    }
+
     @Override
     public long streamTime() {
         throw new RuntimeException("Stream time is not implemented for the global processor context.");
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
index ee21379..21e1c17 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
@@ -16,6 +16,8 @@
  */
 package org.apache.kafka.streams.processor.internals;
 
+import java.time.Duration;
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.errors.StreamsException;
 import org.apache.kafka.streams.processor.Cancellable;
@@ -150,10 +152,19 @@ public class ProcessorContextImpl extends AbstractProcessorContext implements Re
     }
 
     @Override
+    @Deprecated
     public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) {
         return task.schedule(interval, type, callback);
     }
 
+    @Override
+    public Cancellable schedule(final Duration interval,
+                                final PunctuationType type,
+                                final Punctuator callback) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondDuration(interval, "interval");
+        return schedule(interval.toMillis(), type, callback);
+    }
+
     void setStreamTimeSupplier(final TimestampSupplier streamTimeSupplier) {
         this.streamTimeSupplier = streamTimeSupplier;
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java
index e8631aa..6b835d9 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.processor.internals;
 
+import java.time.Duration;
 import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.header.Headers;
@@ -189,6 +190,7 @@ class StandbyContextImpl extends AbstractProcessorContext implements RecordColle
      * @throws UnsupportedOperationException on every invocation
      */
     @Override
+    @SuppressWarnings("deprecation")
     public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) {
         throw new UnsupportedOperationException("this should not happen: schedule() not supported in standby tasks.");
     }
@@ -197,6 +199,14 @@ class StandbyContextImpl extends AbstractProcessorContext implements RecordColle
      * @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
+     */
+    @Override
     public ProcessorRecordContext recordContext() {
         throw new UnsupportedOperationException("this should not happen: recordContext not supported in standby tasks.");
     }
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java
index 0c46fc2..0804338 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.state;
 
+import java.time.Instant;
 import org.apache.kafka.streams.errors.InvalidStateStoreException;
 import org.apache.kafka.streams.kstream.Windowed;
 
@@ -40,9 +41,8 @@ public interface ReadOnlyWindowStore<K, V> {
     V fetch(K key, long time);
 
     /**
-     * Get all the key-value pairs with the given key and the time range from all
-     * the existing windows.
-     *
+     * Get all the key-value pairs with the given key and the time range from all the existing windows.
+     * <p>
      * This iterator must be closed after use.
      * <p>
      * The time range is inclusive and applies to the starting timestamp of the window.
@@ -73,13 +73,51 @@ public interface ReadOnlyWindowStore<K, V> {
      * @return an iterator over key-value pairs {@code <timestamp, value>}
      * @throws InvalidStateStoreException if the store is not initialized
      * @throws NullPointerException If {@code null} is used for key.
+     * @deprecated Use {@link #fetch(K, Instant, Instant)} instead
      */
+    @Deprecated
     WindowStoreIterator<V> fetch(K key, long timeFrom, long timeTo);
 
     /**
-     * Get all the key-value pairs in the given key range and time range from all
-     * the existing windows.
+     * Get all the key-value pairs with the given key and the time range from all the existing windows.
+     * <p>
+     * This iterator must be closed after use.
+     * <p>
+     * The time range is inclusive and applies to the starting timestamp of the window.
+     * For example, if we have the following windows:
+     * <p>
+     * <pre>
+     * +-------------------------------+
+     * |  key  | start time | end time |
+     * +-------+------------+----------+
+     * |   A   |     10     |    20    |
+     * +-------+------------+----------+
+     * |   A   |     15     |    25    |
+     * +-------+------------+----------+
+     * |   A   |     20     |    30    |
+     * +-------+------------+----------+
+     * |   A   |     25     |    35    |
+     * +--------------------------------
+     * </pre>
+     * And we call {@code store.fetch("A", Instant.ofEpochMilli(10), Instant.ofEpochMilli(20))} then the results will contain the first
+     * three windows from the table above, i.e., all those where 10 <= start time <= 20.
+     * <p>
+     * For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest
+     * available window to the newest/latest window.
      *
+     * @param key       the key to fetch
+     * @param from      time range start (inclusive)
+     * @param from      time range end (inclusive)
+     * @return an iterator over key-value pairs {@code <timestamp, value>}
+     * @throws InvalidStateStoreException if the store is not initialized
+     * @throws NullPointerException If {@code null} is used for key.
+     * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds}
+     */
+    WindowStoreIterator<V> fetch(K key, Instant from, Instant to) throws IllegalArgumentException;
+
+    /**
+     * Get all the key-value pairs in the given key range and time range from all the existing windows.
+     * <p>
      * This iterator must be closed after use.
      *
      * @param from      the first key in the range
@@ -89,9 +127,28 @@ public interface ReadOnlyWindowStore<K, V> {
      * @return an iterator over windowed key-value pairs {@code <Windowed<K>, value>}
      * @throws InvalidStateStoreException if the store is not initialized
      * @throws NullPointerException If {@code null} is used for any key.
+     * @deprecated Use {@link #fetch(Object, Object, Instant, Instant)} instead
      */
+    @Deprecated
     KeyValueIterator<Windowed<K>, V> fetch(K from, K to, long timeFrom, long timeTo);
-    
+
+    /**
+     * Get all the key-value pairs in the given key range and time range from all the existing windows.
+     * <p>
+     * This iterator must be closed after use.
+     *
+     * @param from      the first key in the range
+     * @param to        the last key in the range
+     * @param fromTime  time range start (inclusive)
+     * @param toTime    time range end (inclusive)
+     * @return an iterator over windowed key-value pairs {@code <Windowed<K>, value>}
+     * @throws InvalidStateStoreException if the store is not initialized
+     * @throws NullPointerException If {@code null} is used for any key.
+     * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds}
+     */
+    KeyValueIterator<Windowed<K>, V> fetch(K from, K to, Instant fromTime, Instant toTime)
+        throws IllegalArgumentException;
+
     /**
     * Gets all the key-value pairs in the existing windows.
     *
@@ -108,6 +165,20 @@ public interface ReadOnlyWindowStore<K, V> {
      * @return an iterator over windowed key-value pairs {@code <Windowed<K>, value>}
      * @throws InvalidStateStoreException if the store is not initialized
      * @throws NullPointerException if {@code null} is used for any key
+     * @deprecated Use {@link #fetchAll(Instant, Instant)} instead
      */
+    @Deprecated
     KeyValueIterator<Windowed<K>, V> fetchAll(long timeFrom, long timeTo);
+
+    /**
+     * Gets all the key-value pairs that belong to the windows within in the given time range.
+     *
+     * @param from the beginning of the time slot from which to search (inclusive)
+     * @param to   the end of the time slot from which to search (inclusive)
+     * @return an iterator over windowed key-value pairs {@code <Windowed<K>, value>}
+     * @throws InvalidStateStoreException if the store is not initialized
+     * @throws NullPointerException if {@code null} is used for any key
+     * @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds}
+     */
+    KeyValueIterator<Windowed<K>, V> fetchAll(Instant from, Instant to) throws IllegalArgumentException;
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
index 6e965fb..30e5140 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
@@ -16,11 +16,13 @@
  */
 package org.apache.kafka.streams.state;
 
+import java.time.Duration;
 import org.apache.kafka.common.annotation.InterfaceStability;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore;
 import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
 import org.apache.kafka.streams.state.internals.MemoryNavigableLRUCache;
@@ -186,7 +188,9 @@ public class Stores {
      * @param windowSize            size of the windows (cannot be negative)
      * @param retainDuplicates      whether or not to retain duplicates.
      * @return an instance of {@link WindowBytesStoreSupplier}
+     * @deprecated Use {@link #persistentWindowStore(String, Duration, Duration, boolean)} instead
      */
+    @Deprecated
     public static WindowBytesStoreSupplier persistentWindowStore(final String name,
                                                                  final long retentionPeriod,
                                                                  final long windowSize,
@@ -203,11 +207,36 @@ public class Stores {
      *                              Note that the retention period must be at least long enough to contain the
      *                              windowed data's entire life cycle, from window-start through window-end,
      *                              and for the entire grace period.
+     * @param windowSize            size of the windows (cannot be negative)
+     * @param retainDuplicates      whether or not to retain duplicates.
+     * @return an instance of {@link WindowBytesStoreSupplier}
+     * @throws IllegalArgumentException if {@code retentionPeriod} or {@code windowSize} can't be represented as {@code long milliseconds}
+     */
+    public static WindowBytesStoreSupplier persistentWindowStore(final String name,
+                                                                 final Duration retentionPeriod,
+                                                                 final Duration windowSize,
+                                                                 final boolean retainDuplicates) throws IllegalArgumentException {
+        Objects.requireNonNull(name, "name cannot be null");
+        ApiUtils.validateMillisecondDuration(retentionPeriod, "retentionPeriod");
+        ApiUtils.validateMillisecondDuration(windowSize, "windowSize");
+
+        return persistentWindowStore(name, retentionPeriod.toMillis(), windowSize.toMillis(), retainDuplicates);
+    }
+
+    /**
+     * Create a persistent {@link WindowBytesStoreSupplier}.
+     * @param name                  name of the store (cannot be {@code null})
+     * @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 segmentInterval       size of segments in ms (cannot be negative)
      * @param windowSize            size of the windows (cannot be negative)
      * @param retainDuplicates      whether or not to retain duplicates.
      * @return an instance of {@link WindowBytesStoreSupplier}
+     * @deprecated Use {@link #persistentWindowStore(String, Duration, Duration, boolean)} instead
      */
+    @Deprecated
     public static WindowBytesStoreSupplier persistentWindowStore(final String name,
                                                                  final long retentionPeriod,
                                                                  final long windowSize,
@@ -250,6 +279,21 @@ public class Stores {
         return new RocksDbSessionBytesStoreSupplier(name, retentionPeriod);
     }
 
+    /**
+     * Create a persistent {@link SessionBytesStoreSupplier}.
+     * @param name              name of the store (cannot be {@code null})
+     * @param retentionPeriod   length ot time to retain data in the store (cannot be negative)
+     *                          Note that the retention period must be at least long enough to contain the
+     *                          windowed data's entire life cycle, from window-start through window-end,
+     *                          and for the entire grace period.
+     * @return an instance of a {@link  SessionBytesStoreSupplier}
+     */
+    public static SessionBytesStoreSupplier persistentSessionStore(final String name,
+                                                                   final Duration retentionPeriod) {
+        ApiUtils.validateMillisecondDuration(retentionPeriod, "retentionPeriod");
+        return persistentSessionStore(name, retentionPeriod.toMillis());
+    }
+
 
     /**
      * Creates a {@link StoreBuilder} that can be used to build a {@link WindowStore}.
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java
index 1685123..ad74ae1 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java
@@ -16,6 +16,8 @@
  */
 package org.apache.kafka.streams.state;
 
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.kstream.Windowed;
 import org.apache.kafka.streams.processor.StateStore;
 
 /**
@@ -48,4 +50,66 @@ public interface WindowStore<K, V> extends StateStore, ReadOnlyWindowStore<K, V>
      * @throws NullPointerException If null is used for key.
      */
     void put(K key, V value, long windowStartTimestamp);
+
+    /**
+     * Get all the key-value pairs with the given key and the time range from all the existing windows.
+     * <p>
+     * This iterator must be closed after use.
+     * <p>
+     * The time range is inclusive and applies to the starting timestamp of the window.
+     * For example, if we have the following windows:
+     * <p>
+     * <pre>
+     * +-------------------------------+
+     * |  key  | start time | end time |
+     * +-------+------------+----------+
+     * |   A   |     10     |    20    |
+     * +-------+------------+----------+
+     * |   A   |     15     |    25    |
+     * +-------+------------+----------+
+     * |   A   |     20     |    30    |
+     * +-------+------------+----------+
+     * |   A   |     25     |    35    |
+     * +--------------------------------
+     * </pre>
+     * And we call {@code store.fetch("A", 10, 20)} then the results will contain the first
+     * three windows from the table above, i.e., all those where 10 <= start time <= 20.
+     * <p>
+     * For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest
+     * available window to the newest/latest window.
+     *
+     * @param key       the key to fetch
+     * @param timeFrom  time range start (inclusive)
+     * @param timeTo    time range end (inclusive)
+     * @return an iterator over key-value pairs {@code <timestamp, value>}
+     * @throws InvalidStateStoreException if the store is not initialized
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    WindowStoreIterator<V> fetch(K key, long timeFrom, long timeTo);
+
+    /**
+     * Get all the key-value pairs in the given key range and time range from all the existing windows.
+     * <p>
+     * This iterator must be closed after use.
+     *
+     * @param from      the first key in the range
+     * @param to        the last key in the range
+     * @param timeFrom  time range start (inclusive)
+     * @param timeTo    time range end (inclusive)
+     * @return an iterator over windowed key-value pairs {@code <Windowed<K>, value>}
+     * @throws InvalidStateStoreException if the store is not initialized
+     * @throws NullPointerException If {@code null} is used for any key.
+     */
+    KeyValueIterator<Windowed<K>, V> fetch(K from, K to, long timeFrom, long timeTo);
+
+    /**
+     * Gets all the key-value pairs that belong to the windows within in the given time range.
+     *
+     * @param timeFrom the beginning of the time slot from which to search (inclusive)
+     * @param timeTo   the end of the time slot from which to search (inclusive)
+     * @return an iterator over windowed key-value pairs {@code <Windowed<K>, value>}
+     * @throws InvalidStateStoreException if the store is not initialized
+     * @throws NullPointerException if {@code null} is used for any key
+     */
+    KeyValueIterator<Windowed<K>, V> fetchAll(long timeFrom, long timeTo);
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java
index c07130e..1416351 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java
@@ -16,12 +16,14 @@
  */
 package org.apache.kafka.streams.state;
 
+import java.time.Instant;
 import org.apache.kafka.streams.KeyValue;
 
 import java.io.Closeable;
 
 /**
- * Iterator interface of {@link KeyValue} with key typed {@link Long} used for {@link WindowStore#fetch(Object, long, long)}.
+ * Iterator interface of {@link KeyValue} with key typed {@link Long} used for {@link WindowStore#fetch(Object, long, long)}
+ * and {@link WindowStore#fetch(Object, Instant, Instant)}
  *
  * Users must call its {@code close} method explicitly upon completeness to release resources,
  * or use try-with-resources statement (available since JDK7) for this {@link Closeable} class.
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java
index f8d9ad5..f6b62b2 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java
@@ -16,9 +16,11 @@
  */
 package org.apache.kafka.streams.state.internals;
 
+import java.time.Instant;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.kstream.Windowed;
 import org.apache.kafka.streams.kstream.internals.CacheFlushListener;
 import org.apache.kafka.streams.processor.ProcessorContext;
@@ -204,6 +206,13 @@ class CachingWindowStore<K, V> extends WrappedStateStore.AbstractStateStore impl
     }
 
     @Override
+    public WindowStoreIterator<byte[]> fetch(final Bytes key, final Instant from, final Instant to) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(from, "from");
+        ApiUtils.validateMillisecondInstant(to, "to");
+        return fetch(key, from.toEpochMilli(), to.toEpochMilli());
+    }
+
+    @Override
     public KeyValueIterator<Windowed<Bytes>, byte[]> fetch(final Bytes from, final Bytes to, final long timeFrom, final long timeTo) {
         // since this function may not access the underlying inner store, we need to validate
         // if store is open outside as well.
@@ -231,7 +240,17 @@ class CachingWindowStore<K, V> extends WrappedStateStore.AbstractStateStore impl
             cacheFunction
         );
     }
-    
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> fetch(final Bytes from,
+                                                           final Bytes to,
+                                                           final Instant fromTime,
+                                                           final Instant toTime) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(fromTime, "fromTime");
+        ApiUtils.validateMillisecondInstant(toTime, "toTime");
+        return fetch(from, to, fromTime.toEpochMilli(), toTime.toEpochMilli());
+    }
+
     private V fetchPrevious(final Bytes key, final long timestamp) {
         final byte[] value = underlying.fetch(key, timestamp);
         if (value != null) {
@@ -259,10 +278,10 @@ class CachingWindowStore<K, V> extends WrappedStateStore.AbstractStateStore impl
     @Override
     public KeyValueIterator<Windowed<Bytes>, byte[]> fetchAll(final long timeFrom, final long timeTo) {
         validateStoreOpen();
-        
+
         final KeyValueIterator<Windowed<Bytes>, byte[]> underlyingIterator = underlying.fetchAll(timeFrom, timeTo);
         final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.all(name);
-        
+
         final HasNextCondition hasNextCondition = keySchema.hasNextCondition(null, null, timeFrom, timeTo);
         final PeekingKeyValueIterator<Bytes, LRUCacheEntry> filteredCacheIterator = new FilteredCacheIterator(cacheIterator,
                                                                                                               hasNextCondition,
@@ -275,4 +294,11 @@ class CachingWindowStore<K, V> extends WrappedStateStore.AbstractStateStore impl
                 cacheFunction
         );
     }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(from, "from");
+        ApiUtils.validateMillisecondInstant(to, "to");
+        return fetchAll(from.toEpochMilli(), to.toEpochMilli());
+    }
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java
index 785aacd..d4e47c6 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java
@@ -16,8 +16,10 @@
  */
 package org.apache.kafka.streams.state.internals;
 
+import java.time.Instant;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.kstream.Windowed;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.StateStore;
@@ -57,11 +59,28 @@ class ChangeLoggingWindowBytesStore extends WrappedStateStore.AbstractStateStore
     }
 
     @Override
+    public WindowStoreIterator<byte[]> fetch(final Bytes key, final Instant from, final Instant to) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(from, "from");
+        ApiUtils.validateMillisecondInstant(to, "to");
+        return fetch(key, from.toEpochMilli(), to.toEpochMilli());
+    }
+
+    @Override
     public KeyValueIterator<Windowed<Bytes>, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) {
         return bytesStore.fetch(keyFrom, keyTo, from, to);
     }
 
     @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> fetch(final Bytes from,
+                                                           final Bytes to,
+                                                           final Instant fromTime,
+                                                           final Instant toTime) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(fromTime, "fromTime");
+        ApiUtils.validateMillisecondInstant(toTime, "toTime");
+        return fetch(from, to, fromTime.toEpochMilli(), toTime.toEpochMilli());
+    }
+
+    @Override
     public KeyValueIterator<Windowed<Bytes>, byte[]> all() {
         return bytesStore.all();
     }
@@ -70,7 +89,14 @@ class ChangeLoggingWindowBytesStore extends WrappedStateStore.AbstractStateStore
     public KeyValueIterator<Windowed<Bytes>, byte[]> fetchAll(final long timeFrom, final long 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
     public void put(final Bytes key, final byte[] value) {
         put(key, value, context.timestamp());
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java
index 1b5d5e5..d95b442 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java
@@ -16,6 +16,8 @@
  */
 package org.apache.kafka.streams.state.internals;
 
+import java.time.Instant;
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.errors.InvalidStateStoreException;
 import org.apache.kafka.streams.kstream.Windowed;
 import org.apache.kafka.streams.state.KeyValueIterator;
@@ -64,6 +66,7 @@ public class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K
     }
 
     @Override
+    @Deprecated
     public WindowStoreIterator<V> fetch(final K key, final long timeFrom, final long timeTo) {
         Objects.requireNonNull(key, "key can't be null");
         final List<ReadOnlyWindowStore<K, V>> stores = provider.stores(storeName, windowStoreType);
@@ -85,6 +88,13 @@ public class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K
     }
 
     @Override
+    public WindowStoreIterator<V> fetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(from, "from");
+        ApiUtils.validateMillisecondInstant(to, "to");
+        return fetch(key, from.toEpochMilli(), to.toEpochMilli());
+    }
+
+    @Override
     public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) {
         Objects.requireNonNull(from, "from can't be null");
         Objects.requireNonNull(to, "to can't be null");
@@ -99,7 +109,14 @@ public class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K
                                                                provider.stores(storeName, windowStoreType).iterator(),
                                                                nextIteratorFunction));
     }
-    
+
+    @Override
+    public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final Instant fromTime, final Instant toTime) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(fromTime, "fromTime");
+        ApiUtils.validateMillisecondInstant(toTime, "toTime");
+        return fetch(from, to, fromTime.toEpochMilli(), toTime.toEpochMilli());
+    }
+
     @Override
     public KeyValueIterator<Windowed<K>, V> all() {
         final NextIteratorFunction<Windowed<K>, V, ReadOnlyWindowStore<K, V>> nextIteratorFunction = new NextIteratorFunction<Windowed<K>, V, ReadOnlyWindowStore<K, V>>() {
@@ -115,6 +132,7 @@ public class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K
     }
     
     @Override
+    @Deprecated
     public KeyValueIterator<Windowed<K>, V> fetchAll(final long timeFrom, final long timeTo) {
         final NextIteratorFunction<Windowed<K>, V, ReadOnlyWindowStore<K, V>> nextIteratorFunction = new NextIteratorFunction<Windowed<K>, V, ReadOnlyWindowStore<K, V>>() {
             @Override
@@ -127,4 +145,11 @@ public class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K
                         provider.stores(storeName, windowStoreType).iterator(),
                         nextIteratorFunction));
     }
+
+    @Override
+    public KeyValueIterator<Windowed<K>, V> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(from, "from");
+        ApiUtils.validateMillisecondInstant(to, "to");
+        return fetchAll(from.toEpochMilli(), to.toEpochMilli());
+    }
 }
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java
index 5162eac..e1b6cd1 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java
@@ -16,10 +16,12 @@
  */
 package org.apache.kafka.streams.state.internals;
 
+import java.time.Instant;
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.errors.ProcessorStateException;
 import org.apache.kafka.streams.kstream.Windowed;
 import org.apache.kafka.streams.processor.ProcessorContext;
@@ -148,6 +150,13 @@ public class MeteredWindowStore<K, V> extends WrappedStateStore.AbstractStateSto
     }
 
     @Override
+    public WindowStoreIterator<V> fetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(from, "from");
+        ApiUtils.validateMillisecondInstant(to, "to");
+        return fetch(key, from.toEpochMilli(), to.toEpochMilli());
+    }
+
+    @Override
     public KeyValueIterator<Windowed<K>, V> all() {
         return new MeteredWindowedKeyValueIterator<>(inner.all(), fetchTime, metrics, serdes, time);
     }
@@ -162,6 +171,13 @@ public class MeteredWindowStore<K, V> extends WrappedStateStore.AbstractStateSto
     }
 
     @Override
+    public KeyValueIterator<Windowed<K>, V> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(from, "from");
+        ApiUtils.validateMillisecondInstant(to, "to");
+        return fetchAll(from.toEpochMilli(), to.toEpochMilli());
+    }
+
+    @Override
     public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) {
         return new MeteredWindowedKeyValueIterator<>(inner.fetch(keyBytes(from), keyBytes(to), timeFrom, timeTo),
                                                      fetchTime,
@@ -171,6 +187,13 @@ public class MeteredWindowStore<K, V> extends WrappedStateStore.AbstractStateSto
     }
 
     @Override
+    public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final Instant fromTime, final Instant toTime) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(fromTime, "fromTime");
+        ApiUtils.validateMillisecondInstant(toTime, "toTime");
+        return fetch(from, to, fromTime.toEpochMilli(), toTime.toEpochMilli());
+    }
+
+    @Override
     public void flush() {
         final long startNs = time.nanoseconds();
         try {
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java
index d7bb523..e8037bc 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java
@@ -16,8 +16,10 @@
  */
 package org.apache.kafka.streams.state.internals;
 
+import java.time.Instant;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.kstream.Windowed;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.StateStore;
@@ -92,12 +94,26 @@ public class RocksDBWindowStore<K, V> extends WrappedStateStore.AbstractStateSto
     }
 
     @Override
+    public WindowStoreIterator<V> fetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(from, "from");
+        ApiUtils.validateMillisecondInstant(to, "to");
+        return fetch(key, from.toEpochMilli(), to.toEpochMilli());
+    }
+
+    @Override
     public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) {
         final KeyValueIterator<Bytes, byte[]> bytesIterator = bytesStore.fetch(Bytes.wrap(serdes.rawKey(from)), Bytes.wrap(serdes.rawKey(to)), timeFrom, timeTo);
         return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).keyValueIterator();
     }
 
     @Override
+    public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final Instant fromTime, final Instant toTime) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(fromTime, "fromTime");
+        ApiUtils.validateMillisecondInstant(toTime, "toTime");
+        return fetch(from, to, fromTime.toEpochMilli(), toTime.toEpochMilli());
+    }
+
+    @Override
     public KeyValueIterator<Windowed<K>, V> all() {
         final KeyValueIterator<Bytes, byte[]> bytesIterator = bytesStore.all();
         return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).keyValueIterator();
@@ -109,6 +125,13 @@ public class RocksDBWindowStore<K, V> extends WrappedStateStore.AbstractStateSto
         return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).keyValueIterator();
     }
 
+    @Override
+    public KeyValueIterator<Windowed<K>, V> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(from, "from");
+        ApiUtils.validateMillisecondInstant(to, "to");
+        return fetchAll(from.toEpochMilli(), to.toEpochMilli());
+    }
+
     private void maybeUpdateSeqnumForDups() {
         if (retainDuplicates) {
             seqnum = (seqnum + 1) & 0x7FFFFFFF;
diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
index 5e07703..abc4cb9 100644
--- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams;
 
+import java.time.Duration;
 import org.apache.kafka.clients.CommonClientConfigs;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.producer.MockProducer;
@@ -463,7 +464,7 @@ public class KafkaStreamsTest {
                 System.currentTimeMillis());
 
             assertTrue("Timed out waiting to receive single message", latch.await(30, TimeUnit.SECONDS));
-            assertFalse(streams.close(10, TimeUnit.MILLISECONDS));
+            assertFalse(streams.close(Duration.ofMillis(10)));
         } finally {
             // stop the thread so we don't interfere with other tests etc
             keepRunning.set(false);
diff --git a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java
index 02840b4..289cdf0 100644
--- a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java
@@ -47,6 +47,7 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.regex.Pattern;
 
+import static java.time.Duration.ofMillis;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertEquals;
@@ -737,7 +738,7 @@ public class TopologyTest {
         final StreamsBuilder builder = new StreamsBuilder();
         builder.stream("input-topic")
             .groupByKey()
-            .windowedBy(TimeWindows.of(1))
+            .windowedBy(TimeWindows.of(ofMillis(1)))
             .count();
         final TopologyDescription describe = builder.build().describe();
         assertEquals(
@@ -757,7 +758,7 @@ public class TopologyTest {
         final StreamsBuilder builder = new StreamsBuilder();
         builder.stream("input-topic")
             .groupByKey()
-            .windowedBy(TimeWindows.of(1))
+            .windowedBy(TimeWindows.of(ofMillis(1)))
             .count(Materialized.as("count-store"));
         final TopologyDescription describe = builder.build().describe();
         assertEquals(
@@ -777,7 +778,7 @@ public class TopologyTest {
         final StreamsBuilder builder = new StreamsBuilder();
         builder.stream("input-topic")
             .groupByKey()
-            .windowedBy(TimeWindows.of(1))
+            .windowedBy(TimeWindows.of(ofMillis(1)))
             .count(Materialized.with(null, Serdes.Long()));
         final TopologyDescription describe = builder.build().describe();
         assertEquals(
@@ -797,7 +798,7 @@ public class TopologyTest {
         final StreamsBuilder builder = new StreamsBuilder();
         builder.stream("input-topic")
             .groupByKey()
-            .windowedBy(SessionWindows.with(1))
+            .windowedBy(SessionWindows.with(ofMillis(1)))
             .count();
         final TopologyDescription describe = builder.build().describe();
         assertEquals(
@@ -817,7 +818,7 @@ public class TopologyTest {
         final StreamsBuilder builder = new StreamsBuilder();
         builder.stream("input-topic")
             .groupByKey()
-            .windowedBy(SessionWindows.with(1))
+            .windowedBy(SessionWindows.with(ofMillis(1)))
             .count(Materialized.as("count-store"));
         final TopologyDescription describe = builder.build().describe();
         assertEquals(
@@ -837,7 +838,7 @@ public class TopologyTest {
         final StreamsBuilder builder = new StreamsBuilder();
         builder.stream("input-topic")
             .groupByKey()
-            .windowedBy(SessionWindows.with(1))
+            .windowedBy(SessionWindows.with(ofMillis(1)))
             .count(Materialized.with(null, Serdes.Long()));
         final TopologyDescription describe = builder.build().describe();
         assertEquals(
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java
index caa02db..bb199b7 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.integration;
 
+import java.time.Duration;
 import org.apache.kafka.clients.CommonClientConfigs;
 import org.apache.kafka.clients.admin.AdminClient;
 import org.apache.kafka.clients.admin.ConsumerGroupDescription;
@@ -66,6 +67,7 @@ import java.util.concurrent.TimeUnit;
 
 import kafka.tools.StreamsResetter;
 
+import static java.time.Duration.ofMillis;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.MatcherAssert.assertThat;
 
@@ -199,7 +201,7 @@ public abstract class AbstractResetIntegrationTest {
 
     void cleanupTest() throws Exception {
         if (streams != null) {
-            streams.close(30, TimeUnit.SECONDS);
+            streams.close(Duration.ofSeconds(30));
         }
         IntegrationTestUtils.purgeLocalStreamsState(streamsConfig);
     }
@@ -526,7 +528,7 @@ public abstract class AbstractResetIntegrationTest {
 
         input.through(INTERMEDIATE_USER_TOPIC)
             .groupByKey()
-            .windowedBy(TimeWindows.of(35).advanceBy(10))
+            .windowedBy(TimeWindows.of(ofMillis(35)).advanceBy(ofMillis(10)))
             .count()
             .toStream()
             .map(new KeyValueMapper<Windowed<Long>, Long, KeyValue<Long, Long>>() {
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
index cae0f57..a0ada62 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.kafka.streams.integration;
 
+import java.time.Duration;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.LongSerializer;
 import org.apache.kafka.common.serialization.Serdes;
@@ -52,7 +53,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 
 import kafka.utils.MockTime;
 
@@ -148,7 +148,7 @@ public class GlobalThreadShutDownOrderTest {
             }
         }, 10000L, "Has not processed record within 10 seconds");
 
-        kafkaStreams.close(30, TimeUnit.SECONDS);
+        kafkaStreams.close(Duration.ofSeconds(30));
 
         final List<Long> expectedRetrievedValues = Arrays.asList(1L, 2L, 3L, 4L);
         assertEquals(expectedRetrievedValues, retrievedValuesList);
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
index 153c5a1..9bd8c65 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
@@ -57,6 +57,8 @@ import java.util.Properties;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
+import static java.time.Duration.ofMillis;
+import static java.time.Duration.ofSeconds;
 import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForCompletion;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -187,8 +189,8 @@ public class InternalTopicIntegrationTest {
 
         textLines.flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+")))
             .groupBy(MockMapper.selectValueMapper())
-            .windowedBy(TimeWindows.of(1000).grace(0L))
-            .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("CountWindows").withRetention(2_000L));
+            .windowedBy(TimeWindows.of(ofSeconds(1L)).grace(ofMillis(0L)))
+            .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("CountWindows").withRetention(ofSeconds(2L)));
 
         final KafkaStreams streams = new KafkaStreams(builder.build(), streamsProp);
         streams.start();
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
index b51511e..08aa245 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
@@ -53,6 +53,8 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
 
+import static java.time.Duration.ofMillis;
+
 /**
  * Similar to KStreamAggregationIntegrationTest but with dedupping enabled
  * by virtue of having a large commit interval
@@ -144,7 +146,7 @@ public class KStreamAggregationDedupIntegrationTest {
         produceMessages(secondBatchTimestamp);
 
         groupedStream
-            .windowedBy(TimeWindows.of(500L))
+            .windowedBy(TimeWindows.of(ofMillis(500L)))
             .reduce(reducer, Materialized.as("reduce-time-windows"))
             .toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start())
             .to(outputTopic, Produced.with(Serdes.String(), Serdes.String()));
@@ -179,7 +181,7 @@ public class KStreamAggregationDedupIntegrationTest {
         produceMessages(timestamp);
 
         stream.groupByKey(Serialized.with(Serdes.Integer(), Serdes.String()))
-            .windowedBy(TimeWindows.of(500L))
+            .windowedBy(TimeWindows.of(ofMillis(500L)))
             .count(Materialized.as("count-windows"))
             .toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start())
             .to(outputTopic, Produced.with(Serdes.String(), Serdes.Long()));
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
index ce6c352..7642f69 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
@@ -85,6 +85,8 @@ import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
+import static java.time.Duration.ofMillis;
+import static java.time.Instant.ofEpochMilli;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.core.Is.is;
@@ -202,7 +204,7 @@ public class KStreamAggregationIntegrationTest {
 
         final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
         groupedStream
-                .windowedBy(TimeWindows.of(500L))
+                .windowedBy(TimeWindows.of(ofMillis(500L)))
                 .reduce(reducer)
                 .toStream()
                 .to(outputTopic, Produced.with(windowedSerde, Serdes.String()));
@@ -307,7 +309,7 @@ public class KStreamAggregationIntegrationTest {
         produceMessages(secondTimestamp);
 
         final Serde<Windowed<String>> windowedSerde = WindowedSerdes.timeWindowedSerdeFrom(String.class);
-        groupedStream.windowedBy(TimeWindows.of(500L))
+        groupedStream.windowedBy(TimeWindows.of(ofMillis(500L)))
                 .aggregate(
                         initializer,
                         aggregator,
@@ -427,7 +429,7 @@ public class KStreamAggregationIntegrationTest {
         produceMessages(timestamp);
 
         stream.groupByKey(Serialized.with(Serdes.Integer(), Serdes.String()))
-                .windowedBy(TimeWindows.of(500L))
+                .windowedBy(TimeWindows.of(ofMillis(500L)))
                 .count()
                 .toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start()).to(outputTopic, Produced.with(Serdes.String(), Serdes.Long()));
 
@@ -520,7 +522,7 @@ public class KStreamAggregationIntegrationTest {
 
         builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String()))
                 .groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-                .windowedBy(SessionWindows.with(sessionGap))
+                .windowedBy(SessionWindows.with(ofMillis(sessionGap)))
                 .count()
                 .toStream()
                 .transform(() -> new Transformer<Windowed<String>, Long, KeyValue<Object, Object>>() {
@@ -618,7 +620,7 @@ public class KStreamAggregationIntegrationTest {
         final String userSessionsStore = "UserSessionsStore";
         builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String()))
                 .groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-                .windowedBy(SessionWindows.with(sessionGap))
+                .windowedBy(SessionWindows.with(ofMillis(sessionGap)))
                 .reduce((value1, value2) -> value1 + ":" + value2, Materialized.as(userSessionsStore))
                 .toStream()
                 .foreach((key, value) -> {
@@ -705,7 +707,7 @@ public class KStreamAggregationIntegrationTest {
 
         builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String()))
                .groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-               .windowedBy(UnlimitedWindows.of().startOn(startTime))
+               .windowedBy(UnlimitedWindows.of().startOn(ofEpochMilli(startTime)))
                .count()
                .toStream()
                .transform(() -> new Transformer<Windowed<String>, Long, KeyValue<Object, Object>>() {
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java
index dd78190..d5781dd 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java
@@ -18,6 +18,7 @@
 package org.apache.kafka.streams.integration;
 
 
+import java.time.Duration;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.serialization.Serdes;
@@ -51,7 +52,6 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
 
 @Category({IntegrationTest.class})
 public class KTableSourceTopicRestartIntegrationTest {
@@ -130,7 +130,7 @@ public class KTableSourceTopicRestartIntegrationTest {
 
             assertNumberValuesRead(readKeyValues, expectedResultsWithDataWrittenDuringRestoreMap, "Table did not get all values after restart");
         } finally {
-            streamsOne.close(5, TimeUnit.SECONDS);
+            streamsOne.close(Duration.ofSeconds(5));
         }
     }
 
@@ -155,7 +155,7 @@ public class KTableSourceTopicRestartIntegrationTest {
 
             assertNumberValuesRead(readKeyValues, expectedResultsWithDataWrittenDuringRestoreMap, "Table did not get all values after restart");
         } finally {
-            streamsOne.close(5, TimeUnit.SECONDS);
+            streamsOne.close(Duration.ofSeconds(5));
         }
     }
 
@@ -179,7 +179,7 @@ public class KTableSourceTopicRestartIntegrationTest {
 
             assertNumberValuesRead(readKeyValues, expectedValues, "Table did not get all values after restart");
         } finally {
-            streamsOne.close(5, TimeUnit.SECONDS);
+            streamsOne.close(Duration.ofSeconds(5));
         }
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java
index 2269a5d..96d7d14 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.integration;
 
+import java.time.Duration;
 import org.apache.kafka.clients.admin.AdminClient;
 import org.apache.kafka.clients.admin.Config;
 import org.apache.kafka.clients.producer.ProducerConfig;
@@ -51,7 +52,6 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
 
 @Category({IntegrationTest.class})
 public class PurgeRepartitionTopicIntegrationTest {
@@ -173,7 +173,7 @@ public class PurgeRepartitionTopicIntegrationTest {
     @After
     public void shutdown() throws IOException {
         if (kafkaStreams != null) {
-            kafkaStreams.close(30, TimeUnit.SECONDS);
+            kafkaStreams.close(Duration.ofSeconds(30));
         }
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
index 97d1071..76eec71 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
@@ -86,6 +86,9 @@ import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import static java.time.Duration.ofMillis;
+import static java.time.Duration.ofSeconds;
+import static java.time.Instant.ofEpochMilli;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.core.IsEqual.equalTo;
 import static org.junit.Assert.assertEquals;
@@ -220,7 +223,7 @@ public class QueryableStateIntegrationTest {
     @After
     public void shutdown() throws IOException {
         if (kafkaStreams != null) {
-            kafkaStreams.close(30, TimeUnit.SECONDS);
+            kafkaStreams.close(ofSeconds(30));
         }
         IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration);
     }
@@ -256,7 +259,7 @@ public class QueryableStateIntegrationTest {
 
         // Create a Windowed State Store that contains the word count for every 1 minute
         groupedByWord
-            .windowedBy(TimeWindows.of(WINDOW_SIZE))
+            .windowedBy(TimeWindows.of(ofMillis(WINDOW_SIZE)))
             .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as(windowStoreName + "-" + inputTopic))
             .toStream(new KeyValueMapper<Windowed<String>, Long, String>() {
                 @Override
@@ -361,7 +364,7 @@ public class QueryableStateIntegrationTest {
                         final int index = metadata.hostInfo().port();
                         final KafkaStreams streamsWithKey = streamRunnables[index].getStream();
                         final ReadOnlyWindowStore<String, Long> store = streamsWithKey.store(storeName, QueryableStoreTypes.<String, Long>windowStore());
-                        return store != null && store.fetch(key, from, to) != null;
+                        return store != null && store.fetch(key, ofEpochMilli(from), ofEpochMilli(to)) != null;
                     } catch (final IllegalStateException e) {
                         // Kafka Streams instance may have closed but rebalance hasn't happened
                         return false;
@@ -695,7 +698,7 @@ public class QueryableStateIntegrationTest {
 
         final String windowStoreName = "windowed-count";
         s1.groupByKey()
-            .windowedBy(TimeWindows.of(WINDOW_SIZE))
+            .windowedBy(TimeWindows.of(ofMillis(WINDOW_SIZE)))
             .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as(windowStoreName));
         kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration);
         kafkaStreams.start();
@@ -1017,7 +1020,7 @@ public class QueryableStateIntegrationTest {
     private Set<KeyValue<String, Long>> fetch(final ReadOnlyWindowStore<String, Long> store,
                                               final String key) {
 
-        final WindowStoreIterator<Long> fetch = store.fetch(key, 0, System.currentTimeMillis());
+        final WindowStoreIterator<Long> fetch = store.fetch(key, ofEpochMilli(0), ofEpochMilli(System.currentTimeMillis()));
         if (fetch.hasNext()) {
             final KeyValue<Long, Long> next = fetch.next();
             return Collections.singleton(KeyValue.pair(key, next.value));
@@ -1028,7 +1031,7 @@ public class QueryableStateIntegrationTest {
     private Map<String, Long> fetchMap(final ReadOnlyWindowStore<String, Long> store,
                                        final String key) {
 
-        final WindowStoreIterator<Long> fetch = store.fetch(key, 0, System.currentTimeMillis());
+        final WindowStoreIterator<Long> fetch = store.fetch(key, ofEpochMilli(0), ofEpochMilli(System.currentTimeMillis()));
         if (fetch.hasNext()) {
             final KeyValue<Long, Long> next = fetch.next();
             return Collections.singletonMap(key, next.value);
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionOptimizingIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionOptimizingIntegrationTest.java
index cde2349..ec9df19 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionOptimizingIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionOptimizingIntegrationTest.java
@@ -55,10 +55,11 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Locale;
 import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import static java.time.Duration.ofMillis;
+import static java.time.Duration.ofSeconds;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
@@ -161,7 +162,7 @@ public class RepartitionOptimizingIntegrationTest {
 
         mappedStream.filter((k, v) -> k.equals("A"))
             .join(countStream, (v1, v2) -> v1 + ":" + v2.toString(),
-                  JoinWindows.of(5000),
+                  JoinWindows.of(ofMillis(5000)),
                   Joined.with(Serdes.String(), Serdes.String(), Serdes.Long()))
             .to(JOINED_TOPIC);
 
@@ -211,7 +212,7 @@ public class RepartitionOptimizingIntegrationTest {
         assertThat(3, equalTo(processorValueCollector.size()));
         assertThat(processorValueCollector, equalTo(expectedCollectedProcessorValues));
 
-        streams.close(5, TimeUnit.SECONDS);
+        streams.close(ofSeconds(5));
     }
 
 
@@ -431,4 +432,4 @@ public class RepartitionOptimizingIntegrationTest {
                                                                 + "    Sink: KSTREAM-SINK-0000000028 (topic: outputTopic_2)\n"
                                                                 + "      <-- KTABLE-TOSTREAM-0000000027\n\n";
 
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionWithMergeOptimizingIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionWithMergeOptimizingIntegrationTest.java
index 29901a6..bf65264 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionWithMergeOptimizingIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/RepartitionWithMergeOptimizingIntegrationTest.java
@@ -18,6 +18,7 @@
 package org.apache.kafka.streams.integration;
 
 
+import java.time.Duration;
 import kafka.utils.MockTime;
 import org.apache.kafka.common.serialization.LongDeserializer;
 import org.apache.kafka.common.serialization.Serdes;
@@ -46,7 +47,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -167,7 +167,7 @@ public class RepartitionWithMergeOptimizingIntegrationTest {
         final List<KeyValue<String, String>> expectedStringCountKeyValues = Arrays.asList(KeyValue.pair("A", "6"), KeyValue.pair("B", "6"), KeyValue.pair("C", "6"));
         IntegrationTestUtils.waitUntilFinalKeyValueRecordsReceived(consumerConfig2, COUNT_STRING_TOPIC, expectedStringCountKeyValues);
 
-        streams.close(5, TimeUnit.SECONDS);
+        streams.close(Duration.ofSeconds(5));
     }
 
 
@@ -293,4 +293,4 @@ public class RepartitionWithMergeOptimizingIntegrationTest {
                                                                 + "    Sink: KSTREAM-SINK-0000000019 (topic: outputTopic_1)\n"
                                                                 + "      <-- KSTREAM-MAPVALUES-0000000018\n\n";
 
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java
index 5eb4fc7..3fd8220 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.integration;
 
+import java.time.Duration;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
@@ -113,7 +114,7 @@ public class RestoreIntegrationTest {
     @After
     public void shutdown() {
         if (kafkaStreams != null) {
-            kafkaStreams.close(30, TimeUnit.SECONDS);
+            kafkaStreams.close(Duration.ofSeconds(30));
         }
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java
index 38bc7a6..646185e 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java
@@ -32,6 +32,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
+import static java.time.Duration.ofSeconds;
 
 /**
  * Tests all available joins of Kafka Streams DSL.
@@ -79,7 +80,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
             Arrays.asList("D-a", "D-b", "D-c", "D-d")
         );
 
-        leftStream.join(rightStream, valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC);
+        leftStream.join(rightStream, valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC);
 
         runTest(expectedResult);
     }
@@ -109,7 +110,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
         leftStream.map(MockMapper.<Long, String>noOpKeyValueMapper())
                 .join(rightStream.flatMap(MockMapper.<Long, String>noOpFlatKeyValueMapper())
                                  .selectKey(MockMapper.<Long, String>selectKeyKeyValueMapper()),
-                       valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC);
+                       valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC);
 
         runTest(expectedResult);
     }
@@ -136,7 +137,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
             Arrays.asList("D-a", "D-b", "D-c", "D-d")
         );
 
-        leftStream.leftJoin(rightStream, valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC);
+        leftStream.leftJoin(rightStream, valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC);
 
         runTest(expectedResult);
     }
@@ -166,7 +167,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
         leftStream.map(MockMapper.<Long, String>noOpKeyValueMapper())
                 .leftJoin(rightStream.flatMap(MockMapper.<Long, String>noOpFlatKeyValueMapper())
                                      .selectKey(MockMapper.<Long, String>selectKeyKeyValueMapper()),
-                        valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC);
+                        valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC);
 
         runTest(expectedResult);
     }
@@ -193,7 +194,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
             Arrays.asList("D-a", "D-b", "D-c", "D-d")
         );
 
-        leftStream.outerJoin(rightStream, valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC);
+        leftStream.outerJoin(rightStream, valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC);
 
         runTest(expectedResult);
     }
@@ -223,7 +224,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
         leftStream.map(MockMapper.<Long, String>noOpKeyValueMapper())
                 .outerJoin(rightStream.flatMap(MockMapper.<Long, String>noOpFlatKeyValueMapper())
                                 .selectKey(MockMapper.<Long, String>selectKeyKeyValueMapper()),
-                        valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC);
+                        valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC);
 
         runTest(expectedResult);
     }
@@ -254,8 +255,8 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
                         "D-c-b", "D-c-c", "D-c-d", "D-d-a", "D-d-b", "D-d-c", "D-d-d")
         );
 
-        leftStream.join(rightStream, valueJoiner, JoinWindows.of(10000))
-                .join(rightStream, valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC);
+        leftStream.join(rightStream, valueJoiner, JoinWindows.of(ofSeconds(10)))
+                .join(rightStream, valueJoiner, JoinWindows.of(ofSeconds(10))).to(OUTPUT_TOPIC);
 
         runTest(expectedResult);
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java
index 208f1eb..94bc057 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java
@@ -434,7 +434,7 @@ public class SuppressionIntegrationTest {
                     Consumed.with(STRING_SERDE, STRING_SERDE)
             )
             .groupBy((String k1, String v1) -> k1, Grouped.with(STRING_SERDE, STRING_SERDE))
-            .windowedBy(TimeWindows.of(scaledTime(2L)).grace(scaledTime(1L)))
+            .windowedBy(TimeWindows.of(ofMillis(scaledTime(2L))).grace(ofMillis(scaledTime(1L))))
             .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("counts").withCachingDisabled().withLoggingDisabled());
 
         valueCounts
@@ -534,4 +534,4 @@ public class SuppressionIntegrationTest {
         IntegrationTestUtils.verifyKeyValueTimestamps(properties, topic, keyValueTimestamps);
 
     }
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java
index de635b4..1b3c1f1 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java
@@ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream;
 
 import org.junit.Test;
 
+import static java.time.Duration.ofMillis;
 import static org.apache.kafka.streams.EqualityCheck.verifyEquality;
 import static org.apache.kafka.streams.EqualityCheck.verifyInEquality;
 import static org.junit.Assert.assertEquals;
@@ -31,29 +32,29 @@ public class JoinWindowsTest {
 
     @Test
     public void validWindows() {
-        JoinWindows.of(ANY_OTHER_SIZE)   // [ -anyOtherSize ; anyOtherSize ]
-                   .before(ANY_SIZE)                    // [ -anySize ; anyOtherSize ]
-                   .before(0)                          // [ 0 ; anyOtherSize ]
-                   .before(-ANY_SIZE)                   // [ anySize ; anyOtherSize ]
-                   .before(-ANY_OTHER_SIZE);             // [ anyOtherSize ; anyOtherSize ]
-
-        JoinWindows.of(ANY_OTHER_SIZE)   // [ -anyOtherSize ; anyOtherSize ]
-                   .after(ANY_SIZE)                     // [ -anyOtherSize ; anySize ]
-                   .after(0)                           // [ -anyOtherSize ; 0 ]
-                   .after(-ANY_SIZE)                    // [ -anyOtherSize ; -anySize ]
-                   .after(-ANY_OTHER_SIZE);              // [ -anyOtherSize ; -anyOtherSize ]
+        JoinWindows.of(ofMillis(ANY_OTHER_SIZE))   // [ -anyOtherSize ; anyOtherSize ]
+                   .before(ofMillis(ANY_SIZE))                    // [ -anySize ; anyOtherSize ]
+                   .before(ofMillis(0))                          // [ 0 ; anyOtherSize ]
+                   .before(ofMillis(-ANY_SIZE))                   // [ anySize ; anyOtherSize ]
+                   .before(ofMillis(-ANY_OTHER_SIZE));             // [ anyOtherSize ; anyOtherSize ]
+
+        JoinWindows.of(ofMillis(ANY_OTHER_SIZE))   // [ -anyOtherSize ; anyOtherSize ]
+                   .after(ofMillis(ANY_SIZE))                     // [ -anyOtherSize ; anySize ]
+                   .after(ofMillis(0))                           // [ -anyOtherSize ; 0 ]
+                   .after(ofMillis(-ANY_SIZE))                    // [ -anyOtherSize ; -anySize ]
+                   .after(ofMillis(-ANY_OTHER_SIZE));              // [ -anyOtherSize ; -anyOtherSize ]
     }
 
     @Test(expected = IllegalArgumentException.class)
     public void timeDifferenceMustNotBeNegative() {
-        JoinWindows.of(-1);
+        JoinWindows.of(ofMillis(-1));
     }
 
     @Test
     public void endTimeShouldNotBeBeforeStart() {
-        final JoinWindows windowSpec = JoinWindows.of(ANY_SIZE);
+        final JoinWindows windowSpec = JoinWindows.of(ofMillis(ANY_SIZE));
         try {
-            windowSpec.after(-ANY_SIZE - 1);
+            windowSpec.after(ofMillis(-ANY_SIZE - 1));
             fail("window end time should not be before window start time");
         } catch (final IllegalArgumentException e) {
             // expected
@@ -62,9 +63,9 @@ public class JoinWindowsTest {
 
     @Test
     public void startTimeShouldNotBeAfterEnd() {
-        final JoinWindows windowSpec = JoinWindows.of(ANY_SIZE);
+        final JoinWindows windowSpec = JoinWindows.of(ofMillis(ANY_SIZE));
         try {
-            windowSpec.before(-ANY_SIZE - 1);
+            windowSpec.before(ofMillis(-ANY_SIZE - 1));
             fail("window start time should not be after window end time");
         } catch (final IllegalArgumentException e) {
             // expected
@@ -74,7 +75,7 @@ public class JoinWindowsTest {
     @Deprecated
     @Test
     public void untilShouldSetMaintainDuration() {
-        final JoinWindows windowSpec = JoinWindows.of(ANY_SIZE);
+        final JoinWindows windowSpec = JoinWindows.of(ofMillis(ANY_SIZE));
         final long windowSize = windowSpec.size();
         assertEquals(windowSize, windowSpec.until(windowSize).maintainMs());
     }
@@ -82,7 +83,7 @@ public class JoinWindowsTest {
     @Deprecated
     @Test
     public void retentionTimeMustNoBeSmallerThanWindowSize() {
-        final JoinWindows windowSpec = JoinWindows.of(ANY_SIZE);
+        final JoinWindows windowSpec = JoinWindows.of(ofMillis(ANY_SIZE));
         final long windowSize = windowSpec.size();
         try {
             windowSpec.until(windowSize - 1);
@@ -94,10 +95,10 @@ public class JoinWindowsTest {
 
     @Test
     public void gracePeriodShouldEnforceBoundaries() {
-        JoinWindows.of(3L).grace(0L);
+        JoinWindows.of(ofMillis(3L)).grace(ofMillis(0L));
 
         try {
-            JoinWindows.of(3L).grace(-1L);
+            JoinWindows.of(ofMillis(3L)).grace(ofMillis(-1L));
             fail("should not accept negatives");
         } catch (final IllegalArgumentException e) {
             //expected
@@ -106,58 +107,58 @@ public class JoinWindowsTest {
 
     @Test
     public void equalsAndHashcodeShouldBeValidForPositiveCases() {
-        verifyEquality(JoinWindows.of(3), JoinWindows.of(3));
+        verifyEquality(JoinWindows.of(ofMillis(3)), JoinWindows.of(ofMillis(3)));
 
-        verifyEquality(JoinWindows.of(3).after(2), JoinWindows.of(3).after(2));
+        verifyEquality(JoinWindows.of(ofMillis(3)).after(ofMillis(2)), JoinWindows.of(ofMillis(3)).after(ofMillis(2)));
 
-        verifyEquality(JoinWindows.of(3).before(2), JoinWindows.of(3).before(2));
+        verifyEquality(JoinWindows.of(ofMillis(3)).before(ofMillis(2)), JoinWindows.of(ofMillis(3)).before(ofMillis(2)));
 
-        verifyEquality(JoinWindows.of(3).grace(2), JoinWindows.of(3).grace(2));
+        verifyEquality(JoinWindows.of(ofMillis(3)).grace(ofMillis(2)), JoinWindows.of(ofMillis(3)).grace(ofMillis(2)));
 
-        verifyEquality(JoinWindows.of(3).until(60), JoinWindows.of(3).until(60));
+        verifyEquality(JoinWindows.of(ofMillis(3)).until(60), JoinWindows.of(ofMillis(3)).until(60));
 
         verifyEquality(
-            JoinWindows.of(3).before(1).after(2).grace(3).until(60),
-            JoinWindows.of(3).before(1).after(2).grace(3).until(60)
+            JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60),
+            JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60)
         );
         // JoinWindows is a little weird in that before and after set the same fields as of.
         verifyEquality(
-            JoinWindows.of(9).before(1).after(2).grace(3).until(60),
-            JoinWindows.of(3).before(1).after(2).grace(3).until(60)
+            JoinWindows.of(ofMillis(9)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60),
+            JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60)
         );
     }
 
     @Test
     public void equalsAndHashcodeShouldBeValidForNegativeCases() {
-        verifyInEquality(JoinWindows.of(9), JoinWindows.of(3));
+        verifyInEquality(JoinWindows.of(ofMillis(9)), JoinWindows.of(ofMillis(3)));
 
-        verifyInEquality(JoinWindows.of(3).after(9), JoinWindows.of(3).after(2));
+        verifyInEquality(JoinWindows.of(ofMillis(3)).after(ofMillis(9)), JoinWindows.of(ofMillis(3)).after(ofMillis(2)));
 
-        verifyInEquality(JoinWindows.of(3).before(9), JoinWindows.of(3).before(2));
+        verifyInEquality(JoinWindows.of(ofMillis(3)).before(ofMillis(9)), JoinWindows.of(ofMillis(3)).before(ofMillis(2)));
 
-        verifyInEquality(JoinWindows.of(3).grace(9), JoinWindows.of(3).grace(2));
+        verifyInEquality(JoinWindows.of(ofMillis(3)).grace(ofMillis(9)), JoinWindows.of(ofMillis(3)).grace(ofMillis(2)));
 
-        verifyInEquality(JoinWindows.of(3).until(90), JoinWindows.of(3).until(60));
+        verifyInEquality(JoinWindows.of(ofMillis(3)).until(90), JoinWindows.of(ofMillis(3)).until(60));
 
 
         verifyInEquality(
-            JoinWindows.of(3).before(9).after(2).grace(3).until(60),
-            JoinWindows.of(3).before(1).after(2).grace(3).until(60)
+            JoinWindows.of(ofMillis(3)).before(ofMillis(9)).after(ofMillis(2)).grace(ofMillis(3)).until(60),
+            JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60)
         );
 
         verifyInEquality(
-            JoinWindows.of(3).before(1).after(9).grace(3).until(60),
-            JoinWindows.of(3).before(1).after(2).grace(3).until(60)
+            JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(9)).grace(ofMillis(3)).until(60),
+            JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60)
         );
 
         verifyInEquality(
-            JoinWindows.of(3).before(1).after(2).grace(9).until(60),
-            JoinWindows.of(3).before(1).after(2).grace(3).until(60)
+            JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(9)).until(60),
+            JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60)
         );
 
         verifyInEquality(
-            JoinWindows.of(3).before(1).after(2).grace(3).until(90),
-            JoinWindows.of(3).before(1).after(2).grace(3).until(60)
+            JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(90),
+            JoinWindows.of(ofMillis(3)).before(ofMillis(1)).after(ofMillis(2)).grace(ofMillis(3)).until(60)
         );
     }
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java
index 9f99be4..6fc0cac 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java
@@ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream;
 
 import org.junit.Test;
 
+import static java.time.Duration.ofMillis;
 import static org.apache.kafka.streams.EqualityCheck.verifyEquality;
 import static org.apache.kafka.streams.EqualityCheck.verifyInEquality;
 import static org.junit.Assert.assertEquals;
@@ -28,23 +29,23 @@ public class SessionWindowsTest {
     @Test
     public void shouldSetWindowGap() {
         final long anyGap = 42L;
-        assertEquals(anyGap, SessionWindows.with(anyGap).inactivityGap());
+        assertEquals(anyGap, SessionWindows.with(ofMillis(anyGap)).inactivityGap());
     }
 
     @Deprecated
     @Test
     public void shouldSetWindowRetentionTime() {
         final long anyRetentionTime = 42L;
-        assertEquals(anyRetentionTime, SessionWindows.with(1).until(anyRetentionTime).maintainMs());
+        assertEquals(anyRetentionTime, SessionWindows.with(ofMillis(1)).until(anyRetentionTime).maintainMs());
     }
 
 
     @Test
     public void gracePeriodShouldEnforceBoundaries() {
-        SessionWindows.with(3L).grace(0L);
+        SessionWindows.with(ofMillis(3L)).grace(ofMillis(0));
 
         try {
-            SessionWindows.with(3L).grace(-1L);
+            SessionWindows.with(ofMillis(3L)).grace(ofMillis(-1L));
             fail("should not accept negatives");
         } catch (final IllegalArgumentException e) {
             //expected
@@ -53,25 +54,25 @@ public class SessionWindowsTest {
 
     @Test(expected = IllegalArgumentException.class)
     public void windowSizeMustNotBeNegative() {
-        SessionWindows.with(-1);
+        SessionWindows.with(ofMillis(-1));
     }
 
     @Test(expected = IllegalArgumentException.class)
     public void windowSizeMustNotBeZero() {
-        SessionWindows.with(0);
+        SessionWindows.with(ofMillis(0));
     }
 
     @SuppressWarnings("deprecation") // specifically testing deprecated apis
     @Test
     public void retentionTimeShouldBeGapIfGapIsLargerThanDefaultRetentionTime() {
-        final long windowGap = 2 * SessionWindows.with(1).maintainMs();
-        assertEquals(windowGap, SessionWindows.with(windowGap).maintainMs());
+        final long windowGap = 2 * SessionWindows.with(ofMillis(1)).maintainMs();
+        assertEquals(windowGap, SessionWindows.with(ofMillis(windowGap)).maintainMs());
     }
 
     @Deprecated
     @Test
     public void retentionTimeMustNotBeNegative() {
-        final SessionWindows windowSpec = SessionWindows.with(42);
+        final SessionWindows windowSpec = SessionWindows.with(ofMillis(42));
         try {
             windowSpec.until(41);
             fail("should not accept retention time smaller than gap");
@@ -82,27 +83,27 @@ public class SessionWindowsTest {
 
     @Test
     public void equalsAndHashcodeShouldBeValidForPositiveCases() {
-        verifyEquality(SessionWindows.with(1), SessionWindows.with(1));
+        verifyEquality(SessionWindows.with(ofMillis(1)), SessionWindows.with(ofMillis(1)));
 
-        verifyEquality(SessionWindows.with(1).grace(6), SessionWindows.with(1).grace(6));
+        verifyEquality(SessionWindows.with(ofMillis(1)).grace(ofMillis(6)), SessionWindows.with(ofMillis(1)).grace(ofMillis(6)));
 
-        verifyEquality(SessionWindows.with(1).until(7), SessionWindows.with(1).until(7));
+        verifyEquality(SessionWindows.with(ofMillis(1)).until(7), SessionWindows.with(ofMillis(1)).until(7));
 
-        verifyEquality(SessionWindows.with(1).grace(6).until(7), SessionWindows.with(1).grace(6).until(7));
+        verifyEquality(SessionWindows.with(ofMillis(1)).grace(ofMillis(6)).until(7), SessionWindows.with(ofMillis(1)).grace(ofMillis(6)).until(7));
     }
 
     @Test
     public void equalsAndHashcodeShouldBeValidForNegativeCases() {
-        verifyInEquality(SessionWindows.with(9), SessionWindows.with(1));
+        verifyInEquality(SessionWindows.with(ofMillis(9)), SessionWindows.with(ofMillis(1)));
 
-        verifyInEquality(SessionWindows.with(1).grace(9), SessionWindows.with(1).grace(6));
+        verifyInEquality(SessionWindows.with(ofMillis(1)).grace(ofMillis(9)), SessionWindows.with(ofMillis(1)).grace(ofMillis(6)));
 
-        verifyInEquality(SessionWindows.with(1).until(9), SessionWindows.with(1).until(7));
+        verifyInEquality(SessionWindows.with(ofMillis(1)).until(9), SessionWindows.with(ofMillis(1)).until(7));
 
-        verifyInEquality(SessionWindows.with(2).grace(6).until(7), SessionWindows.with(1).grace(6).until(7));
+        verifyInEquality(SessionWindows.with(ofMillis(2)).grace(ofMillis(6)).until(7), SessionWindows.with(ofMillis(1)).grace(ofMillis(6)).until(7));
 
-        verifyInEquality(SessionWindows.with(1).grace(0).until(7), SessionWindows.with(1).grace(6).until(7));
+        verifyInEquality(SessionWindows.with(ofMillis(1)).grace(ofMillis(0)).until(7), SessionWindows.with(ofMillis(1)).grace(ofMillis(6)).until(7));
 
-        verifyInEquality(SessionWindows.with(1).grace(6).until(70), SessionWindows.with(1).grace(6).until(7));
+        verifyInEquality(SessionWindows.with(ofMillis(1)).grace(ofMillis(6)).until(70), SessionWindows.with(ofMillis(1)).grace(ofMillis(6)).until(7));
     }
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java
index 9010bb2..b87a992 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java
@@ -21,6 +21,7 @@ import org.junit.Test;
 
 import java.util.Map;
 
+import static java.time.Duration.ofMillis;
 import static org.apache.kafka.streams.EqualityCheck.verifyEquality;
 import static org.apache.kafka.streams.EqualityCheck.verifyInEquality;
 import static org.junit.Assert.assertEquals;
@@ -33,43 +34,43 @@ public class TimeWindowsTest {
 
     @Test
     public void shouldSetWindowSize() {
-        assertEquals(ANY_SIZE, TimeWindows.of(ANY_SIZE).sizeMs);
+        assertEquals(ANY_SIZE, TimeWindows.of(ofMillis(ANY_SIZE)).sizeMs);
     }
 
     @Test
     public void shouldSetWindowAdvance() {
         final long anyAdvance = 4;
-        assertEquals(anyAdvance, TimeWindows.of(ANY_SIZE).advanceBy(anyAdvance).advanceMs);
+        assertEquals(anyAdvance, TimeWindows.of(ofMillis(ANY_SIZE)).advanceBy(ofMillis(anyAdvance)).advanceMs);
     }
 
     @SuppressWarnings("deprecation") // specifically testing deprecated APIs
     @Test
     public void shouldSetWindowRetentionTime() {
-        assertEquals(ANY_SIZE, TimeWindows.of(ANY_SIZE).until(ANY_SIZE).maintainMs());
+        assertEquals(ANY_SIZE, TimeWindows.of(ofMillis(ANY_SIZE)).until(ANY_SIZE).maintainMs());
     }
 
     @SuppressWarnings("deprecation") // specifically testing deprecated APIs
     @Test
     public void shouldUseWindowSizeAsRentitionTimeIfWindowSizeIsLargerThanDefaultRetentionTime() {
-        final long windowSize = 2 * TimeWindows.of(1).maintainMs();
-        assertEquals(windowSize, TimeWindows.of(windowSize).maintainMs());
+        final long windowSize = 2 * TimeWindows.of(ofMillis(1)).maintainMs();
+        assertEquals(windowSize, TimeWindows.of(ofMillis(windowSize)).maintainMs());
     }
 
     @Test(expected = IllegalArgumentException.class)
     public void windowSizeMustNotBeZero() {
-        TimeWindows.of(0);
+        TimeWindows.of(ofMillis(0));
     }
 
     @Test(expected = IllegalArgumentException.class)
     public void windowSizeMustNotBeNegative() {
-        TimeWindows.of(-1);
+        TimeWindows.of(ofMillis(-1));
     }
 
     @Test
     public void advanceIntervalMustNotBeZero() {
-        final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE);
+        final TimeWindows windowSpec = TimeWindows.of(ofMillis(ANY_SIZE));
         try {
-            windowSpec.advanceBy(0);
+            windowSpec.advanceBy(ofMillis(0));
             fail("should not accept zero advance parameter");
         } catch (final IllegalArgumentException e) {
             // expected
@@ -78,9 +79,9 @@ public class TimeWindowsTest {
 
     @Test
     public void advanceIntervalMustNotBeNegative() {
-        final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE);
+        final TimeWindows windowSpec = TimeWindows.of(ofMillis(ANY_SIZE));
         try {
-            windowSpec.advanceBy(-1);
+            windowSpec.advanceBy(ofMillis(-1));
             fail("should not accept negative advance parameter");
         } catch (final IllegalArgumentException e) {
             // expected
@@ -90,9 +91,9 @@ public class TimeWindowsTest {
     @Deprecated
     @Test
     public void advanceIntervalMustNotBeLargerThanWindowSize() {
-        final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE);
+        final TimeWindows windowSpec = TimeWindows.of(ofMillis(ANY_SIZE));
         try {
-            windowSpec.advanceBy(ANY_SIZE + 1);
+            windowSpec.advanceBy(ofMillis(ANY_SIZE + 1));
             fail("should not accept advance greater than window size");
         } catch (final IllegalArgumentException e) {
             // expected
@@ -102,7 +103,7 @@ public class TimeWindowsTest {
     @Deprecated
     @Test
     public void retentionTimeMustNoBeSmallerThanWindowSize() {
-        final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE);
+        final TimeWindows windowSpec = TimeWindows.of(ofMillis(ANY_SIZE));
         try {
             windowSpec.until(ANY_SIZE - 1);
             fail("should not accept retention time smaller than window size");
@@ -113,10 +114,10 @@ public class TimeWindowsTest {
 
     @Test
     public void gracePeriodShouldEnforceBoundaries() {
-        TimeWindows.of(3L).grace(0L);
+        TimeWindows.of(ofMillis(3L)).grace(ofMillis(0L));
 
         try {
-            TimeWindows.of(3L).grace(-1L);
+            TimeWindows.of(ofMillis(3L)).grace(ofMillis(-1L));
             fail("should not accept negatives");
         } catch (final IllegalArgumentException e) {
             //expected
@@ -125,7 +126,7 @@ public class TimeWindowsTest {
 
     @Test
     public void shouldComputeWindowsForHoppingWindows() {
-        final TimeWindows windows = TimeWindows.of(12L).advanceBy(5L);
+        final TimeWindows windows = TimeWindows.of(ofMillis(12L)).advanceBy(ofMillis(5L));
         final Map<Long, TimeWindow> matched = windows.windowsFor(21L);
         assertEquals(12L / 5L + 1, matched.size());
         assertEquals(new TimeWindow(10L, 22L), matched.get(10L));
@@ -135,7 +136,7 @@ public class TimeWindowsTest {
 
     @Test
     public void shouldComputeWindowsForBarelyOverlappingHoppingWindows() {
-        final TimeWindows windows = TimeWindows.of(6L).advanceBy(5L);
+        final TimeWindows windows = TimeWindows.of(ofMillis(6L)).advanceBy(ofMillis(5L));
         final Map<Long, TimeWindow> matched = windows.windowsFor(7L);
         assertEquals(1, matched.size());
         assertEquals(new TimeWindow(5L, 11L), matched.get(5L));
@@ -143,7 +144,7 @@ public class TimeWindowsTest {
 
     @Test
     public void shouldComputeWindowsForTumblingWindows() {
-        final TimeWindows windows = TimeWindows.of(12L);
+        final TimeWindows windows = TimeWindows.of(ofMillis(12L));
         final Map<Long, TimeWindow> matched = windows.windowsFor(21L);
         assertEquals(1, matched.size());
         assertEquals(new TimeWindow(12L, 24L), matched.get(12L));
@@ -152,49 +153,49 @@ public class TimeWindowsTest {
 
     @Test
     public void equalsAndHashcodeShouldBeValidForPositiveCases() {
-        verifyEquality(TimeWindows.of(3), TimeWindows.of(3));
+        verifyEquality(TimeWindows.of(ofMillis(3)), TimeWindows.of(ofMillis(3)));
 
-        verifyEquality(TimeWindows.of(3).advanceBy(1), TimeWindows.of(3).advanceBy(1));
+        verifyEquality(TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1)), TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1)));
 
-        verifyEquality(TimeWindows.of(3).grace(1), TimeWindows.of(3).grace(1));
+        verifyEquality(TimeWindows.of(ofMillis(3)).grace(ofMillis(1)), TimeWindows.of(ofMillis(3)).grace(ofMillis(1)));
 
-        verifyEquality(TimeWindows.of(3).until(4), TimeWindows.of(3).until(4));
+        verifyEquality(TimeWindows.of(ofMillis(3)).until(4), TimeWindows.of(ofMillis(3)).until(4));
 
         verifyEquality(
-            TimeWindows.of(3).advanceBy(1).grace(1).until(4),
-            TimeWindows.of(3).advanceBy(1).grace(1).until(4)
+            TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1)).grace(ofMillis(1)).until(4),
+            TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1)).grace(ofMillis(1)).until(4)
         );
     }
 
     @Test
     public void equalsAndHashcodeShouldBeValidForNegativeCases() {
-        verifyInEquality(TimeWindows.of(9), TimeWindows.of(3));
+        verifyInEquality(TimeWindows.of(ofMillis(9)), TimeWindows.of(ofMillis(3)));
 
-        verifyInEquality(TimeWindows.of(3).advanceBy(2), TimeWindows.of(3).advanceBy(1));
+        verifyInEquality(TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)), TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1)));
 
-        verifyInEquality(TimeWindows.of(3).grace(2), TimeWindows.of(3).grace(1));
+        verifyInEquality(TimeWindows.of(ofMillis(3)).grace(ofMillis(2)), TimeWindows.of(ofMillis(3)).grace(ofMillis(1)));
 
-        verifyInEquality(TimeWindows.of(3).until(9), TimeWindows.of(3).until(4));
+        verifyInEquality(TimeWindows.of(ofMillis(3)).until(9), TimeWindows.of(ofMillis(3)).until(4));
 
 
         verifyInEquality(
-            TimeWindows.of(4).advanceBy(2).grace(2).until(4),
-            TimeWindows.of(3).advanceBy(2).grace(2).until(4)
+            TimeWindows.of(ofMillis(4)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4),
+            TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4)
         );
 
         verifyInEquality(
-            TimeWindows.of(3).advanceBy(1).grace(2).until(4),
-            TimeWindows.of(3).advanceBy(2).grace(2).until(4)
+            TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(1)).grace(ofMillis(2)).until(4),
+            TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4)
         );
 
         assertNotEquals(
-            TimeWindows.of(3).advanceBy(2).grace(1).until(4),
-            TimeWindows.of(3).advanceBy(2).grace(2).until(4)
+            TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(1)).until(4),
+            TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4)
         );
 
         assertNotEquals(
-            TimeWindows.of(3).advanceBy(2).grace(2).until(9),
-            TimeWindows.of(3).advanceBy(2).grace(2).until(4)
+            TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(9),
+            TimeWindows.of(ofMillis(3)).advanceBy(ofMillis(2)).grace(ofMillis(2)).until(4)
         );
     }
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java
index a140654..023e83f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java
@@ -21,6 +21,7 @@ import org.junit.Test;
 
 import java.util.Map;
 
+import static java.time.Instant.ofEpochMilli;
 import static org.apache.kafka.streams.EqualityCheck.verifyEquality;
 import static org.apache.kafka.streams.EqualityCheck.verifyInEquality;
 import static org.junit.Assert.assertEquals;
@@ -33,12 +34,12 @@ public class UnlimitedWindowsTest {
 
     @Test
     public void shouldSetWindowStartTime() {
-        assertEquals(anyStartTime, UnlimitedWindows.of().startOn(anyStartTime).startMs);
+        assertEquals(anyStartTime, UnlimitedWindows.of().startOn(ofEpochMilli(anyStartTime)).startMs);
     }
 
     @Test(expected = IllegalArgumentException.class)
     public void startTimeMustNotBeNegative() {
-        UnlimitedWindows.of().startOn(-1);
+        UnlimitedWindows.of().startOn(ofEpochMilli(-1));
     }
 
     @Test
@@ -54,7 +55,7 @@ public class UnlimitedWindowsTest {
 
     @Test
     public void shouldIncludeRecordsThatHappenedOnWindowStart() {
-        final UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime);
+        final UnlimitedWindows w = UnlimitedWindows.of().startOn(ofEpochMilli(anyStartTime));
         final Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(w.startMs);
         assertEquals(1, matchedWindows.size());
         assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime));
@@ -62,7 +63,7 @@ public class UnlimitedWindowsTest {
 
     @Test
     public void shouldIncludeRecordsThatHappenedAfterWindowStart() {
-        final UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime);
+        final UnlimitedWindows w = UnlimitedWindows.of().startOn(ofEpochMilli(anyStartTime));
         final long timestamp = w.startMs + 1;
         final Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(timestamp);
         assertEquals(1, matchedWindows.size());
@@ -71,7 +72,7 @@ public class UnlimitedWindowsTest {
 
     @Test
     public void shouldExcludeRecordsThatHappenedBeforeWindowStart() {
-        final UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime);
+        final UnlimitedWindows w = UnlimitedWindows.of().startOn(ofEpochMilli(anyStartTime));
         final long timestamp = w.startMs - 1;
         final Map<Long, UnlimitedWindow> matchedWindows = w.windowsFor(timestamp);
         assertTrue(matchedWindows.isEmpty());
@@ -81,13 +82,13 @@ public class UnlimitedWindowsTest {
     public void equalsAndHashcodeShouldBeValidForPositiveCases() {
         verifyEquality(UnlimitedWindows.of(), UnlimitedWindows.of());
 
-        verifyEquality(UnlimitedWindows.of().startOn(1), UnlimitedWindows.of().startOn(1));
+        verifyEquality(UnlimitedWindows.of().startOn(ofEpochMilli(1)), UnlimitedWindows.of().startOn(ofEpochMilli(1)));
 
     }
 
     @Test
     public void equalsAndHashcodeShouldBeValidForNegativeCases() {
-        verifyInEquality(UnlimitedWindows.of().startOn(9), UnlimitedWindows.of().startOn(1));
+        verifyInEquality(UnlimitedWindows.of().startOn(ofEpochMilli(9)), UnlimitedWindows.of().startOn(ofEpochMilli(1)));
     }
 
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
index a1f8b27..18f36aa 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
@@ -59,6 +59,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
+import static java.time.Duration.ofMillis;
 import static org.apache.kafka.test.StreamsTestUtils.getMetricByName;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.hasItem;
@@ -96,7 +97,7 @@ public class KGroupedStreamImplTest {
 
     @Test(expected = NullPointerException.class)
     public void shouldNotHaveNullReducerWithWindowedReduce() {
-        groupedStream.windowedBy(TimeWindows.of(10)).reduce(null, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("store"));
+        groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).reduce(null, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("store"));
     }
 
     @Test(expected = NullPointerException.class)
@@ -106,7 +107,7 @@ public class KGroupedStreamImplTest {
 
     @Test(expected = InvalidTopicException.class)
     public void shouldNotHaveInvalidStoreNameWithWindowedReduce() {
-        groupedStream.windowedBy(TimeWindows.of(10)).reduce(MockReducer.STRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as(INVALID_STORE_NAME));
+        groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).reduce(MockReducer.STRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as(INVALID_STORE_NAME));
     }
 
     @Test(expected = NullPointerException.class)
@@ -126,12 +127,12 @@ public class KGroupedStreamImplTest {
 
     @Test(expected = NullPointerException.class)
     public void shouldNotHaveNullInitializerOnWindowedAggregate() {
-        groupedStream.windowedBy(TimeWindows.of(10)).aggregate(null, MockAggregator.TOSTRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("store"));
+        groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).aggregate(null, MockAggregator.TOSTRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("store"));
     }
 
     @Test(expected = NullPointerException.class)
     public void shouldNotHaveNullAdderOnWindowedAggregate() {
-        groupedStream.windowedBy(TimeWindows.of(10)).aggregate(MockInitializer.STRING_INIT, null, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("store"));
+        groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).aggregate(MockInitializer.STRING_INIT, null, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("store"));
     }
 
     @Test(expected = NullPointerException.class)
@@ -141,7 +142,7 @@ public class KGroupedStreamImplTest {
 
     @Test(expected = InvalidTopicException.class)
     public void shouldNotHaveInvalidStoreNameOnWindowedAggregate() {
-        groupedStream.windowedBy(TimeWindows.of(10)).aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as(INVALID_STORE_NAME));
+        groupedStream.windowedBy(TimeWindows.of(ofMillis(10))).aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as(INVALID_STORE_NAME));
     }
 
     private void doAggregateSessionWindows(final Map<Windowed<String>, Integer> results) {
@@ -161,7 +162,7 @@ public class KGroupedStreamImplTest {
     @Test
     public void shouldAggregateSessionWindows() {
         final Map<Windowed<String>, Integer> results = new HashMap<>();
-        final KTable<Windowed<String>, Integer> table = groupedStream.windowedBy(SessionWindows.with(30)).aggregate(new Initializer<Integer>() {
+        final KTable<Windowed<String>, Integer> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(new Initializer<Integer>() {
             @Override
             public Integer apply() {
                 return 0;
@@ -191,7 +192,7 @@ public class KGroupedStreamImplTest {
     @Test
     public void shouldAggregateSessionWindowsWithInternalStoreName() {
         final Map<Windowed<String>, Integer> results = new HashMap<>();
-        final KTable<Windowed<String>, Integer> table = groupedStream.windowedBy(SessionWindows.with(30)).aggregate(new Initializer<Integer>() {
+        final KTable<Windowed<String>, Integer> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(new Initializer<Integer>() {
             @Override
             public Integer apply() {
                 return 0;
@@ -234,7 +235,7 @@ public class KGroupedStreamImplTest {
     @Test
     public void shouldCountSessionWindows() {
         final Map<Windowed<String>, Long> results = new HashMap<>();
-        final KTable<Windowed<String>, Long> table = groupedStream.windowedBy(SessionWindows.with(30))
+        final KTable<Windowed<String>, Long> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30)))
                 .count(Materialized.<String, Long, SessionStore<Bytes, byte[]>>as("session-store"));
         table.toStream().foreach(new ForeachAction<Windowed<String>, Long>() {
             @Override
@@ -249,7 +250,7 @@ public class KGroupedStreamImplTest {
     @Test
     public void shouldCountSessionWindowsWithInternalStoreName() {
         final Map<Windowed<String>, Long> results = new HashMap<>();
-        final KTable<Windowed<String>, Long> table = groupedStream.windowedBy(SessionWindows.with(30)).count();
+        final KTable<Windowed<String>, Long> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).count();
         table.toStream().foreach(new ForeachAction<Windowed<String>, Long>() {
             @Override
             public void apply(final Windowed<String> key, final Long value) {
@@ -277,7 +278,7 @@ public class KGroupedStreamImplTest {
     @Test
     public void shouldReduceSessionWindows() {
         final Map<Windowed<String>, String> results = new HashMap<>();
-        final KTable<Windowed<String>, String> table = groupedStream.windowedBy(SessionWindows.with(30))
+        final KTable<Windowed<String>, String> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30)))
                 .reduce(new Reducer<String>() {
                     @Override
                     public String apply(final String value1, final String value2) {
@@ -297,7 +298,7 @@ public class KGroupedStreamImplTest {
     @Test
     public void shouldReduceSessionWindowsWithInternalStoreName() {
         final Map<Windowed<String>, String> results = new HashMap<>();
-        final KTable<Windowed<String>, String> table = groupedStream.windowedBy(SessionWindows.with(30))
+        final KTable<Windowed<String>, String> table = groupedStream.windowedBy(SessionWindows.with(ofMillis(30)))
                 .reduce(new Reducer<String>() {
                     @Override
                     public String apply(final String value1, final String value2) {
@@ -316,7 +317,7 @@ public class KGroupedStreamImplTest {
 
     @Test(expected = NullPointerException.class)
     public void shouldNotAcceptNullReducerWhenReducingSessionWindows() {
-        groupedStream.windowedBy(SessionWindows.with(30)).reduce(null, Materialized.<String, String, SessionStore<Bytes, byte[]>>as("store"));
+        groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).reduce(null, Materialized.<String, String, SessionStore<Bytes, byte[]>>as("store"));
     }
 
     @Test(expected = NullPointerException.class)
@@ -326,17 +327,17 @@ public class KGroupedStreamImplTest {
 
     @Test(expected = InvalidTopicException.class)
     public void shouldNotAcceptInvalidStoreNameWhenReducingSessionWindows() {
-        groupedStream.windowedBy(SessionWindows.with(30)).reduce(MockReducer.STRING_ADDER, Materialized.<String, String, SessionStore<Bytes, byte[]>>as(INVALID_STORE_NAME));
+        groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).reduce(MockReducer.STRING_ADDER, Materialized.<String, String, SessionStore<Bytes, byte[]>>as(INVALID_STORE_NAME));
     }
 
     @Test(expected = NullPointerException.class)
     public void shouldNotAcceptNullStateStoreSupplierWhenReducingSessionWindows() {
-        groupedStream.windowedBy(SessionWindows.with(30)).reduce(null, Materialized.<String, String, SessionStore<Bytes, byte[]>>as(null));
+        groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).reduce(null, Materialized.<String, String, SessionStore<Bytes, byte[]>>as(null));
     }
 
     @Test(expected = NullPointerException.class)
     public void shouldNotAcceptNullInitializerWhenAggregatingSessionWindows() {
-        groupedStream.windowedBy(SessionWindows.with(30)).aggregate(null, MockAggregator.TOSTRING_ADDER, new Merger<String, String>() {
+        groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(null, MockAggregator.TOSTRING_ADDER, new Merger<String, String>() {
             @Override
             public String apply(final String aggKey, final String aggOne, final String aggTwo) {
                 return null;
@@ -346,7 +347,7 @@ public class KGroupedStreamImplTest {
 
     @Test(expected = NullPointerException.class)
     public void shouldNotAcceptNullAggregatorWhenAggregatingSessionWindows() {
-        groupedStream.windowedBy(SessionWindows.with(30)).aggregate(MockInitializer.STRING_INIT, null, new Merger<String, String>() {
+        groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(MockInitializer.STRING_INIT, null, new Merger<String, String>() {
             @Override
             public String apply(final String aggKey, final String aggOne, final String aggTwo) {
                 return null;
@@ -356,7 +357,7 @@ public class KGroupedStreamImplTest {
 
     @Test(expected = NullPointerException.class)
     public void shouldNotAcceptNullSessionMergerWhenAggregatingSessionWindows() {
-        groupedStream.windowedBy(SessionWindows.with(30)).aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER,
+        groupedStream.windowedBy(SessionWindows.with(ofMillis(30))).aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER,
                 null,
                 Materialized.<String, String, SessionStore<Bytes, byte[]>>as("storeName"));
     }
@@ -368,7 +369,7 @@ public class KGroupedStreamImplTest {
 
     @Test
     public void shouldAcceptNullStoreNameWhenAggregatingSessionWindows() {
-        groupedStream.windowedBy(SessionWindows.with(10))
+        groupedStream.windowedBy(SessionWindows.with(ofMillis(10)))
                 .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, new Merger<String, String>() {
                     @Override
                     public String apply(final String aggKey, final String aggOne, final String aggTwo) {
@@ -379,7 +380,7 @@ public class KGroupedStreamImplTest {
 
     @Test(expected = InvalidTopicException.class)
     public void shouldNotAcceptInvalidStoreNameWhenAggregatingSessionWindows() {
-        groupedStream.windowedBy(SessionWindows.with(10))
+        groupedStream.windowedBy(SessionWindows.with(ofMillis(10)))
                 .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, new Merger<String, String>() {
                     @Override
                     public String apply(final String aggKey, final String aggOne, final String aggTwo) {
@@ -558,7 +559,7 @@ public class KGroupedStreamImplTest {
     @Test
     public void shouldCountWindowed() {
         final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>();
-        groupedStream.windowedBy(TimeWindows.of(500L)).count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("aggregate-by-key-windowed"))
+        groupedStream.windowedBy(TimeWindows.of(ofMillis(500L))).count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("aggregate-by-key-windowed"))
             .toStream()
             .foreach(new ForeachAction<Windowed<String>, Long>() {
                 @Override
@@ -573,7 +574,7 @@ public class KGroupedStreamImplTest {
     @Test
     public void shouldCountWindowedWithInternalStoreName() {
         final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>();
-        groupedStream.windowedBy(TimeWindows.of(500L)).count()
+        groupedStream.windowedBy(TimeWindows.of(ofMillis(500L))).count()
             .toStream()
             .foreach(new ForeachAction<Windowed<String>, Long>() {
                 @Override
@@ -584,4 +585,4 @@ public class KGroupedStreamImplTest {
 
         doCountWindowed(results);
     }
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java
index 119a7b7..d033e49 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java
@@ -63,6 +63,7 @@ import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
+import static java.time.Duration.ofMillis;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.core.IsInstanceOf.instanceOf;
 import static org.junit.Assert.assertEquals;
@@ -162,14 +163,14 @@ public class KStreamImplTest {
             public Integer apply(final Integer value1, final Integer value2) {
                 return value1 + value2;
             }
-        }, JoinWindows.of(anyWindowSize), joined);
+        }, JoinWindows.of(ofMillis(anyWindowSize)), joined);
 
         streams2[1].join(streams3[1], new ValueJoiner<Integer, Integer, Integer>() {
             @Override
             public Integer apply(final Integer value1, final Integer value2) {
                 return value1 + value2;
             }
-        }, JoinWindows.of(anyWindowSize), joined);
+        }, JoinWindows.of(ofMillis(anyWindowSize)), joined);
 
         stream4.to("topic-5");
 
@@ -383,7 +384,7 @@ public class KStreamImplTest {
             });
         stream.join(kStream,
                     valueJoiner,
-                    JoinWindows.of(windowSize).until(3 * windowSize),
+                    JoinWindows.of(ofMillis(windowSize)).until(3 * windowSize),
                     Joined.with(Serdes.String(),
                                 Serdes.String(),
                                 Serdes.String()))
@@ -418,7 +419,7 @@ public class KStreamImplTest {
         stream.join(
             kStream,
             valueJoiner,
-            JoinWindows.of(windowSize).grace(3L * windowSize),
+            JoinWindows.of(ofMillis(windowSize)).grace(ofMillis(3L * windowSize)),
             Joined.with(Serdes.String(), Serdes.String(), Serdes.String())
         )
               .to("output-topic", Produced.with(Serdes.String(), Serdes.String()));
@@ -537,12 +538,12 @@ public class KStreamImplTest {
 
     @Test(expected = NullPointerException.class)
     public void shouldNotAllowNullOtherStreamOnJoin() {
-        testStream.join(null, MockValueJoiner.TOSTRING_JOINER, JoinWindows.of(10));
+        testStream.join(null, MockValueJoiner.TOSTRING_JOINER, JoinWindows.of(ofMillis(10)));
     }
 
     @Test(expected = NullPointerException.class)
     public void shouldNotAllowNullValueJoinerOnJoin() {
-        testStream.join(testStream, null, JoinWindows.of(10));
+        testStream.join(testStream, null, JoinWindows.of(ofMillis(10)));
     }
 
     @Test(expected = NullPointerException.class)
@@ -655,12 +656,12 @@ public class KStreamImplTest {
 
     @Test(expected = NullPointerException.class)
     public void shouldThrowNullPointerOnJoinWithStreamWhenJoinedIsNull() {
-        testStream.join(testStream, MockValueJoiner.TOSTRING_JOINER, JoinWindows.of(10), null);
+        testStream.join(testStream, MockValueJoiner.TOSTRING_JOINER, JoinWindows.of(ofMillis(10)), null);
     }
 
     @Test(expected = NullPointerException.class)
     public void shouldThrowNullPointerOnOuterJoinJoinedIsNull() {
-        testStream.outerJoin(testStream, MockValueJoiner.TOSTRING_JOINER, JoinWindows.of(10), null);
+        testStream.outerJoin(testStream, MockValueJoiner.TOSTRING_JOINER, JoinWindows.of(ofMillis(10)), null);
     }
     
     @Test
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java
index 971ee62..f2e3cc9 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java
@@ -41,6 +41,7 @@ import java.util.HashSet;
 import java.util.Properties;
 import java.util.Set;
 
+import static java.time.Duration.ofMillis;
 import static org.apache.kafka.test.StreamsTestUtils.getMetricByName;
 import static org.hamcrest.CoreMatchers.hasItem;
 import static org.junit.Assert.assertEquals;
@@ -71,7 +72,7 @@ public class KStreamKStreamJoinTest {
                     return value1 + value2;
                 }
             },
-            JoinWindows.of(100),
+            JoinWindows.of(ofMillis(100)),
             Joined.with(Serdes.String(), Serdes.Integer(), Serdes.Integer())
         );
 
@@ -101,7 +102,7 @@ public class KStreamKStreamJoinTest {
         joined = stream1.join(
             stream2,
             MockValueJoiner.TOSTRING_JOINER,
-            JoinWindows.of(100),
+            JoinWindows.of(ofMillis(100)),
             Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String()));
         joined.process(supplier);
 
@@ -204,7 +205,7 @@ public class KStreamKStreamJoinTest {
         joined = stream1.outerJoin(
             stream2,
             MockValueJoiner.TOSTRING_JOINER,
-            JoinWindows.of(100),
+            JoinWindows.of(ofMillis(100)),
             Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String()));
         joined.process(supplier);
         final Collection<Set<String>> copartitionGroups = TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
@@ -308,7 +309,7 @@ public class KStreamKStreamJoinTest {
         joined = stream1.join(
             stream2,
             MockValueJoiner.TOSTRING_JOINER,
-            JoinWindows.of(100),
+            JoinWindows.of(ofMillis(100)),
             Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String()));
         joined.process(supplier);
 
@@ -529,7 +530,7 @@ public class KStreamKStreamJoinTest {
         joined = stream1.join(
             stream2,
             MockValueJoiner.TOSTRING_JOINER,
-            JoinWindows.of(0).after(100),
+            JoinWindows.of(ofMillis(0)).after(ofMillis(100)),
             Joined.with(Serdes.Integer(),
                 Serdes.String(),
                 Serdes.String()));
@@ -640,7 +641,7 @@ public class KStreamKStreamJoinTest {
         joined = stream1.join(
             stream2,
             MockValueJoiner.TOSTRING_JOINER,
-            JoinWindows.of(0).before(100),
+            JoinWindows.of(ofMillis(0)).before(ofMillis(100)),
             Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String()));
         joined.process(supplier);
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
index 856de3d..b019411 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
@@ -39,6 +39,7 @@ import java.util.HashSet;
 import java.util.Properties;
 import java.util.Set;
 
+import static java.time.Duration.ofMillis;
 import static org.junit.Assert.assertEquals;
 
 public class KStreamKStreamLeftJoinTest {
@@ -65,7 +66,7 @@ public class KStreamKStreamLeftJoinTest {
 
         joined = stream1.leftJoin(stream2,
                                   MockValueJoiner.TOSTRING_JOINER,
-                                  JoinWindows.of(100),
+                                  JoinWindows.of(ofMillis(100)),
                                   Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String()));
         joined.process(supplier);
 
@@ -151,7 +152,7 @@ public class KStreamKStreamLeftJoinTest {
 
         joined = stream1.leftJoin(stream2,
                                   MockValueJoiner.TOSTRING_JOINER,
-                                  JoinWindows.of(100),
+                                  JoinWindows.of(ofMillis(100)),
                                   Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String()));
         joined.process(supplier);
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java
index c7fd7cd..419c861 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java
@@ -50,6 +50,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import static java.time.Duration.ofMillis;
 import static org.apache.kafka.common.utils.Utils.mkEntry;
 import static org.apache.kafka.common.utils.Utils.mkMap;
 import static org.apache.kafka.test.StreamsTestUtils.getMetricByName;
@@ -69,7 +70,7 @@ public class KStreamSessionWindowAggregateProcessorTest {
     private final Merger<String, Long> sessionMerger = (aggKey, aggOne, aggTwo) -> aggOne + aggTwo;
     private final KStreamSessionWindowAggregate<String, String, Long> sessionAggregator =
         new KStreamSessionWindowAggregate<>(
-            SessionWindows.with(GAP_MS),
+            SessionWindows.with(ofMillis(GAP_MS)),
             STORE_NAME,
             initializer,
             aggregator,
@@ -106,7 +107,7 @@ public class KStreamSessionWindowAggregateProcessorTest {
     }
 
     private void initStore(final boolean enableCaching) {
-        final StoreBuilder<SessionStore<String, Long>> storeBuilder = Stores.sessionStoreBuilder(Stores.persistentSessionStore(STORE_NAME, GAP_MS * 3),
+        final StoreBuilder<SessionStore<String, Long>> storeBuilder = Stores.sessionStoreBuilder(Stores.persistentSessionStore(STORE_NAME, ofMillis(GAP_MS * 3)),
                                                                                                  Serdes.String(),
                                                                                                  Serdes.Long())
             .withLoggingDisabled();
@@ -322,7 +323,7 @@ public class KStreamSessionWindowAggregateProcessorTest {
         LogCaptureAppender.setClassLoggerToDebug(KStreamSessionWindowAggregate.class);
         final LogCaptureAppender appender = LogCaptureAppender.createAndRegister();
         final Processor<String, String> processor = new KStreamSessionWindowAggregate<>(
-            SessionWindows.with(10L).grace(10L),
+            SessionWindows.with(ofMillis(10L)).grace(ofMillis(10L)),
             STORE_NAME,
             initializer,
             aggregator,
@@ -349,4 +350,4 @@ public class KStreamSessionWindowAggregateProcessorTest {
         assertEquals(1.0, dropMetric.metricValue());
         assertThat(appender.getMessages(), hasItem("Skipping record for expired window. key=[A] topic=[topic] partition=[-3] offset=[-2] timestamp=[0] window=[0,0) expiration=[10]"));
     }
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java
index a8ee681..74ad19c 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream.internals;
 
+import java.time.Duration;
 import org.apache.kafka.common.serialization.IntegerSerializer;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.streams.kstream.Consumed;
@@ -112,7 +113,7 @@ public class KStreamTransformTest {
 
                     @Override
                     public void init(final ProcessorContext context) {
-                        context.schedule(1, PunctuationType.WALL_CLOCK_TIME, new Punctuator() {
+                        context.schedule(Duration.ofMillis(1), PunctuationType.WALL_CLOCK_TIME, new Punctuator() {
                             @Override
                             public void punctuate(final long timestamp) {
                                 context.forward(-1, (int) timestamp);
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java
index 5a295b8..8ae6284 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java
@@ -47,6 +47,7 @@ import org.junit.Test;
 import java.util.List;
 import java.util.Properties;
 
+import static java.time.Duration.ofMillis;
 import static org.apache.kafka.common.utils.Utils.mkEntry;
 import static org.apache.kafka.common.utils.Utils.mkMap;
 import static org.apache.kafka.test.StreamsTestUtils.getMetricByName;
@@ -70,7 +71,7 @@ public class KStreamWindowAggregateTest {
         final KTable<Windowed<String>, String> table2 = builder
             .stream(topic1, Consumed.with(Serdes.String(), Serdes.String()))
             .groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-            .windowedBy(TimeWindows.of(10).advanceBy(5))
+            .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5)))
             .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("topic1-Canonized").withValueSerde(Serdes.String()));
 
         final MockProcessorSupplier<Windowed<String>, String> supplier = new MockProcessorSupplier<>();
@@ -128,7 +129,7 @@ public class KStreamWindowAggregateTest {
         final KTable<Windowed<String>, String> table1 = builder
             .stream(topic1, Consumed.with(Serdes.String(), Serdes.String()))
             .groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-            .windowedBy(TimeWindows.of(10).advanceBy(5))
+            .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5)))
             .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("topic1-Canonized").withValueSerde(Serdes.String()));
 
         final MockProcessorSupplier<Windowed<String>, String> supplier = new MockProcessorSupplier<>();
@@ -137,7 +138,7 @@ public class KStreamWindowAggregateTest {
         final KTable<Windowed<String>, String> table2 = builder
             .stream(topic2, Consumed.with(Serdes.String(), Serdes.String()))
             .groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-            .windowedBy(TimeWindows.of(10).advanceBy(5))
+            .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5)))
             .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Materialized.<String, String, WindowStore<Bytes, byte[]>>as("topic2-Canonized").withValueSerde(Serdes.String()));
 
         table2.toStream().process(supplier);
@@ -232,7 +233,7 @@ public class KStreamWindowAggregateTest {
 
         final KStream<String, String> stream1 = builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String()));
         stream1.groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-            .windowedBy(TimeWindows.of(10).advanceBy(5))
+            .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5)))
             .aggregate(
                 MockInitializer.STRING_INIT,
                 MockAggregator.toStringInstance("+"),
@@ -257,7 +258,7 @@ public class KStreamWindowAggregateTest {
 
         final KStream<String, String> stream1 = builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String()));
         stream1.groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-            .windowedBy(TimeWindows.of(10).advanceBy(5).until(100))
+            .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5)).until(100))
             .aggregate(
                 () -> "",
                 MockAggregator.toStringInstance("+"),
@@ -316,7 +317,7 @@ public class KStreamWindowAggregateTest {
 
         final KStream<String, String> stream1 = builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String()));
         stream1.groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-            .windowedBy(TimeWindows.of(10).advanceBy(5).grace(90L))
+            .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5)).grace(ofMillis(90L)))
             .aggregate(
                 () -> "",
                 MockAggregator.toStringInstance("+"),
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduceTest.java
index 3746ae9..913710f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduceTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduceTest.java
@@ -36,6 +36,7 @@ import org.junit.Test;
 
 import java.util.Properties;
 
+import static java.time.Duration.ofMillis;
 import static org.apache.kafka.common.utils.Utils.mkEntry;
 import static org.apache.kafka.common.utils.Utils.mkMap;
 import static org.apache.kafka.test.StreamsTestUtils.getMetricByName;
@@ -58,7 +59,7 @@ public class KStreamWindowReduceTest {
         builder
             .stream("TOPIC", Consumed.with(Serdes.String(), Serdes.String()))
             .groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-            .windowedBy(TimeWindows.of(500L))
+            .windowedBy(TimeWindows.of(ofMillis(500L)))
             .reduce((value1, value2) -> value1 + "+" + value2);
 
 
@@ -80,7 +81,7 @@ public class KStreamWindowReduceTest {
         builder
             .stream("TOPIC", Consumed.with(Serdes.String(), Serdes.String()))
             .groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-            .windowedBy(TimeWindows.of(5L).until(100))
+            .windowedBy(TimeWindows.of(ofMillis(5L)).until(100))
             .reduce((value1, value2) -> value1 + "+" + value2)
             .toStream()
             .map((key, value) -> new KeyValue<>(key.toString(), value))
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java
index 34a235a..9ee918c 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java
@@ -47,6 +47,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
+import static java.time.Duration.ofMillis;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.MatcherAssert.assertThat;
 
@@ -69,7 +70,7 @@ public class SessionWindowedKStreamImplTest {
     public void before() {
         final KStream<String, String> stream = builder.stream(TOPIC, Consumed.with(Serdes.String(), Serdes.String()));
         this.stream = stream.groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-                .windowedBy(SessionWindows.with(500));
+                .windowedBy(SessionWindows.with(ofMillis(500)));
     }
 
     @Test
@@ -260,4 +261,4 @@ public class SessionWindowedKStreamImplTest {
         driver.pipeInput(recordFactory.create(TOPIC, "2", "1", 600));
     }
 
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java
index 04b7ab8..65c51fc 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java
@@ -368,7 +368,7 @@ public class SuppressScenarioTest {
         final KTable<Windowed<String>, Long> valueCounts = builder
             .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE))
             .groupBy((String k, String v) -> k, Grouped.with(STRING_SERDE, STRING_SERDE))
-            .windowedBy(TimeWindows.of(2L).grace(1L))
+            .windowedBy(TimeWindows.of(2L).grace(ofMillis(1L)))
             .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("counts").withCachingDisabled());
         valueCounts
             .suppress(untilWindowCloses(unbounded()))
@@ -422,7 +422,7 @@ public class SuppressScenarioTest {
         final KTable<Windowed<String>, Long> valueCounts = builder
             .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE))
             .groupBy((String k, String v) -> k, Grouped.with(STRING_SERDE, STRING_SERDE))
-            .windowedBy(TimeWindows.of(2L).grace(2L))
+            .windowedBy(TimeWindows.of(2L).grace(ofMillis(2L)))
             .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("counts").withCachingDisabled().withKeySerde(STRING_SERDE));
         valueCounts
             .suppress(untilWindowCloses(unbounded()))
@@ -481,7 +481,7 @@ public class SuppressScenarioTest {
         final KTable<Windowed<String>, Long> valueCounts = builder
             .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE))
             .groupBy((String k, String v) -> k, Grouped.with(STRING_SERDE, STRING_SERDE))
-            .windowedBy(SessionWindows.with(5L).grace(5L))
+            .windowedBy(SessionWindows.with(5L).grace(ofMillis(5L)))
             .count(Materialized.<String, Long, SessionStore<Bytes, byte[]>>as("counts").withCachingDisabled());
         valueCounts
             .suppress(untilWindowCloses(unbounded()))
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressTopologyTest.java
index 70c25c7..97bdb1d 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressTopologyTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressTopologyTest.java
@@ -32,6 +32,7 @@ import org.junit.Test;
 
 import java.time.Duration;
 
+import static java.time.Duration.ofMillis;
 import static org.apache.kafka.streams.kstream.Suppressed.BufferConfig.unbounded;
 import static org.apache.kafka.streams.kstream.Suppressed.untilTimeLimit;
 import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses;
@@ -146,7 +147,7 @@ public class SuppressTopologyTest {
         anonymousNodeBuilder
             .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE))
             .groupBy((String k, String v) -> k, Grouped.with(STRING_SERDE, STRING_SERDE))
-            .windowedBy(SessionWindows.with(5L).grace(5L))
+            .windowedBy(SessionWindows.with(ofMillis(5L)).grace(ofMillis(5L)))
             .count(Materialized.<String, Long, SessionStore<Bytes, byte[]>>as("counts").withCachingDisabled())
             .suppress(untilWindowCloses(unbounded()))
             .toStream()
@@ -164,7 +165,7 @@ public class SuppressTopologyTest {
         namedNodeBuilder
             .stream("input", Consumed.with(STRING_SERDE, STRING_SERDE))
             .groupBy((String k, String v) -> k, Grouped.with(STRING_SERDE, STRING_SERDE))
-            .windowedBy(SessionWindows.with(5L).grace(5L))
+            .windowedBy(SessionWindows.with(ofMillis(5L)).grace(ofMillis(5L)))
             .count(Materialized.<String, Long, SessionStore<Bytes, byte[]>>as("counts").withCachingDisabled())
             .suppress(untilWindowCloses(unbounded()).withName("myname"))
             .toStream()
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java
index f260bee..f6e06e4 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java
@@ -21,6 +21,7 @@ import org.junit.Test;
 
 import java.util.Map;
 
+import static java.time.Duration.ofMillis;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -124,7 +125,7 @@ public class TimeWindowTest {
 
     @Test
     public void shouldReturnMatchedWindowsOrderedByTimestamp() {
-        final TimeWindows windows = TimeWindows.of(12L).advanceBy(5L);
+        final TimeWindows windows = TimeWindows.of(ofMillis(12L)).advanceBy(ofMillis(5L));
         final Map<Long, TimeWindow> matched = windows.windowsFor(21L);
 
         final Long[] expected = matched.keySet().toArray(new Long[matched.size()]);
@@ -132,4 +133,4 @@ public class TimeWindowTest {
         assertEquals(expected[1].longValue(), 15L);
         assertEquals(expected[2].longValue(), 20L);
     }
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java
index 0e541c9..f951ebb 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java
@@ -46,6 +46,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
+import static java.time.Duration.ofMillis;
+import static java.time.Instant.ofEpochMilli;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.MatcherAssert.assertThat;
 
@@ -61,7 +63,7 @@ public class TimeWindowedKStreamImplTest {
     public void before() {
         final KStream<String, String> stream = builder.stream(TOPIC, Consumed.with(Serdes.String(), Serdes.String()));
         windowedStream = stream.groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-                .windowedBy(TimeWindows.of(500L));
+                .windowedBy(TimeWindows.of(ofMillis(500L)));
     }
 
     @Test
@@ -136,7 +138,7 @@ public class TimeWindowedKStreamImplTest {
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) {
             processData(driver);
             final WindowStore<String, Long> windowStore = driver.getWindowStore("count-store");
-            final List<KeyValue<Windowed<String>, Long>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", 0, 1000));
+            final List<KeyValue<Windowed<String>, Long>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
 
             assertThat(data, equalTo(Arrays.asList(
                     KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), 2L),
@@ -155,7 +157,7 @@ public class TimeWindowedKStreamImplTest {
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) {
             processData(driver);
             final WindowStore<String, String> windowStore = driver.getWindowStore("reduced");
-            final List<KeyValue<Windowed<String>, String>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", 0, 1000));
+            final List<KeyValue<Windowed<String>, String>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
 
             assertThat(data, equalTo(Arrays.asList(
                     KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), "1+2"),
@@ -175,7 +177,7 @@ public class TimeWindowedKStreamImplTest {
         try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props, 0L)) {
             processData(driver);
             final WindowStore<String, String> windowStore = driver.getWindowStore("aggregated");
-            final List<KeyValue<Windowed<String>, String>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", 0, 1000));
+            final List<KeyValue<Windowed<String>, String>> data = StreamsTestUtils.toList(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L)));
             assertThat(data, equalTo(Arrays.asList(
                     KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), "0+1+2"),
                     KeyValue.pair(new Windowed<>("1", new TimeWindow(500, 1000)), "0+3"),
@@ -244,4 +246,4 @@ public class TimeWindowedKStreamImplTest {
         driver.pipeInput(recordFactory.create(TOPIC, "2", "1", 500L));
     }
 
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java
index 2b05423..37265fa 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java
@@ -26,6 +26,7 @@ import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.junit.Test;
 
+import static java.time.Duration.ofMillis;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
@@ -78,7 +79,7 @@ public class GraphGraceSearchUtilTest {
 
     @Test
     public void shouldExtractGraceFromKStreamWindowAggregateNode() {
-        final TimeWindows windows = TimeWindows.of(10L).grace(1234L);
+        final TimeWindows windows = TimeWindows.of(ofMillis(10L)).grace(ofMillis(1234L));
         final StatefulProcessorNode<String, Long> node = new StatefulProcessorNode<>(
             "asdf",
             new ProcessorParameters<>(
@@ -101,7 +102,7 @@ public class GraphGraceSearchUtilTest {
 
     @Test
     public void shouldExtractGraceFromKStreamSessionWindowAggregateNode() {
-        final SessionWindows windows = SessionWindows.with(10L).grace(1234L);
+        final SessionWindows windows = SessionWindows.with(ofMillis(10L)).grace(ofMillis(1234L));
 
         final StatefulProcessorNode<String, Long> node = new StatefulProcessorNode<>(
             "asdf",
@@ -126,7 +127,7 @@ public class GraphGraceSearchUtilTest {
 
     @Test
     public void shouldExtractGraceFromAncestorThroughStatefulParent() {
-        final SessionWindows windows = SessionWindows.with(10L).grace(1234L);
+        final SessionWindows windows = SessionWindows.with(ofMillis(10L)).grace(ofMillis(1234L));
         final StatefulProcessorNode<String, Long> graceGrandparent = new StatefulProcessorNode<>(
             "asdf",
             new ProcessorParameters<>(new KStreamSessionWindowAggregate<String, Long, Integer>(
@@ -167,7 +168,7 @@ public class GraphGraceSearchUtilTest {
 
     @Test
     public void shouldExtractGraceFromAncestorThroughStatelessParent() {
-        final SessionWindows windows = SessionWindows.with(10L).grace(1234L);
+        final SessionWindows windows = SessionWindows.with(ofMillis(10L)).grace(ofMillis(1234L));
         final StatefulProcessorNode<String, Long> graceGrandparent = new StatefulProcessorNode<>(
             "asdf",
             new ProcessorParameters<>(
@@ -201,7 +202,7 @@ public class GraphGraceSearchUtilTest {
             "asdf",
             new ProcessorParameters<>(
                 new KStreamSessionWindowAggregate<String, Long, Integer>(
-                    SessionWindows.with(10L).grace(1234L),
+                    SessionWindows.with(ofMillis(10L)).grace(ofMillis(1234L)),
                     "asdf",
                     null,
                     null,
@@ -218,7 +219,7 @@ public class GraphGraceSearchUtilTest {
             "asdf",
             new ProcessorParameters<>(
                 new KStreamWindowAggregate<String, Long, Integer, TimeWindow>(
-                    TimeWindows.of(10L).grace(4321L),
+                    TimeWindows.of(ofMillis(10L)).grace(ofMillis(4321L)),
                     "asdf",
                     null,
                     null
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
index 75e9f51..bd43685 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
@@ -34,6 +34,7 @@ import java.util.Properties;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import static java.time.Duration.ofMillis;
 import static org.junit.Assert.assertEquals;
 
 public class StreamsGraphTest {
@@ -51,7 +52,7 @@ public class StreamsGraphTest {
         final ValueJoiner<String, String, String> valueJoiner = (v, v2) -> v + v2;
 
 
-        final KStream<String, String> joinedStream = stream.join(streamII, valueJoiner, JoinWindows.of(5000));
+        final KStream<String, String> joinedStream = stream.join(streamII, valueJoiner, JoinWindows.of(ofMillis(5000)));
 
         // build step one
         assertEquals(expectedJoinedTopology, builder.build().describe().toString());
@@ -100,7 +101,7 @@ public class StreamsGraphTest {
         final KStream<String, String> mappedKeyStream = inputStream.selectKey((k, v) -> k + v);
 
         mappedKeyStream.mapValues(v -> v.toUpperCase(Locale.getDefault())).groupByKey().count().toStream().to("output");
-        mappedKeyStream.flatMapValues(v -> Arrays.asList(v.split("\\s"))).groupByKey().windowedBy(TimeWindows.of(5000)).count().toStream().to("windowed-output");
+        mappedKeyStream.flatMapValues(v -> Arrays.asList(v.split("\\s"))).groupByKey().windowedBy(TimeWindows.of(ofMillis(5000))).count().toStream().to("windowed-output");
 
         return builder.build(properties);
 
@@ -116,7 +117,7 @@ public class StreamsGraphTest {
         final KStream<String, String> mappedKeyStream = inputStream.selectKey((k, v) -> k + v).through("through-topic");
 
         mappedKeyStream.groupByKey().count().toStream().to("output");
-        mappedKeyStream.groupByKey().windowedBy(TimeWindows.of(5000)).count().toStream().to("windowed-output");
+        mappedKeyStream.groupByKey().windowedBy(TimeWindows.of(ofMillis(5000))).count().toStream().to("windowed-output");
 
         return builder.build(properties);
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
index 0fa0583..4008689 100644
--- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
+++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
@@ -54,7 +54,6 @@ import org.apache.kafka.streams.state.Stores;
 import org.apache.kafka.streams.state.WindowStore;
 
 import java.io.IOException;
-import java.time.Duration;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Locale;
@@ -63,6 +62,10 @@ import java.util.Random;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
+import static java.time.Duration.ofMillis;
+import static java.time.Duration.ofSeconds;
+import static java.time.Instant.ofEpochMilli;
+
 /**
  * Class that provides support for a series of benchmarks. It is usually driven by
  * tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py.
@@ -332,7 +335,7 @@ public class SimpleBenchmark {
             consumer.seekToBeginning(partitions);
 
             while (true) {
-                final ConsumerRecords<Integer, byte[]> records = consumer.poll(Duration.ofMillis(POLL_MS));
+                final ConsumerRecords<Integer, byte[]> records = consumer.poll(ofMillis(POLL_MS));
                 if (records.isEmpty()) {
                     if (processedRecords == numRecords) {
                         break;
@@ -370,7 +373,7 @@ public class SimpleBenchmark {
             consumer.seekToBeginning(partitions);
 
             while (true) {
-                final ConsumerRecords<Integer, byte[]> records = consumer.poll(Duration.ofMillis(POLL_MS));
+                final ConsumerRecords<Integer, byte[]> records = consumer.poll(ofMillis(POLL_MS));
                 if (records.isEmpty()) {
                     if (processedRecords == numRecords) {
                         break;
@@ -498,7 +501,7 @@ public class SimpleBenchmark {
                     @Override
                     public void process(final Integer key, final byte[] value) {
                         final long timestamp = context().timestamp();
-                        final KeyValueIterator<Windowed<Integer>, byte[]> iter = store.fetch(key - 10, key + 10, timestamp - 1000L, timestamp + 1000L);
+                        final KeyValueIterator<Windowed<Integer>, byte[]> iter = store.fetch(key - 10, key + 10, ofEpochMilli(timestamp - 1000L), ofEpochMilli(timestamp));
                         while (iter.hasNext()) {
                             iter.next();
                         }
@@ -550,7 +553,7 @@ public class SimpleBenchmark {
 
         input.peek(new CountDownAction(latch))
                 .groupByKey()
-                .windowedBy(TimeWindows.of(AGGREGATE_WINDOW_SIZE).advanceBy(AGGREGATE_WINDOW_ADVANCE))
+                .windowedBy(TimeWindows.of(ofMillis(AGGREGATE_WINDOW_SIZE)).advanceBy(ofMillis(AGGREGATE_WINDOW_ADVANCE)))
                 .count();
 
         final KafkaStreams streams = createKafkaStreamsWithExceptionHandler(builder, props);
@@ -593,7 +596,7 @@ public class SimpleBenchmark {
         final KStream<Integer, byte[]> input1 = builder.stream(kStreamTopic1);
         final KStream<Integer, byte[]> input2 = builder.stream(kStreamTopic2);
 
-        input1.leftJoin(input2, VALUE_JOINER, JoinWindows.of(STREAM_STREAM_JOIN_WINDOW)).foreach(new CountDownAction(latch));
+        input1.leftJoin(input2, VALUE_JOINER, JoinWindows.of(ofMillis(STREAM_STREAM_JOIN_WINDOW))).foreach(new CountDownAction(latch));
 
         final KafkaStreams streams = createKafkaStreamsWithExceptionHandler(builder, props);
 
@@ -677,7 +680,7 @@ public class SimpleBenchmark {
             public void uncaughtException(final Thread t, final Throwable e) {
                 System.out.println("FATAL: An unexpected exception is encountered on thread " + t + ": " + e);
 
-                streamsClient.close(30, TimeUnit.SECONDS);
+                streamsClient.close(ofSeconds(30));
             }
         });
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java
index 6e09ad1..1109f9d 100644
--- a/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java
+++ b/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java
@@ -17,6 +17,7 @@
 package org.apache.kafka.streams.perf;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import java.time.Duration;
 import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.clients.producer.ProducerRecord;
@@ -334,7 +335,7 @@ public class YahooBenchmark {
         // calculate windowed counts
         keyedByCampaign
             .groupByKey(Serialized.with(Serdes.String(), Serdes.String()))
-            .windowedBy(TimeWindows.of(10 * 1000))
+            .windowedBy(TimeWindows.of(Duration.ofMillis(10 * 1000)))
             .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("time-windows"));
 
         return new KafkaStreams(builder.build(), streamConfig);
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java
index 070dba8..4ce9a9f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.processor.internals;
 
+import java.time.Duration;
 import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.header.internals.RecordHeader;
@@ -203,6 +204,13 @@ public class AbstractProcessorContextTest {
         }
 
         @Override
+        public Cancellable schedule(final Duration interval,
+                                    final PunctuationType type,
+                                    final Punctuator callback) throws IllegalArgumentException {
+            return null;
+        }
+
+        @Override
         public <K, V> void forward(final K key, final V value) {}
 
         @Override
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java
index daf1f33..8ddb0b5 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java
@@ -47,6 +47,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.regex.Pattern;
 
+import static java.time.Duration.ofSeconds;
 import static org.apache.kafka.common.utils.Utils.mkList;
 import static org.apache.kafka.common.utils.Utils.mkSet;
 import static org.hamcrest.core.IsInstanceOf.instanceOf;
@@ -588,7 +589,7 @@ public class InternalTopologyBuilderTest {
         builder.addProcessor("processor", new MockProcessorSupplier(), "source");
         builder.addStateStore(
             Stores.windowStoreBuilder(
-                Stores.persistentWindowStore("store1", 30_000L, 10_000L, false),
+                Stores.persistentWindowStore("store1", ofSeconds(30L), ofSeconds(10L), false),
                 Serdes.String(),
                 Serdes.String()
             ),
@@ -596,7 +597,7 @@ public class InternalTopologyBuilderTest {
         );
         builder.addStateStore(
                 Stores.sessionStoreBuilder(
-                        Stores.persistentSessionStore("store2", 30000), Serdes.String(), Serdes.String()
+                        Stores.persistentSessionStore("store2", ofSeconds(30)), Serdes.String(), Serdes.String()
                 ),
                 "processor"
         );
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java
index e669879..639ebf8 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java
@@ -72,6 +72,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import static java.time.Duration.ofMillis;
 import static java.util.Collections.emptyList;
 import static java.util.Collections.emptySet;
 import static java.util.Collections.singletonList;
@@ -211,7 +212,7 @@ public class StandbyTaskTest {
         }
 
         restoreStateConsumer.seekToBeginning(partition);
-        task.update(partition2, restoreStateConsumer.poll(Duration.ofMillis(100)).records(partition2));
+        task.update(partition2, restoreStateConsumer.poll(ofMillis(100)).records(partition2));
 
         final StandbyContextImpl context = (StandbyContextImpl) task.context();
         final MockKeyValueStore store1 = (MockKeyValueStore) context.getStateMgr().getStore(storeName1);
@@ -239,8 +240,8 @@ public class StandbyTaskTest {
         builder
             .stream(Collections.singleton("topic"), new ConsumedInternal<>())
             .groupByKey()
-            .windowedBy(TimeWindows.of(60_000).grace(0L))
-            .count(Materialized.<Object, Long, WindowStore<Bytes, byte[]>>as(storeName).withRetention(120_000L));
+            .windowedBy(TimeWindows.of(ofMillis(60_000)).grace(ofMillis(0L)))
+            .count(Materialized.<Object, Long, WindowStore<Bytes, byte[]>>as(storeName).withRetention(ofMillis(120_000L)));
 
         builder.buildAndOptimizeTopology();
 
@@ -484,7 +485,7 @@ public class StandbyTaskTest {
     @Test
     public void shouldInitializeWindowStoreWithoutException() throws IOException {
         final InternalStreamsBuilder builder = new InternalStreamsBuilder(new InternalTopologyBuilder());
-        builder.stream(Collections.singleton("topic"), new ConsumedInternal<>()).groupByKey().windowedBy(TimeWindows.of(100)).count();
+        builder.stream(Collections.singleton("topic"), new ConsumedInternal<>()).groupByKey().windowedBy(TimeWindows.of(ofMillis(100))).count();
 
         initializeStandbyStores(builder);
     }
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
index e691c54..b4de5ec 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.processor.internals;
 
+import java.time.Duration;
 import org.apache.kafka.clients.admin.MockAdminClient;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
@@ -1040,13 +1041,13 @@ public class StreamThreadTest {
                 return new Processor<Object, Object>() {
                     @Override
                     public void init(final ProcessorContext context) {
-                        context.schedule(100L, PunctuationType.STREAM_TIME, new Punctuator() {
+                        context.schedule(Duration.ofMillis(100L), PunctuationType.STREAM_TIME, new Punctuator() {
                             @Override
                             public void punctuate(final long timestamp) {
                                 punctuatedStreamTime.add(timestamp);
                             }
                         });
-                        context.schedule(100L, PunctuationType.WALL_CLOCK_TIME, new Punctuator() {
+                        context.schedule(Duration.ofMillis(100L), PunctuationType.WALL_CLOCK_TIME, new Punctuator() {
                             @Override
                             public void punctuate(final long timestamp) {
                                 punctuatedWallClockTime.add(timestamp);
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java
index 6102969..649aa19 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java
@@ -60,6 +60,7 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static java.time.Duration.ofMillis;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.not;
 import static org.junit.Assert.assertEquals;
@@ -961,7 +962,7 @@ public class StreamsPartitionAssignorTest {
                         return null;
                     }
                 },
-                JoinWindows.of(0)
+                JoinWindows.of(ofMillis(0))
             );
 
         final UUID uuid = UUID.randomUUID();
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java
index 8b20b0b..34d9050 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.state;
 
+import java.time.Instant;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.StateStore;
@@ -88,22 +89,43 @@ public class NoOpWindowStore implements ReadOnlyWindowStore, StateStore {
     }
 
     @Override
+    @SuppressWarnings("deprecation")
     public WindowStoreIterator fetch(final Object key, final long timeFrom, final long timeTo) {
         return EMPTY_WINDOW_STORE_ITERATOR;
     }
 
     @Override
+    public WindowStoreIterator fetch(final Object key, final Instant from, final Instant to) {
+        return EMPTY_WINDOW_STORE_ITERATOR;
+    }
+
+    @Override
+    @SuppressWarnings("deprecation")
     public WindowStoreIterator<KeyValue> fetch(final Object from, final Object to, final long timeFrom, final long timeTo) {
         return EMPTY_WINDOW_STORE_ITERATOR;
     }
-    
+
+    @Override
+    public KeyValueIterator fetch(final Object from,
+                                  final Object to,
+                                  final Instant fromTime,
+                                  final Instant toTime) throws IllegalArgumentException {
+        return EMPTY_WINDOW_STORE_ITERATOR;
+    }
+
     @Override
     public WindowStoreIterator<KeyValue> all() {
         return EMPTY_WINDOW_STORE_ITERATOR;
     }
     
     @Override
+    @SuppressWarnings("deprecation")
     public WindowStoreIterator<KeyValue> fetchAll(final long timeFrom, final long timeTo) {
         return EMPTY_WINDOW_STORE_ITERATOR;
     }
+
+    @Override
+    public KeyValueIterator fetchAll(final Instant from, final Instant to) {
+        return EMPTY_WINDOW_STORE_ITERATOR;
+    }
 }
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java
index d0da158..b62364a 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java
@@ -24,6 +24,7 @@ import org.apache.kafka.streams.state.internals.RocksDBStore;
 import org.apache.kafka.streams.state.internals.RocksDBWindowStore;
 import org.junit.Test;
 
+import static java.time.Duration.ofMillis;
 import static org.hamcrest.CoreMatchers.nullValue;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.core.IsInstanceOf.instanceOf;
@@ -70,7 +71,7 @@ public class StoresTest {
 
     @Test(expected = IllegalArgumentException.class)
     public void shouldThrowIfIPersistentWindowStoreIfWindowSizeIsNegative() {
-        Stores.persistentWindowStore("anyName", 0L, -1L, false);
+        Stores.persistentWindowStore("anyName", ofMillis(0L), ofMillis(-1L), false);
     }
 
     @Test(expected = IllegalArgumentException.class)
@@ -80,12 +81,13 @@ public class StoresTest {
 
     @Test(expected = NullPointerException.class)
     public void shouldThrowIfIPersistentSessionStoreStoreNameIsNull() {
-        Stores.persistentSessionStore(null, 0);
+        Stores.persistentSessionStore(null, ofMillis(0));
+
     }
 
     @Test(expected = IllegalArgumentException.class)
     public void shouldThrowIfIPersistentSessionStoreRetentionPeriodIsNegative() {
-        Stores.persistentSessionStore("anyName", -1);
+        Stores.persistentSessionStore("anyName", ofMillis(-1));
     }
 
     @Test(expected = NullPointerException.class)
@@ -120,18 +122,18 @@ public class StoresTest {
 
     @Test
     public void shouldCreateRocksDbWindowStore() {
-        assertThat(Stores.persistentWindowStore("store", 1L, 1L, false).get(), instanceOf(RocksDBWindowStore.class));
+        assertThat(Stores.persistentWindowStore("store", ofMillis(1L), ofMillis(1L), false).get(), instanceOf(RocksDBWindowStore.class));
     }
 
     @Test
     public void shouldCreateRocksDbSessionStore() {
-        assertThat(Stores.persistentSessionStore("store", 1).get(), instanceOf(RocksDBSessionStore.class));
+        assertThat(Stores.persistentSessionStore("store", ofMillis(1)).get(), instanceOf(RocksDBSessionStore.class));
     }
 
     @Test
     public void shouldBuildWindowStore() {
         final WindowStore<String, String> store = Stores.windowStoreBuilder(
-            Stores.persistentWindowStore("store", 3L, 3L, true),
+            Stores.persistentWindowStore("store", ofMillis(3L), ofMillis(3L), true),
             Serdes.String(),
             Serdes.String()
         ).build();
@@ -151,10 +153,10 @@ public class StoresTest {
     @Test
     public void shouldBuildSessionStore() {
         final SessionStore<String, String> store = Stores.sessionStoreBuilder(
-            Stores.persistentSessionStore("name", 10),
+            Stores.persistentSessionStore("name", ofMillis(10)),
             Serdes.String(),
             Serdes.String()
         ).build();
         assertThat(store, not(nullValue()));
     }
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java
index 1c8dd7b..7114731 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java
@@ -49,6 +49,9 @@ import java.util.List;
 import java.util.Properties;
 import java.util.UUID;
 
+import static java.time.Duration.ofHours;
+import static java.time.Duration.ofMinutes;
+import static java.time.Instant.ofEpochMilli;
 import static org.apache.kafka.common.utils.Utils.mkList;
 import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize;
 import static org.apache.kafka.test.StreamsTestUtils.toList;
@@ -101,7 +104,7 @@ public class CachingWindowStoreTest {
         final StreamsBuilder builder = new StreamsBuilder();
 
         final StoreBuilder<WindowStore<String, String>> storeBuilder = Stores.windowStoreBuilder(
-            Stores.persistentWindowStore("store-name", 3600000L, 60000L, false),
+            Stores.persistentWindowStore("store-name", ofHours(1L), ofMinutes(1L), false),
             Serdes.String(),
             Serdes.String())
             .withCachingEnabled();
@@ -197,8 +200,8 @@ public class CachingWindowStoreTest {
         assertThat(cachingStore.fetch(bytesKey("c"), 10), equalTo(null));
         assertThat(cachingStore.fetch(bytesKey("a"), 0), equalTo(null));
 
-        final WindowStoreIterator<byte[]> a = cachingStore.fetch(bytesKey("a"), 10, 10);
-        final WindowStoreIterator<byte[]> b = cachingStore.fetch(bytesKey("b"), 10, 10);
+        final WindowStoreIterator<byte[]> a = cachingStore.fetch(bytesKey("a"), ofEpochMilli(10), ofEpochMilli(10));
+        final WindowStoreIterator<byte[]> b = cachingStore.fetch(bytesKey("b"), ofEpochMilli(10), ofEpochMilli(10));
         verifyKeyValue(a.next(), DEFAULT_TIMESTAMP, "a");
         verifyKeyValue(b.next(), DEFAULT_TIMESTAMP, "b");
         assertFalse(a.hasNext());
@@ -224,7 +227,7 @@ public class CachingWindowStoreTest {
         cachingStore.put(bytesKey("a"), bytesValue("a"));
         cachingStore.put(bytesKey("b"), bytesValue("b"));
 
-        final KeyValueIterator<Windowed<Bytes>, byte[]> iterator = cachingStore.fetch(bytesKey("a"), bytesKey("b"), 10, 10);
+        final KeyValueIterator<Windowed<Bytes>, byte[]> iterator = cachingStore.fetch(bytesKey("a"), bytesKey("b"), ofEpochMilli(10), ofEpochMilli(10));
         verifyWindowedKeyValue(iterator.next(), new Windowed<>(bytesKey("a"), new TimeWindow(DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE)), "a");
         verifyWindowedKeyValue(iterator.next(), new Windowed<>(bytesKey("b"), new TimeWindow(DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE)), "b");
         assertFalse(iterator.hasNext());
@@ -258,21 +261,21 @@ public class CachingWindowStoreTest {
             cachingStore.put(bytesKey(array[i]), bytesValue(array[i]));
         }
 
-        final KeyValueIterator<Windowed<Bytes>, byte[]> iterator = cachingStore.fetchAll(0, 7);
+        final KeyValueIterator<Windowed<Bytes>, byte[]> iterator = cachingStore.fetchAll(ofEpochMilli(0), ofEpochMilli(7));
         for (int i = 0; i < array.length; i++) {
             final String str = array[i];
             verifyWindowedKeyValue(iterator.next(), new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), str);
         }
         assertFalse(iterator.hasNext());
 
-        final KeyValueIterator<Windowed<Bytes>, byte[]> iterator1 = cachingStore.fetchAll(2, 4);
+        final KeyValueIterator<Windowed<Bytes>, byte[]> iterator1 = cachingStore.fetchAll(ofEpochMilli(2), ofEpochMilli(4));
         for (int i = 2; i <= 4; i++) {
             final String str = array[i];
             verifyWindowedKeyValue(iterator1.next(), new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), str);
         }
         assertFalse(iterator1.hasNext());
 
-        final KeyValueIterator<Windowed<Bytes>, byte[]> iterator2 = cachingStore.fetchAll(5, 7);
+        final KeyValueIterator<Windowed<Bytes>, byte[]> iterator2 = cachingStore.fetchAll(ofEpochMilli(5), ofEpochMilli(7));
         for (int i = 5; i <= 7; i++) {
             final String str = array[i];
             verifyWindowedKeyValue(iterator2.next(), new Windowed<>(bytesKey(str), new TimeWindow(i, i + WINDOW_SIZE)), str);
@@ -336,7 +339,7 @@ public class CachingWindowStoreTest {
         cachingStore.flush();
         cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP);
 
-        final WindowStoreIterator<byte[]> fetch = cachingStore.fetch(bytesKey("1"), DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP);
+        final WindowStoreIterator<byte[]> fetch = cachingStore.fetch(bytesKey("1"), ofEpochMilli(DEFAULT_TIMESTAMP), ofEpochMilli(DEFAULT_TIMESTAMP));
         verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP, "b");
         assertFalse(fetch.hasNext());
     }
@@ -346,7 +349,7 @@ public class CachingWindowStoreTest {
         cachingStore.put(bytesKey("1"), bytesValue("a"), DEFAULT_TIMESTAMP);
         cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE);
 
-        final WindowStoreIterator<byte[]> fetch = cachingStore.fetch(bytesKey("1"), DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE);
+        final WindowStoreIterator<byte[]> fetch = cachingStore.fetch(bytesKey("1"), ofEpochMilli(DEFAULT_TIMESTAMP), ofEpochMilli(DEFAULT_TIMESTAMP + WINDOW_SIZE));
         verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP, "a");
         verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP + WINDOW_SIZE, "b");
         assertFalse(fetch.hasNext());
@@ -357,7 +360,7 @@ public class CachingWindowStoreTest {
         final Bytes key = Bytes.wrap("1".getBytes());
         underlying.put(WindowKeySchema.toStoreKeyBinary(key, DEFAULT_TIMESTAMP, 0), "a".getBytes());
         cachingStore.put(key, bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE);
-        final WindowStoreIterator<byte[]> fetch = cachingStore.fetch(bytesKey("1"), DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE);
+        final WindowStoreIterator<byte[]> fetch = cachingStore.fetch(bytesKey("1"), ofEpochMilli(DEFAULT_TIMESTAMP), ofEpochMilli(DEFAULT_TIMESTAMP + WINDOW_SIZE));
         verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP, "a");
         verifyKeyValue(fetch.next(), DEFAULT_TIMESTAMP + WINDOW_SIZE, "b");
         assertFalse(fetch.hasNext());
@@ -370,7 +373,7 @@ public class CachingWindowStoreTest {
         cachingStore.put(key, bytesValue("b"), DEFAULT_TIMESTAMP + WINDOW_SIZE);
 
         final KeyValueIterator<Windowed<Bytes>, byte[]> fetchRange =
-            cachingStore.fetch(key, bytesKey("2"), DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE);
+            cachingStore.fetch(key, bytesKey("2"), ofEpochMilli(DEFAULT_TIMESTAMP), ofEpochMilli(DEFAULT_TIMESTAMP + WINDOW_SIZE));
         verifyWindowedKeyValue(fetchRange.next(), new Windowed<>(key, new TimeWindow(DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE)), "a");
         verifyWindowedKeyValue(fetchRange.next(), new Windowed<>(key, new TimeWindow(DEFAULT_TIMESTAMP + WINDOW_SIZE, DEFAULT_TIMESTAMP + WINDOW_SIZE + WINDOW_SIZE)), "b");
         assertFalse(fetchRange.hasNext());
@@ -387,13 +390,13 @@ public class CachingWindowStoreTest {
     @Test(expected = InvalidStateStoreException.class)
     public void shouldThrowIfTryingToFetchFromClosedCachingStore() {
         cachingStore.close();
-        cachingStore.fetch(bytesKey("a"), 0, 10);
+        cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(10));
     }
 
     @Test(expected = InvalidStateStoreException.class)
     public void shouldThrowIfTryingToFetchRangeFromClosedCachingStore() {
         cachingStore.close();
-        cachingStore.fetch(bytesKey("a"), bytesKey("b"), 0, 10);
+        cachingStore.fetch(bytesKey("a"), bytesKey("b"), ofEpochMilli(0), ofEpochMilli(10));
     }
 
     @Test(expected = InvalidStateStoreException.class)
@@ -415,7 +418,7 @@ public class CachingWindowStoreTest {
             KeyValue.pair(1L, bytesValue("0003")),
             KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005"))
         );
-        final List<KeyValue<Long, byte[]>> actual = toList(cachingStore.fetch(bytesKey("a"), 0, Long.MAX_VALUE));
+        final List<KeyValue<Long, byte[]>> actual = toList(cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)));
         verifyKeyValueList(expected, actual);
     }
 
@@ -433,12 +436,12 @@ public class CachingWindowStoreTest {
                 windowedPair("a", "0003", 1),
                 windowedPair("a", "0005", SEGMENT_INTERVAL)
             ),
-            toList(cachingStore.fetch(bytesKey("a"), bytesKey("a"), 0, Long.MAX_VALUE))
+            toList(cachingStore.fetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
         );
 
         verifyKeyValueList(
             mkList(windowedPair("aa", "0002", 0), windowedPair("aa", "0004", 1)),
-            toList(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), 0, Long.MAX_VALUE))
+            toList(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
         );
 
         verifyKeyValueList(
@@ -449,7 +452,7 @@ public class CachingWindowStoreTest {
                 windowedPair("aa", "0004", 1),
                 windowedPair("a", "0005", SEGMENT_INTERVAL)
             ),
-            toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), 0, Long.MAX_VALUE))
+            toList(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)))
         );
     }
 
@@ -465,17 +468,17 @@ public class CachingWindowStoreTest {
 
     @Test(expected = NullPointerException.class)
     public void shouldThrowNullPointerExceptionOnFetchNullKey() {
-        cachingStore.fetch(null, 1L, 2L);
+        cachingStore.fetch(null, ofEpochMilli(1L), ofEpochMilli(2L));
     }
 
     @Test(expected = NullPointerException.class)
     public void shouldThrowNullPointerExceptionOnRangeNullFromKey() {
-        cachingStore.fetch(null, bytesKey("anyTo"), 1L, 2L);
+        cachingStore.fetch(null, bytesKey("anyTo"), ofEpochMilli(1L), ofEpochMilli(2L));
     }
 
     @Test(expected = NullPointerException.class)
     public void shouldThrowNullPointerExceptionOnRangeNullToKey() {
-        cachingStore.fetch(bytesKey("anyFrom"), null, 1L, 2L);
+        cachingStore.fetch(bytesKey("anyFrom"), null, ofEpochMilli(1L), ofEpochMilli(2L));
     }
 
     private static KeyValue<Windowed<Bytes>, byte[]> windowedPair(final String key, final String value, final long timestamp) {
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java
index ec81b93..6d2d994 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java
@@ -36,6 +36,7 @@ import org.junit.runner.RunWith;
 import java.util.HashMap;
 import java.util.Map;
 
+import static java.time.Instant.ofEpochMilli;
 import static org.junit.Assert.assertArrayEquals;
 
 @RunWith(EasyMockRunner.class)
@@ -101,7 +102,7 @@ public class ChangeLoggingWindowBytesStoreTest {
 
         init();
 
-        store.fetch(bytesKey, 0, 10);
+        store.fetch(bytesKey, ofEpochMilli(0), ofEpochMilli(10));
         EasyMock.verify(inner);
     }
 
@@ -111,7 +112,7 @@ public class ChangeLoggingWindowBytesStoreTest {
 
         init();
 
-        store.fetch(bytesKey, bytesKey, 0, 1);
+        store.fetch(bytesKey, bytesKey, ofEpochMilli(0), ofEpochMilli(1));
         EasyMock.verify(inner);
     }
 
@@ -131,4 +132,4 @@ public class ChangeLoggingWindowBytesStoreTest {
         EasyMock.verify(inner);
     }
 
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java
index 66f318a..79afb78 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java
@@ -35,6 +35,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.NoSuchElementException;
 
+import static java.time.Instant.ofEpochMilli;
 import static java.util.Arrays.asList;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.core.IsEqual.equalTo;
@@ -77,7 +78,7 @@ public class CompositeReadOnlyWindowStoreTest {
         underlyingWindowStore.put("my-key", "my-value", 0L);
         underlyingWindowStore.put("my-key", "my-later-value", 10L);
 
-        final WindowStoreIterator<String> iterator = windowStore.fetch("my-key", 0L, 25L);
+        final WindowStoreIterator<String> iterator = windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L));
         final List<KeyValue<Long, String>> results = StreamsTestUtils.toList(iterator);
 
         assertEquals(asList(new KeyValue<>(0L, "my-value"),
@@ -87,7 +88,7 @@ public class CompositeReadOnlyWindowStoreTest {
 
     @Test
     public void shouldReturnEmptyIteratorIfNoData() {
-        final WindowStoreIterator<String> iterator = windowStore.fetch("my-key", 0L, 25L);
+        final WindowStoreIterator<String> iterator = windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L));
         assertEquals(false, iterator.hasNext());
     }
 
@@ -100,10 +101,10 @@ public class CompositeReadOnlyWindowStoreTest {
         underlyingWindowStore.put("key-one", "value-one", 0L);
         secondUnderlying.put("key-two", "value-two", 10L);
 
-        final List<KeyValue<Long, String>> keyOneResults = StreamsTestUtils.toList(windowStore.fetch("key-one", 0L,
-                                                                                                     1L));
-        final List<KeyValue<Long, String>> keyTwoResults = StreamsTestUtils.toList(windowStore.fetch("key-two", 10L,
-                                                                                                     11L));
+        final List<KeyValue<Long, String>> keyOneResults = StreamsTestUtils.toList(windowStore.fetch("key-one", ofEpochMilli(0L),
+                                                                                                     ofEpochMilli(1L)));
+        final List<KeyValue<Long, String>> keyTwoResults = StreamsTestUtils.toList(windowStore.fetch("key-two", ofEpochMilli(10L),
+                                                                                                     ofEpochMilli(11L)));
 
         assertEquals(Collections.singletonList(KeyValue.pair(0L, "value-one")), keyOneResults);
         assertEquals(Collections.singletonList(KeyValue.pair(10L, "value-two")), keyTwoResults);
@@ -114,14 +115,14 @@ public class CompositeReadOnlyWindowStoreTest {
         otherUnderlyingStore.put("some-key", "some-value", 0L);
         underlyingWindowStore.put("some-key", "my-value", 1L);
 
-        final List<KeyValue<Long, String>> results = StreamsTestUtils.toList(windowStore.fetch("some-key", 0L, 2L));
+        final List<KeyValue<Long, String>> results = StreamsTestUtils.toList(windowStore.fetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L)));
         assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results);
     }
 
     @Test(expected = InvalidStateStoreException.class)
     public void shouldThrowInvalidStateStoreExceptionOnRebalance() {
         final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new StateStoreProviderStub(true), QueryableStoreTypes.windowStore(), "foo");
-        store.fetch("key", 1, 10);
+        store.fetch("key", ofEpochMilli(1), ofEpochMilli(10));
     }
 
     @Test
@@ -130,7 +131,7 @@ public class CompositeReadOnlyWindowStoreTest {
         final CompositeReadOnlyWindowStore<Object, Object> store =
                 new CompositeReadOnlyWindowStore<>(stubProviderOne, QueryableStoreTypes.windowStore(), "window-store");
         try {
-            store.fetch("key", 1, 10);
+            store.fetch("key", ofEpochMilli(1), ofEpochMilli(10));
             Assert.fail("InvalidStateStoreException was expected");
         } catch (final InvalidStateStoreException e) {
             Assert.assertEquals("State store is not available anymore and may have been migrated to another instance; " +
@@ -142,7 +143,7 @@ public class CompositeReadOnlyWindowStoreTest {
     public void emptyIteratorAlwaysReturnsFalse() {
         final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new
                 StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo");
-        final WindowStoreIterator<Object> windowStoreIterator = store.fetch("key", 1, 10);
+        final WindowStoreIterator<Object> windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10));
 
         Assert.assertFalse(windowStoreIterator.hasNext());
     }
@@ -151,7 +152,7 @@ public class CompositeReadOnlyWindowStoreTest {
     public void emptyIteratorPeekNextKeyShouldThrowNoSuchElementException() {
         final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new
                 StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo");
-        final WindowStoreIterator<Object> windowStoreIterator = store.fetch("key", 1, 10);
+        final WindowStoreIterator<Object> windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10));
 
         windowStoreIteratorException.expect(NoSuchElementException.class);
         windowStoreIterator.peekNextKey();
@@ -161,7 +162,7 @@ public class CompositeReadOnlyWindowStoreTest {
     public void emptyIteratorNextShouldThrowNoSuchElementException() {
         final CompositeReadOnlyWindowStore<Object, Object> store = new CompositeReadOnlyWindowStore<>(new
                 StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo");
-        final WindowStoreIterator<Object> windowStoreIterator = store.fetch("key", 1, 10);
+        final WindowStoreIterator<Object> windowStoreIterator = store.fetch("key", ofEpochMilli(1), ofEpochMilli(10));
 
         windowStoreIteratorException.expect(NoSuchElementException.class);
         windowStoreIterator.next();
@@ -173,7 +174,7 @@ public class CompositeReadOnlyWindowStoreTest {
         stubProviderTwo.addStore(storeName, secondUnderlying);
         underlyingWindowStore.put("a", "a", 0L);
         secondUnderlying.put("b", "b", 10L);
-        final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.fetch("a", "b", 0, 10));
+        final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.fetch("a", "b", ofEpochMilli(0), ofEpochMilli(10)));
         assertThat(results, equalTo(Arrays.asList(
                 KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"),
                 KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
@@ -204,7 +205,7 @@ public class CompositeReadOnlyWindowStoreTest {
                 KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"),
                 KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
     }
-    
+
     @Test
     public void shouldFetchAllAcrossStores() {
         final ReadOnlyWindowStoreStub<String, String> secondUnderlying = new
@@ -212,7 +213,7 @@ public class CompositeReadOnlyWindowStoreTest {
         stubProviderTwo.addStore(storeName, secondUnderlying);
         underlyingWindowStore.put("a", "a", 0L);
         secondUnderlying.put("b", "b", 10L);
-        final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.fetchAll(0, 10));
+        final List<KeyValue<Windowed<String>, String>> results = StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(0), ofEpochMilli(10)));
         assertThat(results, equalTo(Arrays.asList(
                 KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"),
                 KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"))));
@@ -220,17 +221,17 @@ public class CompositeReadOnlyWindowStoreTest {
 
     @Test(expected = NullPointerException.class)
     public void shouldThrowNPEIfKeyIsNull() {
-        windowStore.fetch(null, 0, 0);
+        windowStore.fetch(null, ofEpochMilli(0), ofEpochMilli(0));
     }
 
     @Test(expected = NullPointerException.class)
     public void shouldThrowNPEIfFromKeyIsNull() {
-        windowStore.fetch(null, "a", 0, 0);
+        windowStore.fetch(null, "a", ofEpochMilli(0), ofEpochMilli(0));
     }
 
     @Test(expected = NullPointerException.class)
     public void shouldThrowNPEIfToKeyIsNull() {
-        windowStore.fetch("a", null, 0, 0);
+        windowStore.fetch("a", null, ofEpochMilli(0), ofEpochMilli(0));
     }
 
 }
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java
index 1ac6d94..3a6a3b4 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java
@@ -40,6 +40,7 @@ import org.junit.Test;
 
 import java.util.Map;
 
+import static java.time.Instant.ofEpochMilli;
 import static java.util.Collections.singletonMap;
 import static org.apache.kafka.test.StreamsTestUtils.getMetricByNameFilterByTags;
 import static org.junit.Assert.assertEquals;
@@ -114,7 +115,7 @@ public class MeteredWindowStoreTest {
         EasyMock.replay(innerStoreMock);
 
         store.init(context, store);
-        store.fetch("a", 1, 1).close(); // recorded on close;
+        store.fetch("a", ofEpochMilli(1), ofEpochMilli(1)).close(); // recorded on close;
         final Map<MetricName, ? extends Metric> metrics = context.metrics().metrics();
         assertEquals(1.0, getMetricByNameFilterByTags(metrics, "fetch-total", "stream-scope-metrics", singletonMap("scope-id", "all")).metricValue());
         assertEquals(1.0, getMetricByNameFilterByTags(metrics, "fetch-total", "stream-scope-metrics", singletonMap("scope-id", "mocked-store")).metricValue());
@@ -127,7 +128,7 @@ public class MeteredWindowStoreTest {
         EasyMock.replay(innerStoreMock);
 
         store.init(context, store);
-        store.fetch("a", "b", 1, 1).close(); // recorded on close;
+        store.fetch("a", "b", ofEpochMilli(1), ofEpochMilli(1)).close(); // recorded on close;
         final Map<MetricName, ? extends Metric> metrics = context.metrics().metrics();
         assertEquals(1.0, getMetricByNameFilterByTags(metrics, "fetch-total", "stream-scope-metrics", singletonMap("scope-id", "all")).metricValue());
         assertEquals(1.0, getMetricByNameFilterByTags(metrics, "fetch-total", "stream-scope-metrics", singletonMap("scope-id", "mocked-store")).metricValue());
@@ -171,4 +172,4 @@ public class MeteredWindowStoreTest {
         assertNull(store.fetch("a", 0));
     }
 
-}
\ No newline at end of file
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java
index 4818d09..99abdc4 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java
@@ -16,6 +16,8 @@
  */
 package org.apache.kafka.streams.state.internals;
 
+import java.time.Instant;
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.errors.InvalidStateStoreException;
 import org.apache.kafka.streams.kstream.Windowed;
@@ -74,6 +76,13 @@ public class ReadOnlyWindowStoreStub<K, V> implements ReadOnlyWindowStore<K, V>,
     }
 
     @Override
+    public WindowStoreIterator<V> fetch(final K key, final Instant from, final Instant to) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(from, "from");
+        ApiUtils.validateMillisecondInstant(to, "to");
+        return fetch(key, from.toEpochMilli(), to.toEpochMilli());
+    }
+
+    @Override
     public KeyValueIterator<Windowed<K>, V> all() {
         if (!open) {
             throw new InvalidStateStoreException("Store is not open");
@@ -165,6 +174,13 @@ public class ReadOnlyWindowStoreStub<K, V> implements ReadOnlyWindowStore<K, V>,
     }
 
     @Override
+    public KeyValueIterator<Windowed<K>, V> fetchAll(final Instant from, final Instant to) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(from, "from");
+        ApiUtils.validateMillisecondInstant(to, "to");
+        return fetchAll(from.toEpochMilli(), to.toEpochMilli());
+    }
+
+    @Override
     public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) {
         if (!open) {
             throw new InvalidStateStoreException("Store is not open");
@@ -209,6 +225,15 @@ public class ReadOnlyWindowStoreStub<K, V> implements ReadOnlyWindowStore<K, V>,
         };
     }
 
+    @Override public KeyValueIterator<Windowed<K>, V> fetch(final K from,
+                                                            final K to,
+                                                            final Instant fromTime,
+                                                            final Instant toTime) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondInstant(fromTime, "fromTime");
+        ApiUtils.validateMillisecondInstant(toTime, "toTime");
+        return fetch(from, to, fromTime.toEpochMilli(), toTime.toEpochMilli());
+    }
+
     public void put(final K key, final V value, final long timestamp) {
         if (!data.containsKey(timestamp)) {
             data.put(timestamp, new TreeMap<K, V>());
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
index 201a31e..08f019f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
@@ -54,6 +54,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static java.time.Instant.ofEpochMilli;
 import static java.util.Objects.requireNonNull;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.MatcherAssert.assertThat;
@@ -142,7 +143,7 @@ public class RocksDBWindowStoreTest {
         setCurrentTime(currentTime);
         windowStore.put(1, "three");
 
-        final WindowStoreIterator<String> iterator = windowStore.fetch(1, 0, currentTime);
+        final WindowStoreIterator<String> iterator = windowStore.fetch(1, ofEpochMilli(0), ofEpochMilli(currentTime));
 
         // roll to the next segment that will close the first
         currentTime = currentTime + segmentInterval;
@@ -177,12 +178,12 @@ public class RocksDBWindowStoreTest {
         assertEquals("four", windowStore.fetch(4, startTime + 4L));
         assertEquals("five", windowStore.fetch(5, startTime + 5L));
 
-        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime + 0L - windowSize, startTime + 0L + windowSize)));
-        assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + 1L - windowSize, startTime + 1L + windowSize)));
-        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + 3L - windowSize, startTime + 3L + windowSize)));
-        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + 4L - windowSize, startTime + 4L + windowSize)));
-        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + 5L - windowSize, startTime + 5L + windowSize)));
+        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize))));
+        assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L + windowSize))));
+        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L + windowSize))));
+        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L + windowSize))));
+        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L + windowSize))));
 
         putSecondBatch(windowStore, startTime, context);
 
@@ -193,21 +194,21 @@ public class RocksDBWindowStoreTest {
         assertEquals("two+5", windowStore.fetch(2, startTime + 7L));
         assertEquals("two+6", windowStore.fetch(2, startTime + 8L));
 
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime - 2L - windowSize, startTime - 2L + windowSize)));
-        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime - 1L - windowSize, startTime - 1L + windowSize)));
-        assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, startTime - windowSize, startTime + windowSize)));
-        assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, startTime + 1L - windowSize, startTime + 1L + windowSize)));
-        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L + windowSize)));
-        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, startTime + 3L - windowSize, startTime + 3L + windowSize)));
-        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, startTime + 4L - windowSize, startTime + 4L + windowSize)));
-        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 5L - windowSize, startTime + 5L + windowSize)));
-        assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 6L - windowSize, startTime + 6L + windowSize)));
-        assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 7L - windowSize, startTime + 7L + windowSize)));
-        assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 8L - windowSize, startTime + 8L + windowSize)));
-        assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 9L - windowSize, startTime + 9L + windowSize)));
-        assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, startTime + 10L - windowSize, startTime + 10L + windowSize)));
-        assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, startTime + 11L - windowSize, startTime + 11L + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 12L - windowSize, startTime + 12L + windowSize)));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 2L - windowSize), ofEpochMilli(startTime - 2L + windowSize))));
+        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime - 1L - windowSize), ofEpochMilli(startTime - 1L + windowSize))));
+        assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
+        assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L + windowSize))));
+        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize))));
+        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L + windowSize))));
+        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L + windowSize))));
+        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L + windowSize))));
+        assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 6L - windowSize), ofEpochMilli(startTime + 6L + windowSize))));
+        assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 7L - windowSize), ofEpochMilli(startTime + 7L + windowSize))));
+        assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 8L - windowSize), ofEpochMilli(startTime + 8L + windowSize))));
+        assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 9L - windowSize), ofEpochMilli(startTime + 9L + windowSize))));
+        assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 10L - windowSize), ofEpochMilli(startTime + 10L + windowSize))));
+        assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L - windowSize), ofEpochMilli(startTime + 11L + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 12L - windowSize), ofEpochMilli(startTime + 12L + windowSize))));
 
         // Flush the store and verify all current entries were properly flushed ...
         windowStore.flush();
@@ -257,17 +258,17 @@ public class RocksDBWindowStoreTest {
 
         assertEquals(
             Utils.mkList(one, two, four),
-            StreamsTestUtils.toList(windowStore.fetchAll(startTime + 1, startTime + 4))
+            StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4)))
         );
 
         assertEquals(
             Utils.mkList(zero, one, two),
-            StreamsTestUtils.toList(windowStore.fetchAll(startTime + 0, startTime + 3))
+            StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3)))
         );
 
         assertEquals(
             Utils.mkList(one, two, four, five),
-            StreamsTestUtils.toList(windowStore.fetchAll(startTime + 1, startTime + 5))
+            StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5)))
         );
     }
 
@@ -286,36 +287,36 @@ public class RocksDBWindowStoreTest {
 
         assertEquals(
             Utils.mkList(zero, one),
-            StreamsTestUtils.toList(windowStore.fetch(0, 1, startTime + 0L - windowSize, startTime + 0L + windowSize))
+            StreamsTestUtils.toList(windowStore.fetch(0, 1, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize)))
         );
         assertEquals(
             Utils.mkList(one),
-            StreamsTestUtils.toList(windowStore.fetch(1, 1, startTime + 0L - windowSize, startTime + 0L + windowSize))
+            StreamsTestUtils.toList(windowStore.fetch(1, 1, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize)))
         );
         assertEquals(
             Utils.mkList(one, two),
-            StreamsTestUtils.toList(windowStore.fetch(1, 3, startTime + 0L - windowSize, startTime + 0L + windowSize))
+            StreamsTestUtils.toList(windowStore.fetch(1, 3, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize)))
         );
         assertEquals(
             Utils.mkList(zero, one, two),
-            StreamsTestUtils.toList(windowStore.fetch(0, 5, startTime + 0L - windowSize, startTime + 0L + windowSize))
+            StreamsTestUtils.toList(windowStore.fetch(0, 5, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize)))
         );
         assertEquals(
             Utils.mkList(zero, one, two,
                 four, five),
-            StreamsTestUtils.toList(windowStore.fetch(0, 5, startTime + 0L - windowSize, startTime + 0L + windowSize + 5L))
+            StreamsTestUtils.toList(windowStore.fetch(0, 5, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize + 5L)))
         );
         assertEquals(
             Utils.mkList(two, four, five),
-            StreamsTestUtils.toList(windowStore.fetch(0, 5, startTime + 2L, startTime + 0L + windowSize + 5L))
+            StreamsTestUtils.toList(windowStore.fetch(0, 5, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + 0L + windowSize + 5L)))
         );
         assertEquals(
             Utils.mkList(),
-            StreamsTestUtils.toList(windowStore.fetch(4, 5, startTime + 2L, startTime + windowSize))
+            StreamsTestUtils.toList(windowStore.fetch(4, 5, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + windowSize)))
         );
         assertEquals(
             Utils.mkList(),
-            StreamsTestUtils.toList(windowStore.fetch(0, 3, startTime + 3L, startTime + windowSize + 5))
+            StreamsTestUtils.toList(windowStore.fetch(0, 3, ofEpochMilli(startTime + 3L), ofEpochMilli(startTime + windowSize + 5)))
         );
     }
 
@@ -326,30 +327,30 @@ public class RocksDBWindowStoreTest {
 
         putFirstBatch(windowStore, startTime, context);
 
-        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime + 0L - windowSize, startTime + 0L)));
-        assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + 1L - windowSize, startTime + 1L)));
-        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + 3L - windowSize, startTime + 3L)));
-        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + 4L - windowSize, startTime + 4L)));
-        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + 5L - windowSize, startTime + 5L)));
+        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L))));
+        assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L))));
+        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L))));
+        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L))));
+        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L))));
 
         putSecondBatch(windowStore, startTime, context);
 
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime - 1L - windowSize, startTime - 1L)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 0L - windowSize, startTime + 0L)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 1L - windowSize, startTime + 1L)));
-        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L)));
-        assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, startTime + 3L - windowSize, startTime + 3L)));
-        assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, startTime + 4L - windowSize, startTime + 4L)));
-        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, startTime + 5L - windowSize, startTime + 5L)));
-        assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, startTime + 6L - windowSize, startTime + 6L)));
-        assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, startTime + 7L - windowSize, startTime + 7L)));
-        assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 8L - windowSize, startTime + 8L)));
-        assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 9L - windowSize, startTime + 9L)));
-        assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, startTime + 10L - windowSize, startTime + 10L)));
-        assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, startTime + 11L - windowSize, startTime + 11L)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 12L - windowSize, startTime + 12L)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 13L - windowSize, startTime + 13L)));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 1L - windowSize), ofEpochMilli(startTime - 1L))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L))));
+        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L))));
+        assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L))));
+        assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L))));
+        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L))));
+        assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 6L - windowSize), ofEpochMilli(startTime + 6L))));
+        assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 7L - windowSize), ofEpochMilli(startTime + 7L))));
+        assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 8L - windowSize), ofEpochMilli(startTime + 8L))));
+        assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 9L - windowSize), ofEpochMilli(startTime + 9L))));
+        assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 10L - windowSize), ofEpochMilli(startTime + 10L))));
+        assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L - windowSize), ofEpochMilli(startTime + 11L))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 12L - windowSize), ofEpochMilli(startTime + 12L))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 13L - windowSize), ofEpochMilli(startTime + 13L))));
 
         // Flush the store and verify all current entries were properly flushed ...
         windowStore.flush();
@@ -372,30 +373,30 @@ public class RocksDBWindowStoreTest {
 
         putFirstBatch(windowStore, startTime, context);
 
-        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime + 0L, startTime + 0L + windowSize)));
-        assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + 1L, startTime + 1L + windowSize)));
-        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L, startTime + 2L + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + 3L, startTime + 3L + windowSize)));
-        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + 4L, startTime + 4L + windowSize)));
-        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + 5L, startTime + 5L + windowSize)));
+        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L), ofEpochMilli(startTime + 0L + windowSize))));
+        assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L), ofEpochMilli(startTime + 1L + windowSize))));
+        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + 2L + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L), ofEpochMilli(startTime + 3L + windowSize))));
+        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L), ofEpochMilli(startTime + 4L + windowSize))));
+        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L), ofEpochMilli(startTime + 5L + windowSize))));
 
         putSecondBatch(windowStore, startTime, context);
 
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime - 2L, startTime - 2L + windowSize)));
-        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime - 1L, startTime - 1L + windowSize)));
-        assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, startTime, startTime + windowSize)));
-        assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, startTime + 1L, startTime + 1L + windowSize)));
-        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, startTime + 2L, startTime + 2L + windowSize)));
-        assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, startTime + 3L, startTime + 3L + windowSize)));
-        assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, startTime + 4L, startTime + 4L + windowSize)));
-        assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 5L, startTime + 5L + windowSize)));
-        assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 6L, startTime + 6L + windowSize)));
-        assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, startTime + 7L, startTime + 7L + windowSize)));
-        assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, startTime + 8L, startTime + 8L + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 9L, startTime + 9L + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 10L, startTime + 10L + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 11L, startTime + 11L + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 12L, startTime + 12L + windowSize)));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 2L), ofEpochMilli(startTime - 2L + windowSize))));
+        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime - 1L), ofEpochMilli(startTime - 1L + windowSize))));
+        assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, ofEpochMilli(startTime), ofEpochMilli(startTime + windowSize))));
+        assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 1L), ofEpochMilli(startTime + 1L + windowSize))));
+        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + 2L + windowSize))));
+        assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 3L), ofEpochMilli(startTime + 3L + windowSize))));
+        assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 4L), ofEpochMilli(startTime + 4L + windowSize))));
+        assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 5L), ofEpochMilli(startTime + 5L + windowSize))));
+        assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 6L), ofEpochMilli(startTime + 6L + windowSize))));
+        assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 7L), ofEpochMilli(startTime + 7L + windowSize))));
+        assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 8L), ofEpochMilli(startTime + 8L + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 9L), ofEpochMilli(startTime + 9L + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 10L), ofEpochMilli(startTime + 10L + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L), ofEpochMilli(startTime + 11L + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 12L), ofEpochMilli(startTime + 12L + windowSize))));
 
         // Flush the store and verify all current entries were properly flushed ...
         windowStore.flush();
@@ -419,17 +420,17 @@ public class RocksDBWindowStoreTest {
         setCurrentTime(startTime);
         windowStore.put(0, "zero");
 
-        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
+        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
 
         windowStore.put(0, "zero");
         windowStore.put(0, "zero+");
         windowStore.put(0, "zero++");
 
-        assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
-        assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, startTime + 1L - windowSize, startTime + 1L + windowSize)));
-        assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, startTime + 2L - windowSize, startTime + 2L + windowSize)));
-        assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, startTime + 3L - windowSize, startTime + 3L + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime + 4L - windowSize, startTime + 4L + windowSize)));
+        assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
+        assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L + windowSize))));
+        assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize))));
+        assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L + windowSize))));
 
         // Flush the store and verify all current entries were properly flushed ...
         windowStore.flush();
@@ -487,12 +488,12 @@ public class RocksDBWindowStoreTest {
             segmentDirs(baseDir)
         );
 
-        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
-        assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
-        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize)));
-        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize)));
-        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize)));
+        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
+        assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize))));
+        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize))));
+        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize))));
+        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize))));
 
         setCurrentTime(startTime + increment * 6);
         windowStore.put(6, "six");
@@ -506,13 +507,13 @@ public class RocksDBWindowStoreTest {
         );
 
 
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
-        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize)));
-        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize)));
-        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize)));
-        assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize)));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize))));
+        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize))));
+        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize))));
+        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize))));
+        assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize))));
 
 
         setCurrentTime(startTime + increment * 7);
@@ -526,14 +527,14 @@ public class RocksDBWindowStoreTest {
             segmentDirs(baseDir)
         );
 
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
-        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize)));
-        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize)));
-        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize)));
-        assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize)));
-        assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize)));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize))));
+        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize))));
+        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize))));
+        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize))));
+        assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize))));
+        assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize))));
 
         setCurrentTime(startTime + increment * 8);
         windowStore.put(8, "eight");
@@ -547,15 +548,15 @@ public class RocksDBWindowStoreTest {
         );
 
 
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize)));
-        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize)));
-        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize)));
-        assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize)));
-        assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize)));
-        assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize)));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize))));
+        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize))));
+        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize))));
+        assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize))));
+        assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize))));
+        assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize))));
 
         // check segment directories
         windowStore.flush();
@@ -603,27 +604,27 @@ public class RocksDBWindowStoreTest {
         Utils.delete(baseDir);
 
         windowStore = createWindowStore(context, false);
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(8, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize)));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize))));
 
         context.restore(windowName, changeLog);
 
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + increment - windowSize, startTime + increment + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + increment * 2 - windowSize, startTime + increment * 2 + windowSize)));
-        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + increment * 3 - windowSize, startTime + increment * 3 + windowSize)));
-        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + increment * 4 - windowSize, startTime + increment * 4 + windowSize)));
-        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + increment * 5 - windowSize, startTime + increment * 5 + windowSize)));
-        assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + increment * 6 - windowSize, startTime + increment * 6 + windowSize)));
-        assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + increment * 7 - windowSize, startTime + increment * 7 + windowSize)));
-        assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, startTime + increment * 8 - windowSize, startTime + increment * 8 + windowSize)));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize))));
+        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize))));
+        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize))));
+        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize))));
+        assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize))));
+        assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize))));
+        assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize))));
 
         // check segment directories
         windowStore.flush();
@@ -662,7 +663,7 @@ public class RocksDBWindowStoreTest {
         WindowStoreIterator iter;
         int fetchedCount;
 
-        iter = windowStore.fetch(0, 0L, segmentInterval * 4);
+        iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(segmentInterval * 4));
         fetchedCount = 0;
         while (iter.hasNext()) {
             iter.next();
@@ -678,7 +679,7 @@ public class RocksDBWindowStoreTest {
         setCurrentTime(segmentInterval * 3);
         windowStore.put(0, "v");
 
-        iter = windowStore.fetch(0, 0L, segmentInterval * 4);
+        iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(segmentInterval * 4));
         fetchedCount = 0;
         while (iter.hasNext()) {
             iter.next();
@@ -694,7 +695,7 @@ public class RocksDBWindowStoreTest {
         setCurrentTime(segmentInterval * 5);
         windowStore.put(0, "v");
 
-        iter = windowStore.fetch(0, segmentInterval * 4, segmentInterval * 10);
+        iter = windowStore.fetch(0, ofEpochMilli(segmentInterval * 4), ofEpochMilli(segmentInterval * 10));
         fetchedCount = 0;
         while (iter.hasNext()) {
             iter.next();
@@ -736,7 +737,7 @@ public class RocksDBWindowStoreTest {
 
         assertEquals(expected, actual);
 
-        try (final WindowStoreIterator iter = windowStore.fetch(0, 0L, 1000000L)) {
+        try (final WindowStoreIterator iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(1000000L))) {
             while (iter.hasNext()) {
                 iter.next();
             }
@@ -756,7 +757,7 @@ public class RocksDBWindowStoreTest {
         windowStore.put(1, "two", 2L);
         windowStore.put(1, "three", 3L);
 
-        final WindowStoreIterator<String> iterator = windowStore.fetch(1, 1L, 3L);
+        final WindowStoreIterator<String> iterator = windowStore.fetch(1, ofEpochMilli(1L), ofEpochMilli(3L));
         assertTrue(iterator.hasNext());
         windowStore.close();
 
@@ -784,16 +785,17 @@ public class RocksDBWindowStoreTest {
 
 
         final List expected = Utils.mkList("0001", "0003", "0005");
-        assertThat(toList(windowStore.fetch("a", 0, Long.MAX_VALUE)), equalTo(expected));
+        assertThat(toList(windowStore.fetch("a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expected));
 
-        List<KeyValue<Windowed<String>, String>> list = StreamsTestUtils.toList(windowStore.fetch("a", "a", 0, Long.MAX_VALUE));
+        List<KeyValue<Windowed<String>, String>> list =
+            StreamsTestUtils.toList(windowStore.fetch("a", "a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)));
         assertThat(list, equalTo(Utils.mkList(
             windowedPair("a", "0001", 0, windowSize),
             windowedPair("a", "0003", 1, windowSize),
             windowedPair("a", "0005", 0x7a00000000000000L - 1, windowSize)
         )));
 
-        list = StreamsTestUtils.toList(windowStore.fetch("aa", "aa", 0, Long.MAX_VALUE));
+        list = StreamsTestUtils.toList(windowStore.fetch("aa", "aa", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)));
         assertThat(list, equalTo(Utils.mkList(
             windowedPair("aa", "0002", 0, windowSize),
             windowedPair("aa", "0004", 1, windowSize)
@@ -815,19 +817,19 @@ public class RocksDBWindowStoreTest {
     @Test(expected = NullPointerException.class)
     public void shouldThrowNullPointerExceptionOnGetNullKey() {
         windowStore = createWindowStore(context, false);
-        windowStore.fetch(null, 1L, 2L);
+        windowStore.fetch(null, ofEpochMilli(1L), ofEpochMilli(2L));
     }
 
     @Test(expected = NullPointerException.class)
     public void shouldThrowNullPointerExceptionOnRangeNullFromKey() {
         windowStore = createWindowStore(context, false);
-        windowStore.fetch(null, 2, 1L, 2L);
+        windowStore.fetch(null, 2, ofEpochMilli(1L), ofEpochMilli(2L));
     }
 
     @Test(expected = NullPointerException.class)
     public void shouldThrowNullPointerExceptionOnRangeNullToKey() {
         windowStore = createWindowStore(context, false);
-        windowStore.fetch(1, null, 1L, 2L);
+        windowStore.fetch(1, null, ofEpochMilli(1L), ofEpochMilli(2L));
     }
 
     @Test
@@ -866,11 +868,11 @@ public class RocksDBWindowStoreTest {
         windowStore.put(key3, "9", 59999);
 
         final List expectedKey1 = Utils.mkList("1", "4", "7");
-        assertThat(toList(windowStore.fetch(key1, 0, Long.MAX_VALUE)), equalTo(expectedKey1));
+        assertThat(toList(windowStore.fetch(key1, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey1));
         final List expectedKey2 = Utils.mkList("2", "5", "8");
-        assertThat(toList(windowStore.fetch(key2, 0, Long.MAX_VALUE)), equalTo(expectedKey2));
+        assertThat(toList(windowStore.fetch(key2, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey2));
         final List expectedKey3 = Utils.mkList("3", "6", "9");
-        assertThat(toList(windowStore.fetch(key3, 0, Long.MAX_VALUE)), equalTo(expectedKey3));
+        assertThat(toList(windowStore.fetch(key3, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey3));
     }
 
     private void putFirstBatch(final WindowStore<Integer, String> store,
diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java
index 711cdc1..ca059b4 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java
@@ -78,7 +78,7 @@ public class StreamThreadStateStoreProviderTest {
         topology.addStateStore(Stores.keyValueStoreBuilder(Stores.inMemoryKeyValueStore("kv-store"), Serdes.String(), Serdes.String()), "the-processor");
         topology.addStateStore(
             Stores.windowStoreBuilder(
-                Stores.persistentWindowStore("window-store", 10L, 2L, false),
+                Stores.persistentWindowStore("window-store", Duration.ofMillis(10L), Duration.ofMillis(2L), false),
                 Serdes.String(),
                 Serdes.String()),
             "the-processor"
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java
index 01333db..6f801c9 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java
@@ -41,7 +41,6 @@ import java.time.Duration;
 import java.util.Collections;
 import java.util.Locale;
 import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 
 public class BrokerCompatibilityTest {
 
@@ -108,7 +107,7 @@ public class BrokerCompatibilityTest {
                 System.err.println("FATAL: An unexpected exception " + cause);
                 e.printStackTrace(System.err);
                 System.err.flush();
-                streams.close(30, TimeUnit.SECONDS);
+                streams.close(Duration.ofSeconds(30));
             }
         });
         System.out.println("start Kafka Streams");
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java
index 1f8238e..e292f16 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.tests;
 
+import java.time.Duration;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.streams.KafkaStreams;
@@ -56,7 +57,7 @@ public class EosTestClient extends SmokeTestUtil {
             @Override
             public void run() {
                 isRunning = false;
-                streams.close(TimeUnit.SECONDS.toMillis(300), TimeUnit.SECONDS);
+                streams.close(Duration.ofSeconds(300));
 
                 // need to wait for callback to avoid race condition
                 // -> make sure the callback printout to stdout is there as it is expected test output
@@ -102,7 +103,7 @@ public class EosTestClient extends SmokeTestUtil {
                 streams.start();
             }
             if (uncaughtException) {
-                streams.close(TimeUnit.SECONDS.toMillis(60), TimeUnit.SECONDS);
+                streams.close(Duration.ofSeconds(60_000L));
                 streams = null;
             }
             sleep(1000);
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java
index 0bbb889..a6a2ebb 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.tests;
 
+import java.time.Duration;
 import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.clients.producer.ProducerRecord;
@@ -30,7 +31,6 @@ import org.apache.kafka.streams.kstream.ForeachAction;
 import org.apache.kafka.streams.kstream.KStream;
 
 import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 
 public class ShutdownDeadlockTest {
 
@@ -65,7 +65,7 @@ public class ShutdownDeadlockTest {
         Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
             @Override
             public void run() {
-                streams.close(5, TimeUnit.SECONDS);
+                streams.close(Duration.ofSeconds(5));
             }
         }));
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
index 51dc05c..ddff7a8 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.tests;
 
+import java.time.Duration;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.Serdes;
@@ -40,7 +41,6 @@ import org.apache.kafka.streams.state.Stores;
 import org.apache.kafka.streams.state.WindowStore;
 
 import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 
 public class SmokeTestClient extends SmokeTestUtil {
 
@@ -82,7 +82,7 @@ public class SmokeTestClient extends SmokeTestUtil {
     }
 
     public void close() {
-        streams.close(5, TimeUnit.SECONDS);
+        streams.close(Duration.ofSeconds(5));
         // do not remove these printouts since they are needed for health scripts
         if (!uncaughtException) {
             System.out.println("SMOKE-TEST-CLIENT-CLOSED");
@@ -129,7 +129,7 @@ public class SmokeTestClient extends SmokeTestUtil {
             data.groupByKey(Serialized.with(stringSerde, intSerde));
 
         groupedData
-            .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(1)))
+            .windowedBy(TimeWindows.of(Duration.ofDays(1)))
             .aggregate(
                 new Initializer<Integer>() {
                     public Integer apply() {
@@ -154,7 +154,7 @@ public class SmokeTestClient extends SmokeTestUtil {
 
         // max
         groupedData
-            .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2)))
+            .windowedBy(TimeWindows.of(Duration.ofDays(2)))
             .aggregate(
                 new Initializer<Integer>() {
                     public Integer apply() {
@@ -179,7 +179,7 @@ public class SmokeTestClient extends SmokeTestUtil {
 
         // sum
         groupedData
-            .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2)))
+            .windowedBy(TimeWindows.of(Duration.ofDays(2)))
             .aggregate(
                 new Initializer<Long>() {
                     public Long apply() {
@@ -202,7 +202,7 @@ public class SmokeTestClient extends SmokeTestUtil {
 
         // cnt
         groupedData
-            .windowedBy(TimeWindows.of(TimeUnit.DAYS.toMillis(2)))
+            .windowedBy(TimeWindows.of(Duration.ofDays(2)))
             .count(Materialized.<String, Long, WindowStore<Bytes, byte[]>>as("uwin-cnt"))
             .toStream(new Unwindow<String, Long>())
             .to("cnt", Produced.with(stringSerde, longSerde));
@@ -252,7 +252,7 @@ public class SmokeTestClient extends SmokeTestUtil {
             @Override
             public void uncaughtException(final Thread t, final Throwable e) {
                 System.out.println("FATAL: An unexpected exception is encountered on thread " + t + ": " + e);
-                streamsClient.close(30, TimeUnit.SECONDS);
+                streamsClient.close(Duration.ofSeconds(30));
             }
         });
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java
index eb19688..25c642e 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.kafka.streams.tests;
 
+import java.time.Duration;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.Serde;
@@ -33,7 +34,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 
 public class StreamsBrokerDownResilienceTest {
 
@@ -109,7 +109,7 @@ public class StreamsBrokerDownResilienceTest {
             public void uncaughtException(final Thread t, final Throwable e) {
                 System.err.println("FATAL: An unexpected exception " + e);
                 System.err.flush();
-                streams.close(30, TimeUnit.SECONDS);
+                streams.close(Duration.ofSeconds(30));
             }
         });
         System.out.println("Start Kafka Streams");
@@ -118,7 +118,7 @@ public class StreamsBrokerDownResilienceTest {
         Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
             @Override
             public void run() {
-                streams.close(30, TimeUnit.SECONDS);
+                streams.close(Duration.ofSeconds(30));
                 System.out.println("Complete shutdown of streams resilience test app now");
                 System.out.flush();
             }
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java
index db160fe..66b20b2 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.kafka.streams.tests;
 
+import java.time.Duration;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.Serde;
@@ -38,7 +39,6 @@ import java.io.IOException;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
 
 public class StreamsStandByReplicaTest {
 
@@ -164,7 +164,7 @@ public class StreamsStandByReplicaTest {
     }
 
     private static void shutdown(final KafkaStreams streams) {
-        streams.close(10, TimeUnit.SECONDS);
+        streams.close(Duration.ofSeconds(10));
     }
 
     private static boolean confirmCorrectConfigs(final Properties properties) {
diff --git a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java
index 3b5a915..2f356bf 100644
--- a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java
+++ b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.test;
 
+import java.time.Duration;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.header.internals.RecordHeaders;
@@ -210,6 +211,13 @@ public class InternalMockProcessorContext extends AbstractProcessorContext imple
     }
 
     @Override
+    public Cancellable schedule(final Duration interval,
+                                final PunctuationType type,
+                                final Punctuator callback) throws IllegalArgumentException {
+        throw new UnsupportedOperationException("schedule() not supported.");
+    }
+
+    @Override
     public void commit() { }
 
     @Override
diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java
index c95f408..e13e144 100644
--- a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.test;
 
+import java.time.Duration;
 import org.apache.kafka.streams.processor.AbstractProcessor;
 import org.apache.kafka.streams.processor.Cancellable;
 import org.apache.kafka.streams.processor.ProcessorContext;
@@ -55,7 +56,7 @@ public class MockProcessor<K, V> extends AbstractProcessor<K, V> {
     public void init(final ProcessorContext context) {
         super.init(context);
         if (scheduleInterval > 0L) {
-            scheduleCancellable = context.schedule(scheduleInterval, punctuationType, new Punctuator() {
+            scheduleCancellable = context.schedule(Duration.ofMillis(scheduleInterval), punctuationType, new Punctuator() {
                 @Override
                 public void punctuate(final long timestamp) {
                     if (punctuationType == PunctuationType.STREAM_TIME) {
diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java
index ce98389..36d049c 100644
--- a/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java
+++ b/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.test;
 
+import java.time.Duration;
 import org.apache.kafka.common.metrics.Metrics;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.processor.Cancellable;
@@ -52,7 +53,15 @@ public class NoOpProcessorContext extends AbstractProcessorContext {
         return null;
     }
 
-    @Override public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) {
+    @Override
+    public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) {
+        return null;
+    }
+
+    @Override
+    public Cancellable schedule(final Duration interval,
+                                final PunctuationType type,
+                                final Punctuator callback) throws IllegalArgumentException {
         return null;
     }
 
diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KStreamTest.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KStreamTest.scala
index f339756..8626be5 100644
--- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KStreamTest.scala
+++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KStreamTest.scala
@@ -18,6 +18,8 @@
  */
 package org.apache.kafka.streams.scala.kstream
 
+import java.time.Duration.ofSeconds
+
 import org.apache.kafka.streams.kstream.JoinWindows
 import org.apache.kafka.streams.scala.ImplicitConversions._
 import org.apache.kafka.streams.scala.Serdes._
@@ -143,7 +145,7 @@ class KStreamTest extends FlatSpec with Matchers with TestDriver {
 
     val stream1 = builder.stream[String, String](sourceTopic1)
     val stream2 = builder.stream[String, String](sourceTopic2)
-    stream1.join(stream2)((a, b) => s"$a-$b", JoinWindows.of(1000)).to(sinkTopic)
+    stream1.join(stream2)((a, b) => s"$a-$b", JoinWindows.of(ofSeconds(1))).to(sinkTopic)
 
     val testDriver = createTestDriver(builder)
 
diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/MaterializedTest.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/MaterializedTest.scala
index 14fc5d4..5df2916 100644
--- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/MaterializedTest.scala
+++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/MaterializedTest.scala
@@ -18,6 +18,8 @@
  */
 package org.apache.kafka.streams.scala.kstream
 
+import java.time.Duration
+
 import org.apache.kafka.streams.kstream.internals.MaterializedInternal
 import org.apache.kafka.streams.scala.Serdes._
 import org.apache.kafka.streams.scala._
@@ -50,7 +52,7 @@ class MaterializedTest extends FlatSpec with Matchers {
   }
 
   "Create a Materialize with a window store supplier" should "create a Materialized with Serdes and a store supplier" in {
-    val storeSupplier = Stores.persistentWindowStore("store", 1, 1, true)
+    val storeSupplier = Stores.persistentWindowStore("store", Duration.ofMillis(1), Duration.ofMillis(1), true)
     val materialized: Materialized[String, Long, ByteArrayWindowStore] =
       Materialized.as[String, Long](storeSupplier)
 
@@ -72,7 +74,7 @@ class MaterializedTest extends FlatSpec with Matchers {
   }
 
   "Create a Materialize with a session store supplier" should "create a Materialized with Serdes and a store supplier" in {
-    val storeSupplier = Stores.persistentSessionStore("store", 1)
+    val storeSupplier = Stores.persistentSessionStore("store", Duration.ofMillis(1))
     val materialized: Materialized[String, Long, ByteArraySessionStore] =
       Materialized.as[String, Long](storeSupplier)
 
diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java
index fd3dcfe..d10a45c 100644
--- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java
+++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java
@@ -150,7 +150,7 @@ import java.util.regex.Pattern;
  * {@link ProducerRecord#equals(Object)} can simplify your code as you can ignore attributes you are not interested in.
  * <p>
  * Note, that calling {@code pipeInput()} will also trigger {@link PunctuationType#STREAM_TIME event-time} base
- * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) punctuation} callbacks.
+ * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) punctuation} callbacks.
  * However, you won't trigger {@link PunctuationType#WALL_CLOCK_TIME wall-clock} type punctuations that you must
  * trigger manually via {@link #advanceWallClockTime(long)}.
  * <p>
@@ -489,7 +489,7 @@ public class TopologyTestDriver implements Closeable {
     /**
      * Advances the internally mocked wall-clock time.
      * This might trigger a {@link PunctuationType#WALL_CLOCK_TIME wall-clock} type
-     * {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) punctuations}.
+     * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) punctuations}.
      *
      * @param advanceMs the amount of time to advance wall-clock time in milliseconds
      */
diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java
index 553428d..88a7fe7 100644
--- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java
+++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java
@@ -16,10 +16,12 @@
  */
 package org.apache.kafka.streams.processor;
 
+import java.time.Duration;
 import org.apache.kafka.common.annotation.InterfaceStability;
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.metrics.Metrics;
 import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.streams.internals.ApiUtils;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.StreamsMetrics;
@@ -378,6 +380,7 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S
     }
 
     @Override
+    @Deprecated
     public Cancellable schedule(final long intervalMs, final PunctuationType type, final Punctuator callback) {
         final CapturedPunctuator capturedPunctuator = new CapturedPunctuator(intervalMs, type, callback);
 
@@ -386,6 +389,14 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S
         return capturedPunctuator::cancel;
     }
 
+    @Override
+    public Cancellable schedule(final Duration interval,
+                                final PunctuationType type,
+                                final Punctuator callback) throws IllegalArgumentException {
+        ApiUtils.validateMillisecondDuration(interval, "interval");
+        return schedule(interval.toMillis(), type, callback);
+    }
+
     /**
      * Get the punctuators scheduled so far. The returned list is not affected by subsequent calls to {@code schedule(...)}.
      *
diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java
index 878aa35..ba52867 100644
--- a/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java
+++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams;
 
+import java.time.Duration;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.streams.processor.AbstractProcessor;
 import org.apache.kafka.streams.processor.MockProcessorContext;
@@ -345,7 +346,7 @@ public class MockProcessorContextTest {
             @Override
             public void init(final ProcessorContext context) {
                 context.schedule(
-                    1000L,
+                    Duration.ofSeconds(1L),
                     PunctuationType.WALL_CLOCK_TIME,
                     timestamp -> context.commit()
                 );
diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java
index 497a6c3..58f6e02 100644
--- a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java
+++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams;
 
+import java.time.Duration;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.header.Header;
@@ -231,7 +232,7 @@ public class TopologyTestDriverTest {
             initialized = true;
             this.context = context;
             for (final Punctuation punctuation : punctuations) {
-                this.context.schedule(punctuation.intervalMs, punctuation.punctuationType, punctuation.callback);
+                this.context.schedule(Duration.ofMillis(punctuation.intervalMs), punctuation.punctuationType, punctuation.callback);
             }
         }
 
@@ -863,8 +864,8 @@ public class TopologyTestDriverTest {
         @Override
         public void init(final ProcessorContext context) {
             this.context = context;
-            context.schedule(60000, PunctuationType.WALL_CLOCK_TIME, timestamp -> flushStore());
-            context.schedule(10000, PunctuationType.STREAM_TIME, timestamp -> flushStore());
+            context.schedule(Duration.ofMinutes(1), PunctuationType.WALL_CLOCK_TIME, timestamp -> flushStore());
+            context.schedule(Duration.ofSeconds(10), PunctuationType.STREAM_TIME, timestamp -> flushStore());
             store = (KeyValueStore<String, Long>) context.getStateStore("aggStore");
         }
 


Mime
View raw message