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

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

guozhangwang closed pull request #5474: KAFKA-7080: pass segmentInterval to 
CachingWindowStore
URL: https://github.com/apache/kafka/pull/5474
 
 
   

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/state/internals/WindowStoreBuilder.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java
index 97b4883084c..cd0841a0bf7 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java
@@ -52,7 +52,7 @@ public WindowStoreBuilder(final WindowBytesStoreSupplier 
storeSupplier,
                                         keySerde,
                                         valueSerde,
                                         storeSupplier.windowSize(),
-                                        storeSupplier.segments());
+                                        
Segments.segmentInterval(storeSupplier.retentionPeriod(), 
storeSupplier.segments()));
     }
 
     private WindowStore<Bytes, byte[]> maybeWrapLogging(final 
WindowStore<Bytes, byte[]> inner) {


 

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