[ 
https://issues.apache.org/jira/browse/KAFKA-7080?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16653964#comment-16653964
 ] 

ASF GitHub Bot commented on KAFKA-7080:
---------------------------------------

ewencp closed pull request #5806: KAFKA-7080 and KAFKA-7222: Cleanup 
overlapping KIP changes Part 2
URL: https://github.com/apache/kafka/pull/5806
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java
index 8ba02bfefaf..ecfe1554815 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImpl.java
@@ -193,9 +193,9 @@
                 supplier = Stores.persistentWindowStore(
                     materialized.storeName(),
                     windows.maintainMs(),
+                    windows.segments,
                     windows.size(),
-                    false,
-                    windows.segmentInterval()
+                    false
                 );
             }
         }
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java 
b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java
index 8709031a468..49bc56cade8 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java
@@ -40,19 +40,6 @@ public long gracePeriodMs() {
         }
     }
 
-    @SuppressWarnings("deprecation") // specifically testing deprecated APIs
-    @Test
-    public void shouldSetNumberOfSegments() {
-        final int anySegmentSizeLargerThanOne = 5;
-        final TestWindows testWindow = new TestWindows();
-        final long maintainMs = testWindow.maintainMs();
-
-        assertEquals(
-            maintainMs / (anySegmentSizeLargerThanOne - 1),
-            testWindow.segments(anySegmentSizeLargerThanOne).segmentInterval()
-        );
-    }
-
     @SuppressWarnings("deprecation") // specifically testing deprecated APIs
     @Test
     public void shouldSetWindowRetentionTime() {
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java 
b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
index 4008689d3ef..4fe3c07f1cc 100644
--- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
+++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
@@ -473,10 +473,9 @@ private void processStreamWithWindowStore(final String 
topic) {
         final StoreBuilder<WindowStore<Integer, byte[]>> storeBuilder = 
Stores.windowStoreBuilder(
             Stores.persistentWindowStore(
                 "store",
-                AGGREGATE_WINDOW_SIZE * 3,
-                AGGREGATE_WINDOW_SIZE,
-                false,
-                60_000L
+                ofMillis(AGGREGATE_WINDOW_SIZE * 3),
+                ofMillis(AGGREGATE_WINDOW_SIZE),
+                false
             ),
             INTEGER_SERDE,
             BYTE_SERDE
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java 
b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java
index b62364a4a2f..1d4a849a19d 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java
@@ -24,6 +24,7 @@
 import org.apache.kafka.streams.state.internals.RocksDBWindowStore;
 import org.junit.Test;
 
+import static java.time.Duration.ZERO;
 import static java.time.Duration.ofMillis;
 import static org.hamcrest.CoreMatchers.nullValue;
 import static org.hamcrest.MatcherAssert.assertThat;
@@ -55,12 +56,12 @@ public void shouldThrowIfILruMapStoreCapacityIsNegative() {
 
     @Test(expected = NullPointerException.class)
     public void shouldThrowIfIPersistentWindowStoreStoreNameIsNull() {
-        Stores.persistentWindowStore(null, 0L, 0L, false, 0L);
+        Stores.persistentWindowStore(null, ZERO, ZERO, false);
     }
 
     @Test(expected = IllegalArgumentException.class)
     public void shouldThrowIfIPersistentWindowStoreRetentionPeriodIsNegative() 
{
-        Stores.persistentWindowStore("anyName", -1L, 0L, false, 0L);
+        Stores.persistentWindowStore("anyName", ofMillis(-1L), ZERO, false);
     }
 
     @Deprecated
@@ -74,11 +75,6 @@ public void 
shouldThrowIfIPersistentWindowStoreIfWindowSizeIsNegative() {
         Stores.persistentWindowStore("anyName", ofMillis(0L), ofMillis(-1L), 
false);
     }
 
-    @Test(expected = IllegalArgumentException.class)
-    public void 
shouldThrowIfIPersistentWindowStoreIfSegmentIntervalIsTooSmall() {
-        Stores.persistentWindowStore("anyName", 1L, 1L, false, -1L);
-    }
-
     @Test(expected = NullPointerException.class)
     public void shouldThrowIfIPersistentSessionStoreStoreNameIsNull() {
         Stores.persistentSessionStore(null, ofMillis(0));
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
index cd0f49a7372..c41b0942301 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
@@ -54,6 +54,7 @@
 import java.util.Map;
 import java.util.Set;
 
+import static java.time.Duration.ofMillis;
 import static java.time.Instant.ofEpochMilli;
 import static java.util.Objects.requireNonNull;
 import static org.hamcrest.CoreMatchers.equalTo;
@@ -70,7 +71,7 @@
 
     private final int numSegments = 3;
     private final long windowSize = 3L;
-    private final long segmentInterval = 600L;
+    private final long segmentInterval = 60_000L;
     private final long retentionPeriod = segmentInterval * (numSegments - 1);
     private final String windowName = "window";
     private final Segments segments = new Segments(windowName, 
retentionPeriod, segmentInterval);
@@ -108,7 +109,7 @@
 
     private WindowStore<Integer, String> createWindowStore(final 
ProcessorContext context, final boolean retainDuplicates) {
         final WindowStore<Integer, String> store = Stores.windowStoreBuilder(
-            Stores.persistentWindowStore(windowName, retentionPeriod, 
windowSize, retainDuplicates, segmentInterval),
+            Stores.persistentWindowStore(windowName, 
ofMillis(retentionPeriod), ofMillis(windowSize), retainDuplicates),
             Serdes.Integer(),
             Serdes.String()).build();
 
@@ -771,7 +772,7 @@ public void shouldFetchAndIterateOverExactKeys() {
         final long retentionPeriod = 0x7a00000000000000L;
 
         final WindowStore<String, String> windowStore = 
Stores.windowStoreBuilder(
-            Stores.persistentWindowStore(windowName, retentionPeriod, 
windowSize, true),
+            Stores.persistentWindowStore(windowName, 
ofMillis(retentionPeriod), ofMillis(windowSize), true),
             Serdes.String(),
             Serdes.String()).build();
 
@@ -848,7 +849,7 @@ public void shouldNoNullPointerWhenSerdeDoesNotHandleNull() 
{
     @Test
     public void shouldFetchAndIterateOverExactBinaryKeys() {
         final WindowStore<Bytes, String> windowStore = 
Stores.windowStoreBuilder(
-            Stores.persistentWindowStore(windowName, 60_000L, 60_000L, true),
+            Stores.persistentWindowStore(windowName, ofMillis(60_000L), 
ofMillis(60_000L), true),
             Serdes.Bytes(),
             Serdes.String()).build();
 


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> WindowStoreBuilder incorrectly initializes CachingWindowStore
> -------------------------------------------------------------
>
>                 Key: KAFKA-7080
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7080
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 1.0.0, 1.0.1, 1.1.0, 2.0.0
>            Reporter: John Roesler
>            Assignee: John Roesler
>            Priority: Major
>             Fix For: 2.1.0
>
>
> When caching is enabled on the WindowStoreBuilder, it creates a 
> CachingWindowStore. However, it incorrectly passes storeSupplier.segments() 
> (the number of segments) to the segmentInterval argument.
>  
> The impact is low, since any valid number of segments is also a valid segment 
> size, but it likely results in much smaller segments than intended. For 
> example, the segments may be sized 3ms instead of 60,000ms.
>  
> Ideally the WindowBytesStoreSupplier interface would allow suppliers to 
> advertise their segment size instead of segment count. I plan to create a KIP 
> to propose this.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to