[
https://issues.apache.org/jira/browse/FLINK-26334?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
realdengziqi updated FLINK-26334:
---------------------------------
Flags: Important
Language: JAVA
Release Note: If it's really a bug, we'd love to fix it ourselves,
this is our first time working on an open source project
Description:
source code
{code:java}
//Method to get the window start for a timestamp.
//Params:
//timestamp – epoch millisecond to get the window start.
//offset – The offset which window start would be shifted by.
//windowSize – The size of the generated windows.
//Returns:
//window start
public static long getWindowStartWithOffset(long timestamp, long offset, long
windowSize) {
return timestamp - (timestamp - offset + windowSize) % windowSize;
} {code}
If windowSize is 6 seconds, an element with a timestamp of -7000L should be
assigned to a window with a start time of -12000L. But this code will assign it
to the window whose start time is -6000L.
According to the current calculation method, when the timestamp is less than
0-offset, the start time of the calculated time window will be offset by one
windowsSide unit in the direction of 0.
I had a discussion with a friend and thought it was because the current
calculation logic is rounding towards 0. We should make it round to -∞.
Do you think this is a bug. We would like to submit a pull request on github to
fix it.
Environment: flink version 1.14.3
Summary: when the watermark less than (0-offset) , the
calculation result of TimeWindow.getWindowSTartWithOffset is incorrect (was:
when watermark less than (0-offset) , T)
Remaining Estimate: 16h
Original Estimate: 16h
> when the watermark less than (0-offset) , the calculation result of
> TimeWindow.getWindowSTartWithOffset is incorrect
> ---------------------------------------------------------------------------------------------------------------------
>
> Key: FLINK-26334
> URL: https://issues.apache.org/jira/browse/FLINK-26334
> Project: Flink
> Issue Type: Bug
> Components: API / DataStream
> Affects Versions: 1.14.3
> Environment: flink version 1.14.3
> Reporter: realdengziqi
> Priority: Major
> Original Estimate: 16h
> Remaining Estimate: 16h
>
>
> source code
> {code:java}
> //Method to get the window start for a timestamp.
> //Params:
> //timestamp – epoch millisecond to get the window start.
> //offset – The offset which window start would be shifted by.
> //windowSize – The size of the generated windows.
> //Returns:
> //window start
> public static long getWindowStartWithOffset(long timestamp, long offset, long
> windowSize) {
> return timestamp - (timestamp - offset + windowSize) % windowSize;
> } {code}
> If windowSize is 6 seconds, an element with a timestamp of -7000L should be
> assigned to a window with a start time of -12000L. But this code will assign
> it to the window whose start time is -6000L.
> According to the current calculation method, when the timestamp is less than
> 0-offset, the start time of the calculated time window will be offset by one
> windowsSide unit in the direction of 0.
> I had a discussion with a friend and thought it was because the current
> calculation logic is rounding towards 0. We should make it round to -∞.
> Do you think this is a bug. We would like to submit a pull request on github
> to fix it.
>
--
This message was sent by Atlassian Jira
(v8.20.1#820001)