guozhangwang commented on a change in pull request #10378: URL: https://github.com/apache/kafka/pull/10378#discussion_r599306972
########## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java ########## @@ -185,8 +185,8 @@ private void runTest(final String optimizationConfig, final int expectedNumberRe .filter((k, v) -> k.equals("A"), Named.as("join-filter")) .join(countStream, (v1, v2) -> v1 + ":" + v2.toString(), JoinWindows.of(ofMillis(5000)), - StreamJoined.<String, String, Long>with(Stores.inMemoryWindowStore("join-store", ofDays(1), ofMillis(10000), true), - Stores.inMemoryWindowStore("other-join-store", ofDays(1), ofMillis(10000), true)) + StreamJoined.<String, String, Long>with(Stores.inMemoryWindowStore("join-store", ofDays(1).plus(ofMillis(10000)), ofMillis(10000), true), Review comment: This is the by-product of this change as we default grace to 24h, and retention to grace + window-size. We can consider whether we keep the default grace at 24h cc @ableegoldman ########## File path: streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java ########## @@ -70,57 +69,24 @@ */ public final class JoinWindows extends Windows<Window> { - private final long maintainDurationMs; - /** Maximum time difference for tuples that are before the join tuple. */ public final long beforeMs; /** Maximum time difference for tuples that are after the join tuple. */ public final long afterMs; + // By default grace period is 24 hours, Review comment: I chose to use 24h, not 24h - size as the default value. Since it is simpler to set at construction time and I think the difference is insignificant. Ditto for other classes. ########## File path: streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java ########## @@ -317,18 +318,21 @@ public void shouldLogAndMeterWhenSkippingExpiredWindowWithBuiltInMetricsVersion0 shouldLogAndMeterWhenSkippingExpiredWindow(StreamsConfig.METRICS_0100_TO_24); } - @Deprecated // testing deprecated functionality (behavior of until) private void shouldLogAndMeterWhenSkippingExpiredWindow(final String builtInMetricsVersion) { final StreamsBuilder builder = new StreamsBuilder(); final String topic = "topic"; final KStream<String, String> stream1 = builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String())); stream1.groupByKey(Grouped.with(Serdes.String(), Serdes.String())) - .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5)).until(100)) + .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5))) .aggregate( () -> "", MockAggregator.toStringInstance("+"), - Materialized.<String, String, WindowStore<Bytes, byte[]>>as("topic1-Canonicalized").withValueSerde(Serdes.String()).withCachingDisabled().withLoggingDisabled() + Materialized.<String, String, WindowStore<Bytes, byte[]>>as("topic1-Canonicalized") Review comment: I still maintained this test while moving the retention from window to materialized. ########## File path: streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java ########## @@ -42,66 +38,10 @@ */ public abstract class Windows<W extends Window> { - private long maintainDurationMs = DEFAULT_RETENTION_MS; - @Deprecated public int segments = 3; + protected static final long DEFAULT_GRACE_PERIOD_MS = 24 * 60 * 60 * 1000L; // one day Review comment: This class become a pure abstract one, and I'm moving the default value (also renamed it) from WindowsDefault and remove the other class. ########## File path: streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java ########## @@ -1,63 +0,0 @@ -/* - * 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; Review comment: This class is no more needed since Windows is a pure abstract class now. -- 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