mjsax commented on code in PR #15189:
URL: https://github.com/apache/kafka/pull/15189#discussion_r1520671475


##########
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamWindowCloseTest.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.IntegerSerializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.TestInputTopic;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.TopologyTestDriver;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.JoinWindows;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.StreamJoined;
+import org.apache.kafka.test.MockApiProcessor;
+import org.apache.kafka.test.MockApiProcessorSupplier;
+import org.apache.kafka.test.MockValueJoiner;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+
+import static java.time.Duration.ofMillis;
+import static org.apache.kafka.test.StreamsTestUtils.getMetricByName;
+
+public class KStreamKStreamWindowCloseTest {
+
+    private static final String LEFT = "left";
+    private static final String RIGHT = "right";
+    private final static Properties PROPS = 
StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
+    private static final Consumed<Integer, String> CONSUMED = 
Consumed.with(Serdes.Integer(), Serdes.String());
+    private static final JoinWindows WINDOW = 
JoinWindows.ofTimeDifferenceAndGrace(ofMillis(10), ofMillis(5));
+
+    static List<Arguments> streams() {
+        return Arrays.asList(
+            innerJoin(),
+            leftJoin(),
+            outerJoin()
+        );
+    }
+
+    private static Arguments innerJoin() {
+        final StreamsBuilder builder = new StreamsBuilder();
+        final KStream<Integer, String> stream = builder.stream(LEFT, 
CONSUMED).join(
+            builder.stream(RIGHT, CONSUMED),
+            MockValueJoiner.TOSTRING_JOINER,
+            WINDOW,
+            StreamJoined.with(Serdes.Integer(), Serdes.String(), 
Serdes.String())
+        );
+        final MockApiProcessorSupplier<Integer, String, Object, Object> 
processorSupplier = new MockApiProcessorSupplier<>();
+        stream.process(processorSupplier);
+        return Arguments.of(builder.build(PROPS), processorSupplier);
+    }
+
+    private static Arguments leftJoin() {
+        final StreamsBuilder builder = new StreamsBuilder();
+        final KStream<Integer, String> stream = builder.stream(LEFT, 
CONSUMED).leftJoin(
+            builder.stream(RIGHT, CONSUMED),
+            MockValueJoiner.TOSTRING_JOINER,
+            WINDOW,
+            StreamJoined.with(Serdes.Integer(), Serdes.String(), 
Serdes.String())
+        );
+        final MockApiProcessorSupplier<Integer, String, Object, Object> 
processorSupplier = new MockApiProcessorSupplier<>();
+        stream.process(processorSupplier);
+        return Arguments.of(builder.build(PROPS), processorSupplier);
+    }
+
+    private static Arguments outerJoin() {
+        final StreamsBuilder builder = new StreamsBuilder();
+        final KStream<Integer, String> stream = builder.stream(LEFT, 
CONSUMED).outerJoin(
+            builder.stream(RIGHT, CONSUMED),
+            MockValueJoiner.TOSTRING_JOINER,
+            WINDOW,
+            StreamJoined.with(Serdes.Integer(), Serdes.String(), 
Serdes.String())
+        );
+        final MockApiProcessorSupplier<Integer, String, Object, Object> 
supplier = new MockApiProcessorSupplier<>();
+        stream.process(supplier);
+        return Arguments.of(builder.build(PROPS), supplier);
+    }
+
+    @ParameterizedTest
+    @MethodSource("streams")
+    public void recordsArrivingPostWindowCloseShouldBeDropped(
+        final Topology topology,
+        final MockApiProcessorSupplier<Integer, String, Void, Void> supplier) {
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(topology, PROPS)) {
+            final TestInputTopic<Integer, String> left =
+                driver.createInputTopic(KStreamKStreamWindowCloseTest.LEFT, 
new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), 
Duration.ZERO);
+            final TestInputTopic<Integer, String> right =
+                driver.createInputTopic(KStreamKStreamWindowCloseTest.RIGHT, 
new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), 
Duration.ZERO);
+            final MockApiProcessor<Integer, String, Void, Void> processor = 
supplier.theCapturedProcessor();
+
+            left.pipeInput(0, "left", 15);
+            right.pipeInput(-1, "bumpTime", 40);
+            assertRecordDropCount(0.0, processor);
+
+            right.pipeInput(0, "closedAt40", 24);
+            assertRecordDropCount(1.0, processor);
+

Review Comment:
   If find this test a little bit hard to follow...
   
   We start to open a window `[5;15]` with the first input record. This window 
would close at `31` so why do we dump to `40`, and test dropping with `ts=24` 
-- both are totally unrelated to the left input record's window.
   
   Also, it would be good to test that we can join out-of-order record 
successfully as long as the window is open, and that we don't drop re-mature 
before we hit the close time, thus I would suggest something like this:
    ```
    // prepare
    - left input at 15 -> open window
    // positive test
    - bump time to 30 (different key) -> window still open
    - right input at 5 -> joins (no need to test with 4, because 4 won't join 
because of window size anyway...)
    - right input at 25 -> joins 
    // negative test
    - bump time to 31 (different key) -> window closes
    - right input at 5 -> dropped (recorded with metric)
    - right input at 25 -> does not join any longer; window closed (for this 
case we don't drop and don't record metric)
    // test sharp lower drop bound
     - right input at 6 -> does not join any longer; window closed (for this 
case we don't drop though and don't record metric)
   // cont. with additional sanity check:
    - left input at 16 -> joins with both right input at 6 and right input at 
25 -- to verify both records did not get dropped
    ```



-- 
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.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

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

Reply via email to