Re: Flink sql TUMBLE window 不支持offset吗

2020-08-13 文章 nobel一旦
所以总结下,实际不仅仅是 https://issues.apache.org/jira/browse/FLINK-17767
 这一个问题,这个正式我讲到的UTC+8时区下天级别窗口正确划分的解决方案。
但FlinkSQL本身的eventtime的类型问题反而感觉更严重,造成各种误解等。时间戳是最准确的信息,既然采用了日期这种不准确的东西,就应该明确其时区信息。即使时区信息是被隐藏了,那么就正确考虑时区,而不是在将日期翻译回时间戳的时候默认采用了UTC+0的时区转回去,毕竟日期可能是UTC+8时区的表示。

nobel一旦  于2020年8月14日周五 上午11:49写道:

> 窗口周期实际需求是UTC+8时区的(8月)14日0点~14日24点,实际对应UTC+0时区的(8月)*13日*16点~14日16点。
> 1 解释下为什么在FlinkSQL场景下时区设置正确情况下,窗口没划分错误。
> *这个原因比较绕,这也是我想不通的点,作为疑问,希望有人解答(即为什么FlinkSQL使用TIMESTAMP(3)这种日期作为event
> timed定义,以及watermark计算的依据,而不是bigint的UTC+0的时间戳作为eventtime,和datastream
> api保持统一呢)*。
>
> 如上疑问,当然也正是这个疑问导致实际上如果时区设置正确,就不存在窗口划分错误问题了。原因如下:
>
> 记录A,event
> time时间设置为UTC+8的14日0点,这个时间点FlinkSQL估计是按照UTC+0理解的,即将其理解为UTC+0时区的14日0点。
> 记录B,event
> time时间设置为UTC+8的14日24点,这个时间点FlinkSQL估计是按照UTC+0理解的,即将其理解为UTC+0时区的14日24点。
> 如上2句,导致UTC+8时区的0到24点刚好对应到UTC+0时区的0到24点,*因此窗口划分不会出错*。
>
> *但是,这种做法实际很不好,为什么呢?*因为这种做法会导致event
> time实际是错误的,比如你打开Flink的WEB-UI去看watermark,会发现watermark时间戳转成日期(UTC+8)后发现超前了8小时。即现在是14日12点,则watermark大概会是14日20点了已经。
>
>
> 回过头来,那什么做法才最好呢(个人观点)。
> 首先,event time必须按照utc+0的时间戳给出,这意味者要么(1)FlinkSQL使用Timestamp类型作为event
> time这一点要么改变,改为使用时间戳。要么(2)仍然使用Timestamp类型作为event
> time,但是需要用户主动将其设置为UTC+0时间,当然这会导致其他问题,等会讲解决。再或者(3)仍然使用Timestamp类型作为event
> time,同时也使用UTC+8的表示,但为其保留时区信息,即告诉Flink这是UTC+8的时区,然后Flink将其转会时间戳时候可以按照正确的时区信息去转。
>
>
> 关于第(2),(3)种方式下,都会导致窗口划分出现问题(UTC+8情况下的天级别窗口划分问题),但我想说,这个问题本身是应该通过窗口的offset实现的,本身不不应该基于这种错误的设计碰巧去解决,而且还附带一个错误的效果,即WEB-UI的watermark超前8小时。
>
> *希望小伙伴们看看,不管是否有道理,对源码有了解的可以确认下我的说法对不对也。*
>
> Benchao Li  于2020年8月13日周四 上午10:35写道:
>
>> 不管是SQL还是DataStream,底层最终用来划分窗口的时候的时间,都是unix timestamp。
>> SQL里面的Timestamp类型,是可以带着时区的,但是转成unix timestamp表示的时候,时区信息就丢掉了。
>>
>> Zhao,Yi(SEC)  于2020年8月13日周四 上午10:12写道:
>>
>> > 大概懂你意思,sql情况下event time都是Timestamp数据类型,这是一种-MM-dd
>> >
>> HH:mm:ss格式的,从这个角度讲的确是不需要考虑offset的问题(只要周期是1min/1h/1d等类似)。只需要这个Timestamp是我需要的时区格式的日期即可。
>> > 其实我是联系DatastreamAPI的,因为DatastreamAPI中是基于unix timestamp(时间戳)来划分的。
>> >
>> > 在 2020/8/12 下午9:21,“Benchao Li” 写入:
>> >
>> > Hi,
>> >
>> > 目前还没有支持offset,有一个issue[1] 在跟进解决这个问题。
>> > 但是你这个case应该是不需要offset就可以的,现在划分窗口的逻辑是直接按照毫秒级别的时间戳来整除进行计算的。
>> > 所以如果窗口是1min/1h/1d 这种,都是整点的窗口,没有问题的。如果你有这个问题,那应该是你的时间字段有时区问题。
>> > 有一种case是需要offset的,比如一周这种窗口,因为1980-1-1是周四,所以一周的窗口是从周四开始的。
>> >
>> > [1] https://issues.apache.org/jira/browse/FLINK-17767
>> >
>> > Zhao,Yi(SEC)  于2020年8月12日周三 下午8:15写道:
>> >
>> > > 如题,是不是不支持offset呀。在中国,天窗口实际应该是 INTVERVAL ‘1’ DAY + 8小时offset。
>> > > 但是看了文档没发现添加offset的语法。
>> > >
>> > >
>> > > 如果仅仅是时间显示问题还好说,搞个函数转一转,但分窗这个不提供入口没办法。
>> > >
>> >
>> >
>> > --
>> >
>> > Best,
>> > Benchao Li
>> >
>> >
>> >
>>
>> --
>>
>> Best,
>> Benchao Li
>>
>


