ableegoldman commented on a change in pull request #9039:
URL: https://github.com/apache/kafka/pull/9039#discussion_r471798256



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.kstream.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Initializer;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashSet;
+import static 
org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor;
+import static 
org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor;
+import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
+
+public class KStreamSlidingWindowAggregate<K, V, Agg> implements 
KStreamAggProcessorSupplier<K, Windowed<K>, V, Agg> {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final String storeName;
+    private final SlidingWindows windows;
+    private final Initializer<Agg> initializer;
+    private final Aggregator<? super K, ? super V, Agg> aggregator;
+
+    private boolean sendOldValues = false;
+
+    public KStreamSlidingWindowAggregate(final SlidingWindows windows,
+                                         final String storeName,
+                                         final Initializer<Agg> initializer,
+                                         final Aggregator<? super K, ? super 
V, Agg> aggregator) {
+        this.windows = windows;
+        this.storeName = storeName;
+        this.initializer = initializer;
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public Processor<K, V> get() {
+        return new KStreamSlidingWindowAggregateProcessor();
+    }
+
+    public SlidingWindows windows() {
+        return windows;
+    }
+
+    @Override
+    public void enableSendingOldValues() {
+        sendOldValues = true;
+    }
+
+    private class KStreamSlidingWindowAggregateProcessor extends 
AbstractProcessor<K, V> {
+        private TimestampedWindowStore<K, Agg> windowStore;
+        private TimestampedTupleForwarder<Windowed<K>, Agg> tupleForwarder;
+        private StreamsMetricsImpl metrics;
+        private InternalProcessorContext internalProcessorContext;
+        private Sensor lateRecordDropSensor;
+        private Sensor droppedRecordsSensor;
+        private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(final ProcessorContext context) {
+            super.init(context);
+            internalProcessorContext = (InternalProcessorContext) context;
+            metrics = internalProcessorContext.metrics();
+            final String threadId = Thread.currentThread().getName();
+            lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor(
+                threadId,
+                context.taskId().toString(),
+                internalProcessorContext.currentNode().name(),
+                metrics
+            );
+            droppedRecordsSensor = 
droppedRecordsSensorOrSkippedRecordsSensor(threadId, 
context.taskId().toString(), metrics);
+            windowStore = (TimestampedWindowStore<K, Agg>) 
context.getStateStore(storeName);
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+
+        @Override
+        public void process(final K key, final V value) {
+            if (key == null || value == null) {
+                log.warn(
+                    "Skipping record due to null key or value. value=[{}] 
topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), 
context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+
+            final long timestamp = context().timestamp();
+            //don't process records that don't fall within a full sliding 
window
+            if (timestamp < windows.timeDifferenceMs()) {
+                log.warn(
+                    "Skipping record due to early arrival. value=[{}] 
topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), 
context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+            processInOrder(key, value, timestamp);
+        }
+
+        public void processInOrder(final K key, final V value, final long 
timestamp) {
+
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+            final long closeTime = observedStreamTime - 
windows.gracePeriodMs();
+
+            //store start times of windows we find
+            final HashSet<Long> windowStartTimes = new HashSet<Long>();

Review comment:
       ```suggestion
               final Set<Long> windowStartTimes = new HashSet<>();
   ```
   Also I think this set is pretty clearly named, so we probably don't need a 
comment for it 

##########
File path: checkstyle/suppressions.xml
##########
@@ -167,6 +167,9 @@
     <suppress 
checks="(FinalLocalVariable|UnnecessaryParentheses|BooleanExpressionComplexity|CyclomaticComplexity|WhitespaceAfter|LocalVariableName)"
               files="Murmur3.java"/>
 
+    <suppress checks="(CyclomaticComplexity)"
+              files="CogroupedStreamAggregateBuilder.java"/>

Review comment:
       Do we still need this one after the cleanup you did?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.kstream.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Initializer;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashSet;
+import static 
org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor;
+import static 
org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor;
+import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
+
+public class KStreamSlidingWindowAggregate<K, V, Agg> implements 
KStreamAggProcessorSupplier<K, Windowed<K>, V, Agg> {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final String storeName;
+    private final SlidingWindows windows;
+    private final Initializer<Agg> initializer;
+    private final Aggregator<? super K, ? super V, Agg> aggregator;
+
+    private boolean sendOldValues = false;
+
+    public KStreamSlidingWindowAggregate(final SlidingWindows windows,
+                                         final String storeName,
+                                         final Initializer<Agg> initializer,
+                                         final Aggregator<? super K, ? super 
V, Agg> aggregator) {
+        this.windows = windows;
+        this.storeName = storeName;
+        this.initializer = initializer;
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public Processor<K, V> get() {
+        return new KStreamSlidingWindowAggregateProcessor();
+    }
+
+    public SlidingWindows windows() {
+        return windows;
+    }
+
+    @Override
+    public void enableSendingOldValues() {
+        sendOldValues = true;
+    }
+
+    private class KStreamSlidingWindowAggregateProcessor extends 
AbstractProcessor<K, V> {
+        private TimestampedWindowStore<K, Agg> windowStore;
+        private TimestampedTupleForwarder<Windowed<K>, Agg> tupleForwarder;
+        private StreamsMetricsImpl metrics;
+        private InternalProcessorContext internalProcessorContext;
+        private Sensor lateRecordDropSensor;
+        private Sensor droppedRecordsSensor;
+        private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(final ProcessorContext context) {
+            super.init(context);
+            internalProcessorContext = (InternalProcessorContext) context;
+            metrics = internalProcessorContext.metrics();
+            final String threadId = Thread.currentThread().getName();
+            lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor(
+                threadId,
+                context.taskId().toString(),
+                internalProcessorContext.currentNode().name(),
+                metrics
+            );
+            droppedRecordsSensor = 
droppedRecordsSensorOrSkippedRecordsSensor(threadId, 
context.taskId().toString(), metrics);
+            windowStore = (TimestampedWindowStore<K, Agg>) 
context.getStateStore(storeName);
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+
+        @Override
+        public void process(final K key, final V value) {
+            if (key == null || value == null) {
+                log.warn(
+                    "Skipping record due to null key or value. value=[{}] 
topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), 
context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+
+            final long timestamp = context().timestamp();
+            //don't process records that don't fall within a full sliding 
window
+            if (timestamp < windows.timeDifferenceMs()) {
+                log.warn(
+                    "Skipping record due to early arrival. value=[{}] 
topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), 
context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+            processInOrder(key, value, timestamp);
+        }
+
+        public void processInOrder(final K key, final V value, final long 
timestamp) {
+
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+            final long closeTime = observedStreamTime - 
windows.gracePeriodMs();
+
+            //store start times of windows we find
+            final HashSet<Long> windowStartTimes = new HashSet<Long>();
+
+            // aggregate that will go in the current record’s left/right 
window (if needed)
+            ValueAndTimestamp<Agg> leftWinAgg = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+
+            //if current record's left/right windows already exist
+            boolean leftWinAlreadyCreated = false;
+            boolean rightWinAlreadyCreated = false;
+
+            // keep the left type window closest to the record
+            Window latestLeftTypeWindow = null;
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> 
iterator = windowStore.fetch(
+                    key,
+                    key,
+                    timestamp - 2 * windows.timeDifferenceMs(),
+                    // to catch the current record's right window, if it 
exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + 
windows.timeDifferenceMs();
+
+                    if (endTime < timestamp) {
+                        leftWinAgg = next.value;
+                        if (isLeftWindow(next)) {
+                            latestLeftTypeWindow = next.key.window();
+                        }
+                    } else if (endTime == timestamp) {
+                        leftWinAlreadyCreated = true;
+                        putAndForward(next.key.window(), next.value, key, 
value, closeTime, timestamp);
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, 
value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            //create right window for previous record
+            if (latestLeftTypeWindow != null) {
+                final long rightWinStart = latestLeftTypeWindow.end() + 1;
+                if (!windowStartTimes.contains(rightWinStart)) {
+                    final TimeWindow window = new TimeWindow(rightWinStart, 
rightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = 
ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, 
timestamp);
+                }
+            }
+
+            //create left window for new record
+            if (!leftWinAlreadyCreated) {
+                final ValueAndTimestamp<Agg> valueAndTime;
+                //there's a right window that the new record could create --> 
new record's left window is not empty
+                if (latestLeftTypeWindow != null) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), 
timestamp);
+                } else {
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), 
timestamp);
+                }
+                final TimeWindow window = new TimeWindow(timestamp - 
windows.timeDifferenceMs(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, 
timestamp);
+            }
+            //create right window for new record
+            if (!rightWinAlreadyCreated && rightWinIsNotEmpty(rightWinAgg, 
timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, 
timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = 
ValueAndTimestamp.make(getValueOrNull(rightWinAgg), 
Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, 
timestamp);
+            }
+        }
+
+        private boolean rightWinIsNotEmpty(final ValueAndTimestamp<Agg> 
rightWinAgg, final long timestamp) {

Review comment:
       nit: can we use the full word `Window` in method names at least

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
##########
@@ -0,0 +1,466 @@
+/*
+ * 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.kstream.internals;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.KeyValueTimestamp;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.TestOutputTopic;
+import org.apache.kafka.streams.TopologyTestDriver;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.Grouped;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.kstream.Windowed;
+import 
org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.TestInputTopic;
+import org.apache.kafka.streams.test.TestRecord;
+import org.apache.kafka.test.MockAggregator;
+import org.apache.kafka.test.MockInitializer;
+import org.apache.kafka.test.MockProcessor;
+import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.hamcrest.Matcher;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Properties;
+
+import static java.time.Duration.ofMillis;
+import static java.util.Arrays.asList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.hasItems;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class KStreamSlidingWindowAggregateTest {
+    private final Properties props = 
StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
+    private final String threadId = Thread.currentThread().getName();
+
+    @Test
+    public void testAggBasic() {
+        final StreamsBuilder builder = new StreamsBuilder();
+        final String topic1 = "topic1";
+
+        final KTable<Windowed<String>, String> table2 = builder
+                .stream(topic1, Consumed.with(Serdes.String(), 
Serdes.String()))
+                .groupByKey(Grouped.with(Serdes.String(), Serdes.String()))
+                
.windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(10), 
ofMillis(50)))
+                .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<>();
+        table2.toStream().process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            final TestInputTopic<String, String> inputTopic1 =
+                    driver.createInputTopic(topic1, new StringSerializer(), 
new StringSerializer());
+            inputTopic1.pipeInput("A", "1", 10L);
+            inputTopic1.pipeInput("A", "1", 20L);
+            inputTopic1.pipeInput("A", "1", 22L);
+            inputTopic1.pipeInput("A", "3", 15L);
+
+            inputTopic1.pipeInput("B", "2", 12L);
+            inputTopic1.pipeInput("B", "2", 13L);
+            inputTopic1.pipeInput("B", "2", 18L);
+            inputTopic1.pipeInput("B", "1", 19L);
+            inputTopic1.pipeInput("B", "2", 25L);
+            inputTopic1.pipeInput("B", "3", 14L);
+
+            inputTopic1.pipeInput("C", "3", 11L);
+            inputTopic1.pipeInput("C", "4", 15L);
+            inputTopic1.pipeInput("C", "1", 16L);
+            inputTopic1.pipeInput("C", "1", 21);
+            inputTopic1.pipeInput("C", "1", 23L);
+
+            inputTopic1.pipeInput("D", "4", 11L);
+            inputTopic1.pipeInput("D", "2", 12L);
+            inputTopic1.pipeInput("D", "3", 29L);
+            inputTopic1.pipeInput("D", "5", 16L);
+        }
+
+        assertEquals(
+                asList(
+                        // A@10 left window created when A@10 processed

Review comment:
       For readability, could we mark the final results for each window? We 
want to make sure all the intermediate results are as expected, but what we 
really care about is what we got in the end. It would just help to have the 
critical output easier to find and get oriented in the tests

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java
##########
@@ -0,0 +1,438 @@
+/*
+ * 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.kstream.internals;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.TopologyTestDriver;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.Grouped;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.kstream.Named;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.kstream.TimeWindowedKStream;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.apache.kafka.streams.state.WindowBytesStoreSupplier;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.TestInputTopic;
+import org.apache.kafka.test.MockAggregator;
+import org.apache.kafka.test.MockInitializer;
+import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.MockReducer;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+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;
+import static org.junit.Assert.assertThrows;
+
+public class SlidingWindowedKStreamImplTest {
+
+    private static final String TOPIC = "input";
+    private final StreamsBuilder builder = new StreamsBuilder();
+    private final Properties props = 
StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
+    private TimeWindowedKStream<String, String> windowedStream;
+
+    @Before
+    public void before() {
+        final KStream<String, String> stream = builder.stream(TOPIC, 
Consumed.with(Serdes.String(), Serdes.String()));
+        windowedStream = stream.
+            groupByKey(Grouped.with(Serdes.String(), Serdes.String()))
+            
.windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(100L), 
ofMillis(1000L)));
+    }
+
+    @Test
+    public void shouldCountSlidingWindows() {
+        final MockProcessorSupplier<Windowed<String>, Long> supplier = new 
MockProcessorSupplier<>();
+        windowedStream
+            .count()
+            .toStream()
+            .process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+        }
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(0L, 100L))),
+            equalTo(ValueAndTimestamp.make(1L, 100L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(101L, 201L))),
+            equalTo(ValueAndTimestamp.make(1L, 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make(2L, 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(400L, 500L))),
+            equalTo(ValueAndTimestamp.make(1L, 500L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(100L, 200L))),
+            equalTo(ValueAndTimestamp.make(2L, 200L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make(1L, 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(151L, 251L))),
+            equalTo(ValueAndTimestamp.make(1L, 200L)));
+    }
+
+    @Test
+    public void shouldReduceSlidingWindows() {
+        final MockProcessorSupplier<Windowed<String>, String> supplier = new 
MockProcessorSupplier<>();
+        windowedStream
+            .reduce(MockReducer.STRING_ADDER)
+            .toStream()
+            .process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+        }
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(0L, 100L))),
+            equalTo(ValueAndTimestamp.make("1", 100L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(101L, 201L))),
+            equalTo(ValueAndTimestamp.make("2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("1+2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(400L, 500L))),
+            equalTo(ValueAndTimestamp.make("3", 500L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(100L, 200L))),
+            equalTo(ValueAndTimestamp.make("10+20", 200L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("20", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(151L, 251L))),
+            equalTo(ValueAndTimestamp.make("10", 200L)));
+    }
+
+    @Test
+    public void shouldAggregateSlidingWindows() {
+        final MockProcessorSupplier<Windowed<String>, String> supplier = new 
MockProcessorSupplier<>();
+        windowedStream
+            .aggregate(
+                MockInitializer.STRING_INIT,
+                MockAggregator.TOSTRING_ADDER,
+                Materialized.with(Serdes.String(), Serdes.String()))
+            .toStream()
+            .process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+        }
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(0L, 100L))),
+            equalTo(ValueAndTimestamp.make("0+1", 100L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(101L, 201L))),
+            equalTo(ValueAndTimestamp.make("0+2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("0+1+2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(400L, 500L))),
+            equalTo(ValueAndTimestamp.make("0+3", 500L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(100L, 200L))),
+            equalTo(ValueAndTimestamp.make("0+10+20", 200L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("0+20", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(151L, 251L))),
+            equalTo(ValueAndTimestamp.make("0+10", 200L)));
+    }
+
+    @Test
+    public void shouldMaterializeCount() {
+        windowedStream.count(
+            Materialized.<String, Long, WindowStore<Bytes, 
byte[]>>as("count-store")
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.Long()));
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+            {
+                final WindowStore<String, Long> windowStore = 
driver.getWindowStore("count-store");
+                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, 100)), 
1L),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), 2L),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), 1L),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), 1L),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), 1L),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), 2L),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), 1L))));
+            }
+            {
+                final WindowStore<String, ValueAndTimestamp<Long>> windowStore 
=
+                    driver.getTimestampedWindowStore("count-store");
+                final List<KeyValue<Windowed<String>, 
ValueAndTimestamp<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, 100)), 
ValueAndTimestamp.make(1L, 100L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), ValueAndTimestamp.make(2L, 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), ValueAndTimestamp.make(1L, 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), ValueAndTimestamp.make(1L, 500L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), ValueAndTimestamp.make(1L, 150L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), ValueAndTimestamp.make(2L, 200L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), ValueAndTimestamp.make(1L, 200L)))));            }
+        }
+    }
+
+    @Test
+    public void shouldMaterializeReduced() {
+        windowedStream.reduce(
+            MockReducer.STRING_ADDER,
+            Materialized.<String, String, WindowStore<Bytes, 
byte[]>>as("reduced")
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.String()));
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+            {
+                final WindowStore<String, String> windowStore = 
driver.getWindowStore("reduced");
+                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, 100)), 
"1"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), "1+2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), "2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), "3"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), "20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), "10+20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), "10"))));
+            }
+            {
+                final WindowStore<String, ValueAndTimestamp<Long>> windowStore 
=
+                    driver.getTimestampedWindowStore("reduced");
+                final List<KeyValue<Windowed<String>, 
ValueAndTimestamp<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, 100)), 
ValueAndTimestamp.make("1", 100L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), ValueAndTimestamp.make("1+2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), ValueAndTimestamp.make("2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), ValueAndTimestamp.make("3", 500L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), ValueAndTimestamp.make("20", 150L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), ValueAndTimestamp.make("10+20", 200L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), ValueAndTimestamp.make("10", 200L)))));
+            }
+        }
+    }
+
+    @Test
+    public void shouldMaterializeAggregated() {
+        windowedStream.aggregate(
+            MockInitializer.STRING_INIT,
+            MockAggregator.TOSTRING_ADDER,
+            Materialized.<String, String, WindowStore<Bytes, 
byte[]>>as("aggregated")
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.String()));
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+            {
+                final WindowStore<String, String> windowStore = 
driver.getWindowStore("aggregated");
+                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, 100)), 
"0+1"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), "0+1+2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), "0+2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), "0+3"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), "0+20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), "0+10+20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), "0+10"))));
+            }
+            {
+                final WindowStore<String, ValueAndTimestamp<Long>> windowStore 
=
+                    driver.getTimestampedWindowStore("aggregated");
+                final List<KeyValue<Windowed<String>, 
ValueAndTimestamp<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, 100)), 
ValueAndTimestamp.make("0+1", 100L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), ValueAndTimestamp.make("0+1+2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), ValueAndTimestamp.make("0+2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), ValueAndTimestamp.make("0+3", 500L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), ValueAndTimestamp.make("0+20", 150L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), ValueAndTimestamp.make("0+10+20", 200L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), ValueAndTimestamp.make("0+10", 200L)))));
+            }
+        }
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnAggregateIfInitializerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(null, MockAggregator.TOSTRING_ADDER));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnAggregateIfAggregatorIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(MockInitializer.STRING_INIT, null));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnReduceIfReducerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(null));
+    }
+
+    @Test
+    public void 
shouldThrowNullPointerOnMaterializedAggregateIfInitializerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(null, MockAggregator.TOSTRING_ADDER, 
Materialized.as("store")));
+    }
+
+    @Test
+    public void 
shouldThrowNullPointerOnMaterializedAggregateIfAggregatorIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(
+            MockInitializer.STRING_INIT,
+            null,
+            Materialized.as("store")));
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void 
shouldThrowNullPointerOnMaterializedAggregateIfMaterializedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(MockInitializer.STRING_INIT, 
MockAggregator.TOSTRING_ADDER, (Materialized) null));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnMaterializedReduceIfReducerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(null, Materialized.as("store")));
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void 
shouldThrowNullPointerOnMaterializedReduceIfMaterializedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(MockReducer.STRING_ADDER, (Materialized) null));
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void shouldThrowNullPointerOnMaterializedReduceIfNamedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(MockReducer.STRING_ADDER, (Named) null));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnCountIfMaterializedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.count((Materialized<String, Long, WindowStore<Bytes, byte[]>>) 
null));
+    }
+
+    @Test
+    public void shouldThrowIllegalArgumentWhenRetentionIsTooSmall() {
+        assertThrows(IllegalArgumentException.class, () -> windowedStream
+            .aggregate(
+                MockInitializer.STRING_INIT,
+                MockAggregator.TOSTRING_ADDER,
+                Materialized
+                    .<String, String, WindowStore<Bytes, 
byte[]>>as("aggregated")
+                    .withKeySerde(Serdes.String())
+                    .withValueSerde(Serdes.String())
+                    .withRetention(ofMillis(1L))
+            )
+        );
+    }
+
+    @Test
+    public void ShouldDropWindowsOutsideOfRetention() {
+        final WindowBytesStoreSupplier storeSupplies = 
Stores.inMemoryWindowStore("aggregated", ofMillis(1200L), ofMillis(100L), 
false);

Review comment:
       ```suggestion
           final WindowBytesStoreSupplier storeSupplier = 
Stores.inMemoryWindowStore("aggregated", ofMillis(1200L), ofMillis(100L), 
false);
   ```

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java
##########
@@ -0,0 +1,438 @@
+/*
+ * 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.kstream.internals;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.TopologyTestDriver;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.Grouped;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.kstream.Named;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.kstream.TimeWindowedKStream;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.apache.kafka.streams.state.WindowBytesStoreSupplier;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.TestInputTopic;
+import org.apache.kafka.test.MockAggregator;
+import org.apache.kafka.test.MockInitializer;
+import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.MockReducer;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+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;
+import static org.junit.Assert.assertThrows;
+
+public class SlidingWindowedKStreamImplTest {
+
+    private static final String TOPIC = "input";
+    private final StreamsBuilder builder = new StreamsBuilder();
+    private final Properties props = 
StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
+    private TimeWindowedKStream<String, String> windowedStream;
+
+    @Before
+    public void before() {
+        final KStream<String, String> stream = builder.stream(TOPIC, 
Consumed.with(Serdes.String(), Serdes.String()));
+        windowedStream = stream.
+            groupByKey(Grouped.with(Serdes.String(), Serdes.String()))
+            
.windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(100L), 
ofMillis(1000L)));
+    }
+
+    @Test
+    public void shouldCountSlidingWindows() {
+        final MockProcessorSupplier<Windowed<String>, Long> supplier = new 
MockProcessorSupplier<>();
+        windowedStream
+            .count()
+            .toStream()
+            .process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+        }
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(0L, 100L))),
+            equalTo(ValueAndTimestamp.make(1L, 100L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(101L, 201L))),
+            equalTo(ValueAndTimestamp.make(1L, 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make(2L, 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(400L, 500L))),
+            equalTo(ValueAndTimestamp.make(1L, 500L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(100L, 200L))),
+            equalTo(ValueAndTimestamp.make(2L, 200L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make(1L, 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(151L, 251L))),
+            equalTo(ValueAndTimestamp.make(1L, 200L)));
+    }
+
+    @Test
+    public void shouldReduceSlidingWindows() {
+        final MockProcessorSupplier<Windowed<String>, String> supplier = new 
MockProcessorSupplier<>();
+        windowedStream
+            .reduce(MockReducer.STRING_ADDER)
+            .toStream()
+            .process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+        }
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(0L, 100L))),
+            equalTo(ValueAndTimestamp.make("1", 100L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(101L, 201L))),
+            equalTo(ValueAndTimestamp.make("2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("1+2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(400L, 500L))),
+            equalTo(ValueAndTimestamp.make("3", 500L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(100L, 200L))),
+            equalTo(ValueAndTimestamp.make("10+20", 200L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("20", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(151L, 251L))),
+            equalTo(ValueAndTimestamp.make("10", 200L)));
+    }
+
+    @Test
+    public void shouldAggregateSlidingWindows() {
+        final MockProcessorSupplier<Windowed<String>, String> supplier = new 
MockProcessorSupplier<>();
+        windowedStream
+            .aggregate(
+                MockInitializer.STRING_INIT,
+                MockAggregator.TOSTRING_ADDER,
+                Materialized.with(Serdes.String(), Serdes.String()))
+            .toStream()
+            .process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+        }
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(0L, 100L))),
+            equalTo(ValueAndTimestamp.make("0+1", 100L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(101L, 201L))),
+            equalTo(ValueAndTimestamp.make("0+2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("0+1+2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(400L, 500L))),
+            equalTo(ValueAndTimestamp.make("0+3", 500L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(100L, 200L))),
+            equalTo(ValueAndTimestamp.make("0+10+20", 200L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("0+20", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(151L, 251L))),
+            equalTo(ValueAndTimestamp.make("0+10", 200L)));
+    }
+
+    @Test
+    public void shouldMaterializeCount() {
+        windowedStream.count(
+            Materialized.<String, Long, WindowStore<Bytes, 
byte[]>>as("count-store")
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.Long()));
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+            {
+                final WindowStore<String, Long> windowStore = 
driver.getWindowStore("count-store");
+                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, 100)), 
1L),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), 2L),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), 1L),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), 1L),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), 1L),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), 2L),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), 1L))));
+            }
+            {
+                final WindowStore<String, ValueAndTimestamp<Long>> windowStore 
=
+                    driver.getTimestampedWindowStore("count-store");
+                final List<KeyValue<Windowed<String>, 
ValueAndTimestamp<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, 100)), 
ValueAndTimestamp.make(1L, 100L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), ValueAndTimestamp.make(2L, 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), ValueAndTimestamp.make(1L, 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), ValueAndTimestamp.make(1L, 500L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), ValueAndTimestamp.make(1L, 150L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), ValueAndTimestamp.make(2L, 200L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), ValueAndTimestamp.make(1L, 200L)))));            }
+        }
+    }
+
+    @Test
+    public void shouldMaterializeReduced() {
+        windowedStream.reduce(
+            MockReducer.STRING_ADDER,
+            Materialized.<String, String, WindowStore<Bytes, 
byte[]>>as("reduced")
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.String()));
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+            {
+                final WindowStore<String, String> windowStore = 
driver.getWindowStore("reduced");
+                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, 100)), 
"1"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), "1+2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), "2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), "3"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), "20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), "10+20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), "10"))));
+            }
+            {
+                final WindowStore<String, ValueAndTimestamp<Long>> windowStore 
=
+                    driver.getTimestampedWindowStore("reduced");
+                final List<KeyValue<Windowed<String>, 
ValueAndTimestamp<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, 100)), 
ValueAndTimestamp.make("1", 100L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), ValueAndTimestamp.make("1+2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), ValueAndTimestamp.make("2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), ValueAndTimestamp.make("3", 500L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), ValueAndTimestamp.make("20", 150L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), ValueAndTimestamp.make("10+20", 200L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), ValueAndTimestamp.make("10", 200L)))));
+            }
+        }
+    }
+
+    @Test
+    public void shouldMaterializeAggregated() {
+        windowedStream.aggregate(
+            MockInitializer.STRING_INIT,
+            MockAggregator.TOSTRING_ADDER,
+            Materialized.<String, String, WindowStore<Bytes, 
byte[]>>as("aggregated")
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.String()));
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+            {
+                final WindowStore<String, String> windowStore = 
driver.getWindowStore("aggregated");
+                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, 100)), 
"0+1"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), "0+1+2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), "0+2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), "0+3"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), "0+20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), "0+10+20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), "0+10"))));
+            }
+            {
+                final WindowStore<String, ValueAndTimestamp<Long>> windowStore 
=
+                    driver.getTimestampedWindowStore("aggregated");
+                final List<KeyValue<Windowed<String>, 
ValueAndTimestamp<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, 100)), 
ValueAndTimestamp.make("0+1", 100L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), ValueAndTimestamp.make("0+1+2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), ValueAndTimestamp.make("0+2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), ValueAndTimestamp.make("0+3", 500L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), ValueAndTimestamp.make("0+20", 150L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), ValueAndTimestamp.make("0+10+20", 200L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), ValueAndTimestamp.make("0+10", 200L)))));
+            }
+        }
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnAggregateIfInitializerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(null, MockAggregator.TOSTRING_ADDER));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnAggregateIfAggregatorIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(MockInitializer.STRING_INIT, null));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnReduceIfReducerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(null));
+    }
+
+    @Test
+    public void 
shouldThrowNullPointerOnMaterializedAggregateIfInitializerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(null, MockAggregator.TOSTRING_ADDER, 
Materialized.as("store")));
+    }
+
+    @Test
+    public void 
shouldThrowNullPointerOnMaterializedAggregateIfAggregatorIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(
+            MockInitializer.STRING_INIT,
+            null,
+            Materialized.as("store")));
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void 
shouldThrowNullPointerOnMaterializedAggregateIfMaterializedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(MockInitializer.STRING_INIT, 
MockAggregator.TOSTRING_ADDER, (Materialized) null));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnMaterializedReduceIfReducerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(null, Materialized.as("store")));
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void 
shouldThrowNullPointerOnMaterializedReduceIfMaterializedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(MockReducer.STRING_ADDER, (Materialized) null));
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void shouldThrowNullPointerOnMaterializedReduceIfNamedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(MockReducer.STRING_ADDER, (Named) null));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnCountIfMaterializedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.count((Materialized<String, Long, WindowStore<Bytes, byte[]>>) 
null));
+    }
+
+    @Test
+    public void shouldThrowIllegalArgumentWhenRetentionIsTooSmall() {
+        assertThrows(IllegalArgumentException.class, () -> windowedStream
+            .aggregate(
+                MockInitializer.STRING_INIT,
+                MockAggregator.TOSTRING_ADDER,
+                Materialized
+                    .<String, String, WindowStore<Bytes, 
byte[]>>as("aggregated")
+                    .withKeySerde(Serdes.String())
+                    .withValueSerde(Serdes.String())
+                    .withRetention(ofMillis(1L))
+            )
+        );
+    }
+
+    @Test
+    public void ShouldDropWindowsOutsideOfRetention() {
+        final WindowBytesStoreSupplier storeSupplies = 
Stores.inMemoryWindowStore("aggregated", ofMillis(1200L), ofMillis(100L), 
false);
+        windowedStream.aggregate(
+            MockInitializer.STRING_INIT,
+            MockAggregator.TOSTRING_ADDER,
+            Materialized.<String, String>as(storeSupplies)
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.String())
+                .withCachingDisabled());
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            final TestInputTopic<String, String> inputTopic =
+                driver.createInputTopic(TOPIC, new StringSerializer(), new 
StringSerializer());
+
+            inputTopic.pipeInput("1", "2", 100L);
+            inputTopic.pipeInput("1", "3", 500L);
+            inputTopic.pipeInput("1", "4", 1000L);
+            inputTopic.pipeInput("1", "5", 2000L);
+
+            {
+                final WindowStore<String, String> windowStore = 
driver.getWindowStore("aggregated");
+                final List<KeyValue<Windowed<String>, String>> data =
+                    StreamsTestUtils.toList(windowStore.fetch("1", "1", 
ofEpochMilli(0), ofEpochMilli(10000L)));
+                assertThat(data, equalTo(Arrays.asList(
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(900, 
1000)), "0+4"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(1900, 
2000)), "0+5"))));

Review comment:
       nit: you could use the version of `fetch` that just takes a single key 
instead of a key range, since there's only one key here

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
##########
@@ -0,0 +1,466 @@
+/*
+ * 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.kstream.internals;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.KeyValueTimestamp;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.TestOutputTopic;
+import org.apache.kafka.streams.TopologyTestDriver;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.Grouped;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.kstream.Windowed;
+import 
org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.TestInputTopic;
+import org.apache.kafka.streams.test.TestRecord;
+import org.apache.kafka.test.MockAggregator;
+import org.apache.kafka.test.MockInitializer;
+import org.apache.kafka.test.MockProcessor;
+import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.hamcrest.Matcher;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Properties;
+
+import static java.time.Duration.ofMillis;
+import static java.util.Arrays.asList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.hasItems;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class KStreamSlidingWindowAggregateTest {
+    private final Properties props = 
StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
+    private final String threadId = Thread.currentThread().getName();
+
+    @Test
+    public void testAggBasic() {
+        final StreamsBuilder builder = new StreamsBuilder();
+        final String topic1 = "topic1";
+
+        final KTable<Windowed<String>, String> table2 = builder
+                .stream(topic1, Consumed.with(Serdes.String(), 
Serdes.String()))
+                .groupByKey(Grouped.with(Serdes.String(), Serdes.String()))
+                
.windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(10), 
ofMillis(50)))
+                .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<>();
+        table2.toStream().process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            final TestInputTopic<String, String> inputTopic1 =
+                    driver.createInputTopic(topic1, new StringSerializer(), 
new StringSerializer());
+            inputTopic1.pipeInput("A", "1", 10L);
+            inputTopic1.pipeInput("A", "1", 20L);
+            inputTopic1.pipeInput("A", "1", 22L);
+            inputTopic1.pipeInput("A", "3", 15L);
+
+            inputTopic1.pipeInput("B", "2", 12L);

Review comment:
       It might be nice to use different values for each record (at least 
within the same key). I don't think there are really any edge cases we should 
worry about when records have the same value so we may as well use a distinct 
one to make the tests a bit easier to read

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
##########
@@ -0,0 +1,466 @@
+/*
+ * 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.kstream.internals;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.KeyValueTimestamp;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.TestOutputTopic;
+import org.apache.kafka.streams.TopologyTestDriver;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.Grouped;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.kstream.Windowed;
+import 
org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.TestInputTopic;
+import org.apache.kafka.streams.test.TestRecord;
+import org.apache.kafka.test.MockAggregator;
+import org.apache.kafka.test.MockInitializer;
+import org.apache.kafka.test.MockProcessor;
+import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.hamcrest.Matcher;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Properties;
+
+import static java.time.Duration.ofMillis;
+import static java.util.Arrays.asList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.hasItems;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class KStreamSlidingWindowAggregateTest {
+    private final Properties props = 
StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
+    private final String threadId = Thread.currentThread().getName();
+
+    @Test
+    public void testAggBasic() {
+        final StreamsBuilder builder = new StreamsBuilder();
+        final String topic1 = "topic1";
+
+        final KTable<Windowed<String>, String> table2 = builder
+                .stream(topic1, Consumed.with(Serdes.String(), 
Serdes.String()))
+                .groupByKey(Grouped.with(Serdes.String(), Serdes.String()))
+                
.windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(10), 
ofMillis(50)))
+                .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<>();
+        table2.toStream().process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            final TestInputTopic<String, String> inputTopic1 =
+                    driver.createInputTopic(topic1, new StringSerializer(), 
new StringSerializer());
+            inputTopic1.pipeInput("A", "1", 10L);
+            inputTopic1.pipeInput("A", "1", 20L);
+            inputTopic1.pipeInput("A", "1", 22L);
+            inputTopic1.pipeInput("A", "3", 15L);
+
+            inputTopic1.pipeInput("B", "2", 12L);
+            inputTopic1.pipeInput("B", "2", 13L);
+            inputTopic1.pipeInput("B", "2", 18L);
+            inputTopic1.pipeInput("B", "1", 19L);
+            inputTopic1.pipeInput("B", "2", 25L);
+            inputTopic1.pipeInput("B", "3", 14L);
+
+            inputTopic1.pipeInput("C", "3", 11L);
+            inputTopic1.pipeInput("C", "4", 15L);
+            inputTopic1.pipeInput("C", "1", 16L);
+            inputTopic1.pipeInput("C", "1", 21);
+            inputTopic1.pipeInput("C", "1", 23L);
+
+            inputTopic1.pipeInput("D", "4", 11L);
+            inputTopic1.pipeInput("D", "2", 12L);
+            inputTopic1.pipeInput("D", "3", 29L);
+            inputTopic1.pipeInput("D", "5", 16L);
+        }
+
+        assertEquals(
+                asList(
+                        // A@10 left window created when A@10 processed
+                        new KeyValueTimestamp<>(new Windowed<>("A", new 
TimeWindow(0, 10)), "0+1", 10),
+                        // A@10 right window created when A@20 processed
+                        new KeyValueTimestamp<>(new Windowed<>("A", new 
TimeWindow(11, 21)), "0+1", 20),
+                        // A@20 left window created when A@20 processed
+                        new KeyValueTimestamp<>(new Windowed<>("A", new 
TimeWindow(10, 20)), "0+1+1", 20),
+                        // A@20 right window created when A@22 processed
+                        new KeyValueTimestamp<>(new Windowed<>("A", new 
TimeWindow(21, 31)), "0+1", 22),
+                        // A@22 left window created when A@22 processed
+                        new KeyValueTimestamp<>(new Windowed<>("A", new 
TimeWindow(12, 22)), "0+1+1", 22),
+                        // A@20 left window updated when A@15 processed
+                        new KeyValueTimestamp<>(new Windowed<>("A", new 
TimeWindow(10, 20)), "0+1+1+3", 20),
+                        // A@10 right window updated when A@15 processed
+                        new KeyValueTimestamp<>(new Windowed<>("A", new 
TimeWindow(11, 21)), "0+1+3", 20),
+                        // A@22 left window updated when A@15 processed
+                        new KeyValueTimestamp<>(new Windowed<>("A", new 
TimeWindow(12, 22)), "0+1+1+3", 22),
+                        // A@15 left window created when A@15 processed
+                        new KeyValueTimestamp<>(new Windowed<>("A", new 
TimeWindow(5, 15)), "0+1+3", 15),
+                        // A@15 right window created when A@15 processed
+                        new KeyValueTimestamp<>(new Windowed<>("A", new 
TimeWindow(16, 26)), "0+1+1", 22),
+
+                        // B@12 left window created when B@12 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(2, 12)), "0+2", 12),
+                        // B@12 right window created when B@13 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(13, 23)), "0+2", 13),
+                        // B@13 left window created when B@13 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(3, 13)), "0+2+2", 13),
+                        // B@12 right window updated when B@18 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(13, 23)), "0+2+2", 18),
+                        // B@13 right window created when B@18 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(14, 24)), "0+2", 18),
+                        // B@18 left window created when B@18 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(8, 18)), "0+2+2+2", 18),
+                        // B@12 right window updated when B@19 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(13, 23)), "0+2+2+1", 19),
+                        // B@13 right window updated when B@19 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(14, 24)), "0+2+1", 19),
+                        // B@18 right window created when B@19 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(19, 29)), "0+1", 19),
+                        // B@19 left window created when B@19 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(9, 19)), "0+2+2+2+1", 19),
+                        // B@18 right window updated when B@25 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(19, 29)), "0+1+2", 25),
+                        // B@19 right window updated when B@25 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(20, 30)), "0+2", 25),
+                        // B@25 left window created when B@25 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(15, 25)), "0+2+1+2", 25),
+                        // B@18 left window updated when B@14 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(8, 18)), "0+2+2+2+3", 18),
+                        // B@19 left window updated when B@14 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(9, 19)), "0+2+2+2+1+3", 19),
+                        // B@12 right window updated when B@14 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(13, 23)), "0+2+2+1+3", 19),
+                        // B@13 right window updated when B@14 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(14, 24)), "0+2+1+3", 19),
+                        // B@14 left window created when B@14 processed
+                        new KeyValueTimestamp<>(new Windowed<>("B", new 
TimeWindow(4, 14)), "0+2+2+3", 14),
+
+                        // C@11 left window created when C@11 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(1, 11)), "0+3", 11),
+                        // C@11 right window created when C@15 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(12, 22)), "0+4", 15),
+                        // C@15 left window created when C@15 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(5, 15)), "0+3+4", 15),
+                        // C@11 right window updated when C@16 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(12, 22)), "0+4+1", 16),
+                        // C@15 right window created when C@16 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(16, 26)), "0+1", 16),
+                        // C@16 left window created when C@16 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(6, 16)), "0+3+4+1", 16),
+                        // C@11 right window updated when C@21 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(12, 22)), "0+4+1+1", 21),
+                        // C@15 right window updated when C@21 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(16, 26)), "0+1+1", 21),
+                        // C@16 right window created when C@21 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(17, 27)), "0+1", 21),
+                        // C@21 left window created when C@21 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(11, 21)), "0+3+4+1+1", 21),
+                        // C@15 right window updated when C@23 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(16, 26)), "0+1+1+1", 23),
+                        // C@16 right window updated when C@23 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(17, 27)), "0+1+1", 23),
+                        // C@21 right window created when C@23 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(22, 32)), "0+1", 23),
+                        // C@23 left window created when C@23 processed
+                        new KeyValueTimestamp<>(new Windowed<>("C", new 
TimeWindow(13, 23)), "0+4+1+1+1", 23),
+
+                        // D@11 left window created when D@11 processed
+                        new KeyValueTimestamp<>(new Windowed<>("D", new 
TimeWindow(1, 11)), "0+4", 11),
+                        // D@11 right window created when D@12 processed
+                        new KeyValueTimestamp<>(new Windowed<>("D", new 
TimeWindow(12, 22)), "0+2", 12),
+                        // D@12 left window created when D@12 processed
+                        new KeyValueTimestamp<>(new Windowed<>("D", new 
TimeWindow(2, 12)), "0+4+2", 12),
+                        // D@29 left window created when D@29 processed
+                        new KeyValueTimestamp<>(new Windowed<>("D", new 
TimeWindow(19, 29)), "0+3", 29),
+                        // D@11 right window updated when D@16 processed
+                        new KeyValueTimestamp<>(new Windowed<>("D", new 
TimeWindow(12, 22)), "0+2+5", 16),
+                        // D@12 right window created when D@16 processed
+                        new KeyValueTimestamp<>(new Windowed<>("D", new 
TimeWindow(13, 23)), "0+5", 16),
+                        // D@16 left window created when D@16 processed
+                        new KeyValueTimestamp<>(new Windowed<>("D", new 
TimeWindow(6, 16)), "0+4+2+5", 16)
+                        ),
+                supplier.theCapturedProcessor().processed
+        );
+    }
+
+    @Test
+    public void testJoin() {

Review comment:
       I still don't exactly understand why we have a join test in the 
`KStreamXXWindowAggregateTest`, but thanks for adding it for sliding windows. 
I'm sure there was a good reason for it, probably long ago

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
##########
@@ -0,0 +1,466 @@
+/*
+ * 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.kstream.internals;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.KeyValueTimestamp;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.TestOutputTopic;
+import org.apache.kafka.streams.TopologyTestDriver;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.Grouped;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.kstream.Windowed;
+import 
org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.TestInputTopic;
+import org.apache.kafka.streams.test.TestRecord;
+import org.apache.kafka.test.MockAggregator;
+import org.apache.kafka.test.MockInitializer;
+import org.apache.kafka.test.MockProcessor;
+import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.hamcrest.Matcher;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Properties;
+
+import static java.time.Duration.ofMillis;
+import static java.util.Arrays.asList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.hasItems;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class KStreamSlidingWindowAggregateTest {
+    private final Properties props = 
StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
+    private final String threadId = Thread.currentThread().getName();
+
+    @Test
+    public void testAggBasic() {

Review comment:
       Sorry that I only just got to looking through this class 😞 . The tests 
here look good but can we add some more test coverage of possible edge cases? I 
know we can't test early records until the next PR, but we should probably have 
more than just the one test of the core functionality.
   
   I know it's really annoying to have to think through all the intermediate 
output, so maybe you can write a helper method that just grabs the final result 
of each window in the output? Then we could have a number of tests that go 
through a larger number of input records without you having to spend all day 
manually processing them yourself 😄 

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java
##########
@@ -0,0 +1,438 @@
+/*
+ * 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.kstream.internals;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.TopologyTestDriver;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.Grouped;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.kstream.Named;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.kstream.TimeWindowedKStream;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.apache.kafka.streams.state.WindowBytesStoreSupplier;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.TestInputTopic;
+import org.apache.kafka.test.MockAggregator;
+import org.apache.kafka.test.MockInitializer;
+import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.MockReducer;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+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;
+import static org.junit.Assert.assertThrows;
+
+public class SlidingWindowedKStreamImplTest {
+
+    private static final String TOPIC = "input";
+    private final StreamsBuilder builder = new StreamsBuilder();
+    private final Properties props = 
StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
+    private TimeWindowedKStream<String, String> windowedStream;
+
+    @Before
+    public void before() {
+        final KStream<String, String> stream = builder.stream(TOPIC, 
Consumed.with(Serdes.String(), Serdes.String()));
+        windowedStream = stream.
+            groupByKey(Grouped.with(Serdes.String(), Serdes.String()))
+            
.windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(100L), 
ofMillis(1000L)));
+    }
+
+    @Test
+    public void shouldCountSlidingWindows() {
+        final MockProcessorSupplier<Windowed<String>, Long> supplier = new 
MockProcessorSupplier<>();
+        windowedStream
+            .count()
+            .toStream()
+            .process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+        }
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(0L, 100L))),
+            equalTo(ValueAndTimestamp.make(1L, 100L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(101L, 201L))),
+            equalTo(ValueAndTimestamp.make(1L, 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make(2L, 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(400L, 500L))),
+            equalTo(ValueAndTimestamp.make(1L, 500L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(100L, 200L))),
+            equalTo(ValueAndTimestamp.make(2L, 200L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make(1L, 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(151L, 251L))),
+            equalTo(ValueAndTimestamp.make(1L, 200L)));
+    }
+
+    @Test
+    public void shouldReduceSlidingWindows() {
+        final MockProcessorSupplier<Windowed<String>, String> supplier = new 
MockProcessorSupplier<>();
+        windowedStream
+            .reduce(MockReducer.STRING_ADDER)
+            .toStream()
+            .process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+        }
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(0L, 100L))),
+            equalTo(ValueAndTimestamp.make("1", 100L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(101L, 201L))),
+            equalTo(ValueAndTimestamp.make("2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("1+2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(400L, 500L))),
+            equalTo(ValueAndTimestamp.make("3", 500L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(100L, 200L))),
+            equalTo(ValueAndTimestamp.make("10+20", 200L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("20", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(151L, 251L))),
+            equalTo(ValueAndTimestamp.make("10", 200L)));
+    }
+
+    @Test
+    public void shouldAggregateSlidingWindows() {
+        final MockProcessorSupplier<Windowed<String>, String> supplier = new 
MockProcessorSupplier<>();
+        windowedStream
+            .aggregate(
+                MockInitializer.STRING_INIT,
+                MockAggregator.TOSTRING_ADDER,
+                Materialized.with(Serdes.String(), Serdes.String()))
+            .toStream()
+            .process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+        }
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(0L, 100L))),
+            equalTo(ValueAndTimestamp.make("0+1", 100L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(101L, 201L))),
+            equalTo(ValueAndTimestamp.make("0+2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("0+1+2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(400L, 500L))),
+            equalTo(ValueAndTimestamp.make("0+3", 500L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(100L, 200L))),
+            equalTo(ValueAndTimestamp.make("0+10+20", 200L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("0+20", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(151L, 251L))),
+            equalTo(ValueAndTimestamp.make("0+10", 200L)));
+    }
+
+    @Test
+    public void shouldMaterializeCount() {
+        windowedStream.count(
+            Materialized.<String, Long, WindowStore<Bytes, 
byte[]>>as("count-store")
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.Long()));
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+            {
+                final WindowStore<String, Long> windowStore = 
driver.getWindowStore("count-store");
+                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, 100)), 
1L),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), 2L),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), 1L),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), 1L),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), 1L),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), 2L),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), 1L))));
+            }
+            {
+                final WindowStore<String, ValueAndTimestamp<Long>> windowStore 
=
+                    driver.getTimestampedWindowStore("count-store");
+                final List<KeyValue<Windowed<String>, 
ValueAndTimestamp<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, 100)), 
ValueAndTimestamp.make(1L, 100L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), ValueAndTimestamp.make(2L, 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), ValueAndTimestamp.make(1L, 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), ValueAndTimestamp.make(1L, 500L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), ValueAndTimestamp.make(1L, 150L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), ValueAndTimestamp.make(2L, 200L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), ValueAndTimestamp.make(1L, 200L)))));            }
+        }
+    }
+
+    @Test
+    public void shouldMaterializeReduced() {
+        windowedStream.reduce(
+            MockReducer.STRING_ADDER,
+            Materialized.<String, String, WindowStore<Bytes, 
byte[]>>as("reduced")
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.String()));
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+            {
+                final WindowStore<String, String> windowStore = 
driver.getWindowStore("reduced");
+                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, 100)), 
"1"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), "1+2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), "2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), "3"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), "20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), "10+20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), "10"))));
+            }
+            {
+                final WindowStore<String, ValueAndTimestamp<Long>> windowStore 
=
+                    driver.getTimestampedWindowStore("reduced");
+                final List<KeyValue<Windowed<String>, 
ValueAndTimestamp<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, 100)), 
ValueAndTimestamp.make("1", 100L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), ValueAndTimestamp.make("1+2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), ValueAndTimestamp.make("2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), ValueAndTimestamp.make("3", 500L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), ValueAndTimestamp.make("20", 150L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), ValueAndTimestamp.make("10+20", 200L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), ValueAndTimestamp.make("10", 200L)))));
+            }
+        }
+    }
+
+    @Test
+    public void shouldMaterializeAggregated() {
+        windowedStream.aggregate(
+            MockInitializer.STRING_INIT,
+            MockAggregator.TOSTRING_ADDER,
+            Materialized.<String, String, WindowStore<Bytes, 
byte[]>>as("aggregated")
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.String()));
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+            {
+                final WindowStore<String, String> windowStore = 
driver.getWindowStore("aggregated");
+                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, 100)), 
"0+1"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), "0+1+2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), "0+2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), "0+3"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), "0+20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), "0+10+20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), "0+10"))));
+            }
+            {
+                final WindowStore<String, ValueAndTimestamp<Long>> windowStore 
=
+                    driver.getTimestampedWindowStore("aggregated");
+                final List<KeyValue<Windowed<String>, 
ValueAndTimestamp<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, 100)), 
ValueAndTimestamp.make("0+1", 100L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), ValueAndTimestamp.make("0+1+2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), ValueAndTimestamp.make("0+2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), ValueAndTimestamp.make("0+3", 500L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), ValueAndTimestamp.make("0+20", 150L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), ValueAndTimestamp.make("0+10+20", 200L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), ValueAndTimestamp.make("0+10", 200L)))));
+            }
+        }
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnAggregateIfInitializerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(null, MockAggregator.TOSTRING_ADDER));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnAggregateIfAggregatorIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(MockInitializer.STRING_INIT, null));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnReduceIfReducerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(null));
+    }
+
+    @Test
+    public void 
shouldThrowNullPointerOnMaterializedAggregateIfInitializerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(null, MockAggregator.TOSTRING_ADDER, 
Materialized.as("store")));
+    }
+
+    @Test
+    public void 
shouldThrowNullPointerOnMaterializedAggregateIfAggregatorIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(
+            MockInitializer.STRING_INIT,
+            null,
+            Materialized.as("store")));
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void 
shouldThrowNullPointerOnMaterializedAggregateIfMaterializedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(MockInitializer.STRING_INIT, 
MockAggregator.TOSTRING_ADDER, (Materialized) null));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnMaterializedReduceIfReducerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(null, Materialized.as("store")));
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void 
shouldThrowNullPointerOnMaterializedReduceIfMaterializedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(MockReducer.STRING_ADDER, (Materialized) null));
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void shouldThrowNullPointerOnMaterializedReduceIfNamedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(MockReducer.STRING_ADDER, (Named) null));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnCountIfMaterializedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.count((Materialized<String, Long, WindowStore<Bytes, byte[]>>) 
null));
+    }
+
+    @Test
+    public void shouldThrowIllegalArgumentWhenRetentionIsTooSmall() {
+        assertThrows(IllegalArgumentException.class, () -> windowedStream
+            .aggregate(
+                MockInitializer.STRING_INIT,
+                MockAggregator.TOSTRING_ADDER,
+                Materialized
+                    .<String, String, WindowStore<Bytes, 
byte[]>>as("aggregated")
+                    .withKeySerde(Serdes.String())
+                    .withValueSerde(Serdes.String())
+                    .withRetention(ofMillis(1L))
+            )
+        );
+    }
+
+    @Test
+    public void ShouldDropWindowsOutsideOfRetention() {

Review comment:
       ```suggestion
       public void shouldDropWindowsOutsideOfRetention() {
   ```

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java
##########
@@ -0,0 +1,438 @@
+/*
+ * 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.kstream.internals;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.TopologyTestDriver;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.Grouped;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.kstream.Named;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.kstream.TimeWindowedKStream;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.apache.kafka.streams.state.WindowBytesStoreSupplier;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.TestInputTopic;
+import org.apache.kafka.test.MockAggregator;
+import org.apache.kafka.test.MockInitializer;
+import org.apache.kafka.test.MockProcessorSupplier;
+import org.apache.kafka.test.MockReducer;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+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;
+import static org.junit.Assert.assertThrows;
+
+public class SlidingWindowedKStreamImplTest {
+
+    private static final String TOPIC = "input";
+    private final StreamsBuilder builder = new StreamsBuilder();
+    private final Properties props = 
StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
+    private TimeWindowedKStream<String, String> windowedStream;
+
+    @Before
+    public void before() {
+        final KStream<String, String> stream = builder.stream(TOPIC, 
Consumed.with(Serdes.String(), Serdes.String()));
+        windowedStream = stream.
+            groupByKey(Grouped.with(Serdes.String(), Serdes.String()))
+            
.windowedBy(SlidingWindows.withTimeDifferenceAndGrace(ofMillis(100L), 
ofMillis(1000L)));
+    }
+
+    @Test
+    public void shouldCountSlidingWindows() {
+        final MockProcessorSupplier<Windowed<String>, Long> supplier = new 
MockProcessorSupplier<>();
+        windowedStream
+            .count()
+            .toStream()
+            .process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+        }
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(0L, 100L))),
+            equalTo(ValueAndTimestamp.make(1L, 100L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(101L, 201L))),
+            equalTo(ValueAndTimestamp.make(1L, 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make(2L, 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(400L, 500L))),
+            equalTo(ValueAndTimestamp.make(1L, 500L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(100L, 200L))),
+            equalTo(ValueAndTimestamp.make(2L, 200L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make(1L, 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(151L, 251L))),
+            equalTo(ValueAndTimestamp.make(1L, 200L)));
+    }
+
+    @Test
+    public void shouldReduceSlidingWindows() {
+        final MockProcessorSupplier<Windowed<String>, String> supplier = new 
MockProcessorSupplier<>();
+        windowedStream
+            .reduce(MockReducer.STRING_ADDER)
+            .toStream()
+            .process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+        }
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(0L, 100L))),
+            equalTo(ValueAndTimestamp.make("1", 100L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(101L, 201L))),
+            equalTo(ValueAndTimestamp.make("2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("1+2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(400L, 500L))),
+            equalTo(ValueAndTimestamp.make("3", 500L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(100L, 200L))),
+            equalTo(ValueAndTimestamp.make("10+20", 200L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("20", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(151L, 251L))),
+            equalTo(ValueAndTimestamp.make("10", 200L)));
+    }
+
+    @Test
+    public void shouldAggregateSlidingWindows() {
+        final MockProcessorSupplier<Windowed<String>, String> supplier = new 
MockProcessorSupplier<>();
+        windowedStream
+            .aggregate(
+                MockInitializer.STRING_INIT,
+                MockAggregator.TOSTRING_ADDER,
+                Materialized.with(Serdes.String(), Serdes.String()))
+            .toStream()
+            .process(supplier);
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+        }
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(0L, 100L))),
+            equalTo(ValueAndTimestamp.make("0+1", 100L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(101L, 201L))),
+            equalTo(ValueAndTimestamp.make("0+2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("0+1+2", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("1", new TimeWindow(400L, 500L))),
+            equalTo(ValueAndTimestamp.make("0+3", 500L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(100L, 200L))),
+            equalTo(ValueAndTimestamp.make("0+10+20", 200L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(50L, 150L))),
+            equalTo(ValueAndTimestamp.make("0+20", 150L)));
+        assertThat(
+            supplier.theCapturedProcessor().lastValueAndTimestampPerKey
+                .get(new Windowed<>("2", new TimeWindow(151L, 251L))),
+            equalTo(ValueAndTimestamp.make("0+10", 200L)));
+    }
+
+    @Test
+    public void shouldMaterializeCount() {
+        windowedStream.count(
+            Materialized.<String, Long, WindowStore<Bytes, 
byte[]>>as("count-store")
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.Long()));
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+            {
+                final WindowStore<String, Long> windowStore = 
driver.getWindowStore("count-store");
+                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, 100)), 
1L),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), 2L),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), 1L),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), 1L),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), 1L),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), 2L),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), 1L))));
+            }
+            {
+                final WindowStore<String, ValueAndTimestamp<Long>> windowStore 
=
+                    driver.getTimestampedWindowStore("count-store");
+                final List<KeyValue<Windowed<String>, 
ValueAndTimestamp<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, 100)), 
ValueAndTimestamp.make(1L, 100L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), ValueAndTimestamp.make(2L, 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), ValueAndTimestamp.make(1L, 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), ValueAndTimestamp.make(1L, 500L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), ValueAndTimestamp.make(1L, 150L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), ValueAndTimestamp.make(2L, 200L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), ValueAndTimestamp.make(1L, 200L)))));            }
+        }
+    }
+
+    @Test
+    public void shouldMaterializeReduced() {
+        windowedStream.reduce(
+            MockReducer.STRING_ADDER,
+            Materialized.<String, String, WindowStore<Bytes, 
byte[]>>as("reduced")
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.String()));
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+            {
+                final WindowStore<String, String> windowStore = 
driver.getWindowStore("reduced");
+                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, 100)), 
"1"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), "1+2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), "2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), "3"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), "20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), "10+20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), "10"))));
+            }
+            {
+                final WindowStore<String, ValueAndTimestamp<Long>> windowStore 
=
+                    driver.getTimestampedWindowStore("reduced");
+                final List<KeyValue<Windowed<String>, 
ValueAndTimestamp<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, 100)), 
ValueAndTimestamp.make("1", 100L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), ValueAndTimestamp.make("1+2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), ValueAndTimestamp.make("2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), ValueAndTimestamp.make("3", 500L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), ValueAndTimestamp.make("20", 150L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), ValueAndTimestamp.make("10+20", 200L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), ValueAndTimestamp.make("10", 200L)))));
+            }
+        }
+    }
+
+    @Test
+    public void shouldMaterializeAggregated() {
+        windowedStream.aggregate(
+            MockInitializer.STRING_INIT,
+            MockAggregator.TOSTRING_ADDER,
+            Materialized.<String, String, WindowStore<Bytes, 
byte[]>>as("aggregated")
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.String()));
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            processData(driver);
+            {
+                final WindowStore<String, String> windowStore = 
driver.getWindowStore("aggregated");
+                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, 100)), 
"0+1"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), "0+1+2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), "0+2"),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), "0+3"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), "0+20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), "0+10+20"),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), "0+10"))));
+            }
+            {
+                final WindowStore<String, ValueAndTimestamp<Long>> windowStore 
=
+                    driver.getTimestampedWindowStore("aggregated");
+                final List<KeyValue<Windowed<String>, 
ValueAndTimestamp<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, 100)), 
ValueAndTimestamp.make("0+1", 100L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 
150)), ValueAndTimestamp.make("0+1+2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(101, 
201)), ValueAndTimestamp.make("0+2", 150L)),
+                    KeyValue.pair(new Windowed<>("1", new TimeWindow(400, 
500)), ValueAndTimestamp.make("0+3", 500L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(50, 
150)), ValueAndTimestamp.make("0+20", 150L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(100, 
200)), ValueAndTimestamp.make("0+10+20", 200L)),
+                    KeyValue.pair(new Windowed<>("2", new TimeWindow(151, 
251)), ValueAndTimestamp.make("0+10", 200L)))));
+            }
+        }
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnAggregateIfInitializerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(null, MockAggregator.TOSTRING_ADDER));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnAggregateIfAggregatorIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(MockInitializer.STRING_INIT, null));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnReduceIfReducerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(null));
+    }
+
+    @Test
+    public void 
shouldThrowNullPointerOnMaterializedAggregateIfInitializerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(null, MockAggregator.TOSTRING_ADDER, 
Materialized.as("store")));
+    }
+
+    @Test
+    public void 
shouldThrowNullPointerOnMaterializedAggregateIfAggregatorIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(
+            MockInitializer.STRING_INIT,
+            null,
+            Materialized.as("store")));
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void 
shouldThrowNullPointerOnMaterializedAggregateIfMaterializedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.aggregate(MockInitializer.STRING_INIT, 
MockAggregator.TOSTRING_ADDER, (Materialized) null));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnMaterializedReduceIfReducerIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(null, Materialized.as("store")));
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void 
shouldThrowNullPointerOnMaterializedReduceIfMaterializedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(MockReducer.STRING_ADDER, (Materialized) null));
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void shouldThrowNullPointerOnMaterializedReduceIfNamedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.reduce(MockReducer.STRING_ADDER, (Named) null));
+    }
+
+    @Test
+    public void shouldThrowNullPointerOnCountIfMaterializedIsNull() {
+        assertThrows(NullPointerException.class, () -> 
windowedStream.count((Materialized<String, Long, WindowStore<Bytes, byte[]>>) 
null));
+    }
+
+    @Test
+    public void shouldThrowIllegalArgumentWhenRetentionIsTooSmall() {
+        assertThrows(IllegalArgumentException.class, () -> windowedStream
+            .aggregate(
+                MockInitializer.STRING_INIT,
+                MockAggregator.TOSTRING_ADDER,
+                Materialized
+                    .<String, String, WindowStore<Bytes, 
byte[]>>as("aggregated")
+                    .withKeySerde(Serdes.String())
+                    .withValueSerde(Serdes.String())
+                    .withRetention(ofMillis(1L))
+            )
+        );
+    }
+
+    @Test
+    public void ShouldDropWindowsOutsideOfRetention() {
+        final WindowBytesStoreSupplier storeSupplies = 
Stores.inMemoryWindowStore("aggregated", ofMillis(1200L), ofMillis(100L), 
false);
+        windowedStream.aggregate(
+            MockInitializer.STRING_INIT,
+            MockAggregator.TOSTRING_ADDER,
+            Materialized.<String, String>as(storeSupplies)
+                .withKeySerde(Serdes.String())
+                .withValueSerde(Serdes.String())
+                .withCachingDisabled());
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(builder.build(), props)) {
+            final TestInputTopic<String, String> inputTopic =
+                driver.createInputTopic(TOPIC, new StringSerializer(), new 
StringSerializer());
+
+            inputTopic.pipeInput("1", "2", 100L);
+            inputTopic.pipeInput("1", "3", 500L);
+            inputTopic.pipeInput("1", "4", 1000L);

Review comment:
       Can we insert one that's like right on the border of the retention 
period? So if the streamtime at the end is 2,000 then the window cut off is 800 
(or start time of 700), and verify that anything starting before 699 is gone 
and everything after that is there.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to