[GitHub] [kafka] mjsax commented on pull request #10953: MINOR: Default GRACE with Old API should set as 24H minus window-size / inactivity-gap

2021-07-14 Thread GitBox


mjsax commented on pull request #10953:
URL: https://github.com/apache/kafka/pull/10953#issuecomment-880159699


   > but when users upgrade the Streams apps to 3.0 we guarantee retention time 
T+X
   
   I don't agree to this statement. It's a bug in the implementation (that this 
PR fixes) that we keep data for T+X, but the contract to guarantee that data is 
preserved (only) up to T does not change IMHO.
   
   Overall, the PR LGTM. If I read the code of this PR and the previous PRs 
correctly, it should put us back to the 2.8 behavior. We need to include the 
fix for JoinWindows though.


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




[GitHub] [kafka] mjsax commented on pull request #10953: MINOR: Default GRACE with Old API should set as 24H minus window-size / inactivity-gap

2021-07-13 Thread GitBox


mjsax commented on pull request #10953:
URL: https://github.com/apache/kafka/pull/10953#issuecomment-879596929


   > There could be data loss, because locally the windowed state store would 
store records for a longer period of time than in the changelog topic. If a 
Kafka Streams client is restarted with wiped out state it might restore less 
records into the state store than the state store had before the restart. In 
other words, if the Kafka Streams state store were not restarted, it would have 
more records than after the restart. This might happen because records that are 
within the larger retention time of the windowed store (i.e. the segments) 
might be outside the shorter retention time of the changelog topic, hence those 
records might have already been removed from the changelog topic before 
restoration starts.
   
   Yes, but the point is that if the _guaranteed_ retention time is T and T is 
applied to the changelog, the fact that T+X is applied to the state-store does 
not mean we _lose_ the data for this case, because we only guaranteed to hold 
data up to T anyway, and this guarantee is met. In the end, the changelog topic 
is the source of truth, not the state store.


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




[GitHub] [kafka] mjsax commented on pull request #10953: MINOR: Default GRACE with Old API should set as 24H minus window-size / inactivity-gap

2021-07-12 Thread GitBox


mjsax commented on pull request #10953:
URL: https://github.com/apache/kafka/pull/10953#issuecomment-878690334


   > Otherwise, Streams app that run with 2.8 and before might not be 
compatible with Streams 3.0 because the retention time of the changelog topics 
created with older Streams apps will be smaller than the assumed retention time 
for Streams apps in 3.0.
   
   Why would this be fatal? It seems in 3.0 the retention time would 
potentially become _larger_ (if a new changlog topic is created). And for any 
existing application, we won't reconfigure the topic config. While I agree that 
we should keep the behavior in-sync, I don't see how there could be data loss?


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




[GitHub] [kafka] mjsax commented on pull request #10953: MINOR: Default GRACE with Old API should set as 24H minus window-size / inactivity-gap

2021-07-12 Thread GitBox


mjsax commented on pull request #10953:
URL: https://github.com/apache/kafka/pull/10953#issuecomment-878687476


   Correct, sliding-windows were added later and never had a default grace. 
Also, we deprecated `until() / maintainMs()` before and because sliding-windows 
do not inherit from `Windows` is never added `until() / maintainMs()` -- 
thus, I don't think we need to update them. Note that there is an interplay 
between grace period and retention time in the old API.
   
   However, looking into the code of the different classes, it seems only 
`TimeWindow` needs to be updated, because it implements:
   ```
   public long maintainMs() {
   return Math.max(maintainDurationMs, sizeMs + gracePeriodMs());
   }
   ```
   
   In contrast, `JoinWindows` used `Math.max(maintainDurationMs, size());` and 
`SessionWindows` used `Math.max(maintainDurationMs, gapMs);` -- thus, there 
retention-time should not be affected by any change to grace-period?
   
   


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