Re: Flink sql TUMBLE window 不支持offset吗

2020-08-13 文章 nobel一旦
窗口周期实际需求是UTC+8时区的(8月)14日0点~14日24点,实际对应UTC+0时区的(8月)*13日*16点~14日16点。
1 解释下为什么在FlinkSQL场景下时区设置正确情况下,窗口没划分错误。
*这个原因比较绕,这也是我想不通的点,作为疑问,希望有人解答(即为什么FlinkSQL使用TIMESTAMP(3)这种日期作为event
timed定义,以及watermark计算的依据,而不是bigint的UTC+0的时间戳作为eventtime,和datastream
api保持统一呢)*。

如上疑问,当然也正是这个疑问导致实际上如果时区设置正确,就不存在窗口划分错误问题了。原因如下:

记录A,event
time时间设置为UTC+8的14日0点,这个时间点FlinkSQL估计是按照UTC+0理解的,即将其理解为UTC+0时区的14日0点。
记录B,event
time时间设置为UTC+8的14日24点,这个时间点FlinkSQL估计是按照UTC+0理解的,即将其理解为UTC+0时区的14日24点。
如上2句,导致UTC+8时区的0到24点刚好对应到UTC+0时区的0到24点,*因此窗口划分不会出错*。

*但是,这种做法实际很不好,为什么呢?*因为这种做法会导致event
time实际是错误的,比如你打开Flink的WEB-UI去看watermark,会发现watermark时间戳转成日期(UTC+8)后发现超前了8小时。即现在是14日12点,则watermark大概会是14日20点了已经。


回过头来,那什么做法才最好呢(个人观点)。
首先,event time必须按照utc+0的时间戳给出,这意味者要么(1)FlinkSQL使用Timestamp类型作为event
time这一点要么改变,改为使用时间戳。要么(2)仍然使用Timestamp类型作为event
time,但是需要用户主动将其设置为UTC+0时间,当然这会导致其他问题,等会讲解决。再或者(3)仍然使用Timestamp类型作为event
time,同时也使用UTC+8的表示,但为其保留时区信息,即告诉Flink这是UTC+8的时区,然后Flink将其转会时间戳时候可以按照正确的时区信息去转。

关于第(2),(3)种方式下,都会导致窗口划分出现问题(UTC+8情况下的天级别窗口划分问题),但我想说,这个问题本身是应该通过窗口的offset实现的,本身不不应该基于这种错误的设计碰巧去解决,而且还附带一个错误的效果,即WEB-UI的watermark超前8小时。

*希望小伙伴们看看,不管是否有道理,对源码有了解的可以确认下我的说法对不对也。*

Benchao Li  于2020年8月13日周四 上午10:35写道:

> 不管是SQL还是DataStream,底层最终用来划分窗口的时候的时间,都是unix timestamp。
> SQL里面的Timestamp类型,是可以带着时区的,但是转成unix timestamp表示的时候,时区信息就丢掉了。
>
> Zhao,Yi(SEC)  于2020年8月13日周四 上午10:12写道:
>
> > 大概懂你意思,sql情况下event time都是Timestamp数据类型,这是一种-MM-dd
> >
> HH:mm:ss格式的,从这个角度讲的确是不需要考虑offset的问题(只要周期是1min/1h/1d等类似)。只需要这个Timestamp是我需要的时区格式的日期即可。
> > 其实我是联系DatastreamAPI的,因为DatastreamAPI中是基于unix timestamp(时间戳)来划分的。
> >
> > 在 2020/8/12 下午9:21,“Benchao Li” 写入:
> >
> > Hi,
> >
> > 目前还没有支持offset,有一个issue[1] 在跟进解决这个问题。
> > 但是你这个case应该是不需要offset就可以的,现在划分窗口的逻辑是直接按照毫秒级别的时间戳来整除进行计算的。
> > 所以如果窗口是1min/1h/1d 这种,都是整点的窗口,没有问题的。如果你有这个问题,那应该是你的时间字段有时区问题。
> > 有一种case是需要offset的,比如一周这种窗口,因为1980-1-1是周四,所以一周的窗口是从周四开始的。
> >
> > [1] https://issues.apache.org/jira/browse/FLINK-17767
> >
> > Zhao,Yi(SEC)  于2020年8月12日周三 下午8:15写道:
> >
> > > 如题,是不是不支持offset呀。在中国,天窗口实际应该是 INTVERVAL ‘1’ DAY + 8小时offset。
> > > 但是看了文档没发现添加offset的语法。
> > >
> > >
> > > 如果仅仅是时间显示问题还好说,搞个函数转一转,但分窗这个不提供入口没办法。
> > >
> >
> >
> > --
> >
> > Best,
> > Benchao Li
> >
> >
> >
>
> --
>
> Best,
> Benchao Li
>


