[ 
https://issues.apache.org/jira/browse/FLINK-26307?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

郭元方 updated FLINK-26307:
------------------------
    Description: 
对于底层实现窗口时,求其开启窗口起始时间的方法有一个问题:
public static long getWindowStartWithOffset(long timestamp, long offset, long 
windowSize)

{         return timestamp - (timestamp - offset + windowSize) % windowSize;    
 }

   例如:窗口大小为10,事件时间分别为1s,7s,12s得到的起始时间,正确。
            => timestamp - (timestamp - offset + windowSize) % windowSize
            => 1 - (1 -0 + 10) % 10  =>  start = 0s
            => 7 - (7 -0 + 10) % 10  =>  start = 0s
            => 12 - (12 - 0 + 10) % 10 => start = 10s

但是当事件时间为 -1s,-7s,-12s,-15s,-17s,-19s时,窗口的起始时间却也都为-10s,

是否由于在java中的取余规则,正数对正数取余的商值向0逼近,而负数对正数取余的商值也向0逼近导致

该算法对事件时间在(Long.MIN_VALUE,-windowsize)的这个范围内的数据不能正确的求到开启窗口的起始时间呢?
           => timestamp - (timestamp - offset + windowSize) % windowSize
            => -1 - (-1 -0 + 10) % 10  =>  start = -10s
            => -7 - (-7 -0 + 10) % 10  =>  start = -10s

            => -12 - (-12 - 0 + 10) % 10 => start = -10s
             => -15 - (-15 - 0 + 10) % 10 => start = -10s
             => -17 - (-17 - 0 + 10) % 10 => start = -10s
              => -19 - (-19 - 0 + 10) % 10 => start = -10s

 

和同学邓子琦的讨论,并没有经历过业务,所以此处事件时间求到了负值,现实世界可能不会这么输入一个负数的事件时间,但逻辑上来说事件时间来说应该可以为负值的吧。

  was:
对于底层实现窗口时,求其起始时间的方法:
public static long getWindowStartWithOffset(long timestamp, long offset, long 
windowSize) {
        return timestamp - (timestamp - offset + windowSize) % windowSize;
    }

   例如:窗口大小为10,事件时间分别为1s,7s,12s得到的起始时间,正确。
            => timestamp - (timestamp - offset + windowSize) % windowSize
            => 1 - (1 -0 + 10) % 10  =>  start = 0s
            => 7 - (7 -0 + 10) % 10  =>  start = 0s
            => 12 - (12 - 0 + 10) % 10 => start = 10s

但是当事件时间为 
-1s,-7s,-12s,-15s,-17s,-19s时,窗口的起始时间却也都为-10s,是否由于在java中的取余规则,正数对正数取余的商值向0逼近,而负数对正数取余的商值也向0逼近
导致该算法对事件时间在(Long.MIN_VALUE,-windowsize)的这个范围内的数据不能正确的求到开启窗口的起始时间呢?
           => timestamp - (timestamp - offset + windowSize) % windowSize
            => -1 - (-1 -0 + 10) % 10  =>  start = -10s
            => -7 - (-7 -0 + 10) % 10  =>  start = -10s

            => -12 - (-12 - 0 + 10) % 10 => start = -10s
             => -15 - (-15 - 0 + 10) % 10 => start = -10s
             => -17 - (-17 - 0 + 10) % 10 => start = -10s
              => -19 - (-19 - 0 + 10) % 10 => start = -10s

 

和同学邓子琦的讨论,并没有经历过业务,所以此处事件时间求到了负值,现实世界来说不一定,但逻辑上来说应该可以求到负值吧。


> 当事件时间在(Long.MIN_VALUE ,-windowsize)时,不能正确的求到开启窗口的起始时间
> -----------------------------------------------------
>
>                 Key: FLINK-26307
>                 URL: https://issues.apache.org/jira/browse/FLINK-26307
>             Project: Flink
>          Issue Type: Improvement
>          Components: API / DataStream
>    Affects Versions: 1.13.6
>            Reporter: 郭元方
>            Priority: Major
>              Labels: 事件时间, 水位线, 窗口
>
> 对于底层实现窗口时,求其开启窗口起始时间的方法有一个问题:
> public static long getWindowStartWithOffset(long timestamp, long offset, long 
> windowSize)
> {         return timestamp - (timestamp - offset + windowSize) % windowSize;  
>    }
>    例如:窗口大小为10,事件时间分别为1s,7s,12s得到的起始时间,正确。
>             => timestamp - (timestamp - offset + windowSize) % windowSize
>             => 1 - (1 -0 + 10) % 10  =>  start = 0s
>             => 7 - (7 -0 + 10) % 10  =>  start = 0s
>             => 12 - (12 - 0 + 10) % 10 => start = 10s
> 但是当事件时间为 -1s,-7s,-12s,-15s,-17s,-19s时,窗口的起始时间却也都为-10s,
> 是否由于在java中的取余规则,正数对正数取余的商值向0逼近,而负数对正数取余的商值也向0逼近导致
> 该算法对事件时间在(Long.MIN_VALUE,-windowsize)的这个范围内的数据不能正确的求到开启窗口的起始时间呢?
>            => timestamp - (timestamp - offset + windowSize) % windowSize
>             => -1 - (-1 -0 + 10) % 10  =>  start = -10s
>             => -7 - (-7 -0 + 10) % 10  =>  start = -10s
>             => -12 - (-12 - 0 + 10) % 10 => start = -10s
>              => -15 - (-15 - 0 + 10) % 10 => start = -10s
>              => -17 - (-17 - 0 + 10) % 10 => start = -10s
>               => -19 - (-19 - 0 + 10) % 10 => start = -10s
>  
> 和同学邓子琦的讨论,并没有经历过业务,所以此处事件时间求到了负值,现实世界可能不会这么输入一个负数的事件时间,但逻辑上来说事件时间来说应该可以为负值的吧。



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to