cadonna commented on a change in pull request #10953: URL: https://github.com/apache/kafka/pull/10953#discussion_r662807687
########## File path: streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java ########## @@ -103,6 +104,11 @@ public void gracePeriodShouldEnforceBoundaries() { } } + @Test + public void oldAPIShouldSetDefaultGracePeriod() { + assertEquals(DEPRECATED_OLD_24_HR_GRACE_PERIOD - 3L, TimeWindows.of(ofMillis(3L)).gracePeriodMs());; + } Review comment: I think, we need additional tests: ```suggestion @Test public void deprecatedDefaultGracePeriodShouldBeOneDay() { assertThat(DEPRECATED_OLD_24_HR_GRACE_PERIOD, is(Duration.ofDays(1).toMillis())); } @Test public void oldAPIShouldSetGracePeriodToDefaultGracePeriodMinusWindowSize() { final Duration windowSize = Duration.ofMillis(3L); assertThat(TimeWindows.of(windowSize).gracePeriodMs(), is(DEPRECATED_OLD_24_HR_GRACE_PERIOD - windowSize.toMillis())); } @Test public void oldAPIShouldSetGracePeriodToZeroWhenWindowSizeIsDefaultGracePeriod()() { final Duration windowSize = Duration.ofMillis(DEPRECATED_OLD_24_HR_GRACE_PERIOD); assertThat(TimeWindows.of(windowSize).gracePeriodMs(), is(0L)); } @Test public void oldAPIShouldSetGracePeriodToZeroWhenWindowSizeGreaterDefaultGracePeriod()() { final Duration windowSize = Duration.ofMillis(DEPRECATED_OLD_24_HR_GRACE_PERIOD + 1); assertThat(TimeWindows.of(windowSize).gracePeriodMs(), is(0L)); } ``` The first test makes the deprecated default grace period robust against refactorings. The other three tests correspond to the three important cases, namely: - window size less than default grace period - window size equal to default grace period - window size greater than default grace period You might need some imports for this code to work. Similar needs to be done for `SessionWindows`. ########## File path: streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java ########## @@ -146,7 +146,7 @@ public static TimeWindows of(final Duration size) throws IllegalArgumentExceptio final String msgPrefix = prepareMillisCheckFailMsgPrefix(size, "size"); final long sizeMs = validateMillisecondDuration(size, msgPrefix); - return new TimeWindows(sizeMs, sizeMs, DEPRECATED_OLD_24_HR_GRACE_PERIOD); + return new TimeWindows(sizeMs, sizeMs, Math.max(DEPRECATED_OLD_24_HR_GRACE_PERIOD - sizeMs, 0)); Review comment: The javadoc `@return` needs to be adapted. -- 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