Re: Flink sql TUMBLE window 不支持offset吗

2020-08-12 文章 Benchao Li
不管是SQL还是DataStream,底层最终用来划分窗口的时候的时间,都是unix timestamp。
SQL里面的Timestamp类型,是可以带着时区的,但是转成unix timestamp表示的时候,时区信息就丢掉了。

Zhao,Yi(SEC)  于2020年8月13日周四 上午10:12写道:

> 大概懂你意思,sql情况下event time都是Timestamp数据类型,这是一种-MM-dd
> HH:mm:ss格式的,从这个角度讲的确是不需要考虑offset的问题(只要周期是1min/1h/1d等类似)。只需要这个Timestamp是我需要的时区格式的日期即可。
> 其实我是联系DatastreamAPI的,因为DatastreamAPI中是基于unix timestamp(时间戳)来划分的。
>
> 在 2020/8/12 下午9:21,“Benchao Li” 写入:
>
> Hi,
>
> 目前还没有支持offset,有一个issue[1] 在跟进解决这个问题。
> 但是你这个case应该是不需要offset就可以的,现在划分窗口的逻辑是直接按照毫秒级别的时间戳来整除进行计算的。
> 所以如果窗口是1min/1h/1d 这种,都是整点的窗口,没有问题的。如果你有这个问题,那应该是你的时间字段有时区问题。
> 有一种case是需要offset的,比如一周这种窗口,因为1980-1-1是周四,所以一周的窗口是从周四开始的。
>
> [1] https://issues.apache.org/jira/browse/FLINK-17767
>
> Zhao,Yi(SEC)  于2020年8月12日周三 下午8:15写道:
>
> > 如题,是不是不支持offset呀。在中国,天窗口实际应该是 INTVERVAL ‘1’ DAY + 8小时offset。
> > 但是看了文档没发现添加offset的语法。
> >
> >
> > 如果仅仅是时间显示问题还好说,搞个函数转一转,但分窗这个不提供入口没办法。
> >
>
>
> --
>
> Best,
> Benchao Li
>
>
>

-- 

Best,
Benchao Li


Re: Flink sql TUMBLE window 不支持offset吗

2020-08-12 文章 Zhao,Yi(SEC)
大概懂你意思,sql情况下event time都是Timestamp数据类型,这是一种-MM-dd 
HH:mm:ss格式的,从这个角度讲的确是不需要考虑offset的问题(只要周期是1min/1h/1d等类似)。只需要这个Timestamp是我需要的时区格式的日期即可。
其实我是联系DatastreamAPI的,因为DatastreamAPI中是基于unix timestamp(时间戳)来划分的。

在 2020/8/12 下午9:21,“Benchao Li” 写入:

Hi,

目前还没有支持offset,有一个issue[1] 在跟进解决这个问题。
但是你这个case应该是不需要offset就可以的,现在划分窗口的逻辑是直接按照毫秒级别的时间戳来整除进行计算的。
所以如果窗口是1min/1h/1d 这种,都是整点的窗口,没有问题的。如果你有这个问题,那应该是你的时间字段有时区问题。
有一种case是需要offset的,比如一周这种窗口,因为1980-1-1是周四,所以一周的窗口是从周四开始的。

[1] https://issues.apache.org/jira/browse/FLINK-17767

Zhao,Yi(SEC)  于2020年8月12日周三 下午8:15写道:

> 如题,是不是不支持offset呀。在中国,天窗口实际应该是 INTVERVAL ‘1’ DAY + 8小时offset。
> 但是看了文档没发现添加offset的语法。
>
>
> 如果仅仅是时间显示问题还好说,搞个函数转一转,但分窗这个不提供入口没办法。
>


-- 

Best,
Benchao Li




Re: Flink sql TUMBLE window 不支持offset吗

2020-08-12 文章 Benchao Li
Hi,

目前还没有支持offset,有一个issue[1] 在跟进解决这个问题。
但是你这个case应该是不需要offset就可以的,现在划分窗口的逻辑是直接按照毫秒级别的时间戳来整除进行计算的。
所以如果窗口是1min/1h/1d 这种,都是整点的窗口,没有问题的。如果你有这个问题,那应该是你的时间字段有时区问题。
有一种case是需要offset的,比如一周这种窗口,因为1980-1-1是周四,所以一周的窗口是从周四开始的。

[1] https://issues.apache.org/jira/browse/FLINK-17767

Zhao,Yi(SEC)  于2020年8月12日周三 下午8:15写道:

> 如题,是不是不支持offset呀。在中国,天窗口实际应该是 INTVERVAL ‘1’ DAY + 8小时offset。
> 但是看了文档没发现添加offset的语法。
>
>
> 如果仅仅是时间显示问题还好说,搞个函数转一转,但分窗这个不提供入口没办法。
>


-- 

Best,
Benchao Li