You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user-zh@flink.apache.org by op <52...@qq.com.INVALID> on 2021/06/23 09:03:12 UTC
Processing-time temporal join is not supported yet
hi,我用两个kaka表做temporal join时报这个错误:
org.apache.flink.table.api.TableException: Processing-time temporal join is not supported yet.
sql如下,我看官网上好像支持这种呢,不知道什么情况
create view visioned_table as
select
user_id,
event
from
(select
user_id,
event,
row_number() over(partition by user_id order by event_time desc) as rn
from kafka_table1
)ta where rn=1;
select
t1.*,t2.*
from mvp_rtdwd_event_app_quit t1
join visioned_table FOR SYSTEM_TIME AS OF t1.proc_time AS t2
on t1.user_id=t2.user_id
where t1.user_id is not null
回复: Processing-time temporal join is not supported yet
Posted by "jiangshan0204@163.com" <ji...@163.com>.
当前版本,基于处理时间的时态 Join 中, 如果右侧表不是可以直接查询外部系统的表而是普通的数据流,时态表函数 Join 和 时态表 Join 的语义都有问题,时态表函数 Join 仍然允许使用,但是时态表 Join 禁用了该功能。 语义问题的原因是 join 算子没办法知道右侧时态表(构建侧)的完整快照是否到齐,这可能导致左侧的流在启动时关联不到用户期待的数据, 在生产环境中可能误导用户。
可以考虑使用基于事件时间的语义,并为左右表设置合适的watermark;或者右侧流落地成维表先
jiangshan0204@163.com
发件人: op
发送时间: 2021-06-23 17:03
收件人: user-zh
主题: Processing-time temporal join is not supported yet
hi,我用两个kaka表做temporal join时报这个错误:
org.apache.flink.table.api.TableException: Processing-time temporal join is not supported yet.
sql如下,我看官网上好像支持这种呢,不知道什么情况
create view visioned_table as
select
user_id,
event
from
(select
user_id,
event,
row_number() over(partition by user_id order by event_time desc) as rn
from kafka_table1
)ta where rn=1;
select
t1.*,t2.*
from mvp_rtdwd_event_app_quit t1
join visioned_table FOR SYSTEM_TIME AS OF t1.proc_time AS t2
on t1.user_id=t2.user_id
where t1.user_id is not null
回复: Processing-time temporal join is not supported yet
Posted by op <52...@qq.com.INVALID>.
您好,请问靠watermark清理过期数据是指,维表中eventtime小于watermark的数据会被清理掉是吧?就是说不能保证每个key至少保留一份数据
------------------ 原始邮件 ------------------
发件人: "user-zh" <xbjtdcq@gmail.com>;
发送时间: 2021年6月23日(星期三) 晚上7:38
收件人: "user-zh"<user-zh@flink.apache.org>;
主题: Re: Processing-time temporal join is not supported yet
会保留维表状态的,靠watermark清理过期数据。
祝好
Leonard
> 在 2021年6月23日,19:20,op <520075694@qq.com.INVALID> 写道:
>
> 谢谢,Event time&nbsp; temporal join 会保存temporal每个的key的最新状态吗,官网文档说跟两边watermark有关,每太看明白。。。
>
>
>
>
> ------------------&nbsp;原始邮件&nbsp;------------------
> 发件人: "user-zh" <xbjtdcq@gmail.com&gt;;
> 发送时间:&nbsp;2021年6月23日(星期三) 下午5:40
> 收件人:&nbsp;"user-zh"<user-zh@flink.apache.org&gt;;
>
> 主题:&nbsp;Re: Processing-time temporal join is not supported yet
>
>
>
> Hi,
>
> Flink SQL 目前支持 Event time&nbsp; temporal join 任意表/视图,还不支持 Processing-time temporal&nbsp; join 任意表/视图(支持Processing-time&nbsp; join 实现了LookupTableSource的表)。
>
> Processing-time temporal&nbsp; join 任意表目前不支持的原因主要是语义问题,具体来说:&nbsp; 在Processing time关联时,Flink SQL 层面还没比较好的机制保证维表加载完后再关联。比如如用来做维表流的kafka中有 1000万 条数据,但目前没有办法实现将这 1000万条先记录全部加载完后主流过来的数据再去关联,在作业启动阶段,主流的数据预期能够关联上的数据可能因为维表还未加载完成而关联不上。
>
> 可以参考下 https://issues.apache.org/jira/browse/FLINK-19830 <https://issues.apache.org/jira/browse/FLINK-19830&gt;
>
> 祝好
> Leonard
>
>
>
> &gt; 在 2021年6月23日,17:03,op <520075694@qq.com.INVALID&gt; 写道:
> &gt;
> &gt;&nbsp; Processing-time temporal join is not supported yet.
Re: Processing-time temporal join is not supported yet
Posted by Leonard Xu <xb...@gmail.com>.
会保留维表状态的,靠watermark清理过期数据。
祝好
Leonard
> 在 2021年6月23日,19:20,op <52...@qq.com.INVALID> 写道:
>
> 谢谢,Event time temporal join 会保存temporal每个的key的最新状态吗,官网文档说跟两边watermark有关,每太看明白。。。
>
>
>
>
> ------------------ 原始邮件 ------------------
> 发件人: "user-zh" <xbjtdcq@gmail.com>;
> 发送时间: 2021年6月23日(星期三) 下午5:40
> 收件人: "user-zh"<user-zh@flink.apache.org>;
>
> 主题: Re: Processing-time temporal join is not supported yet
>
>
>
> Hi,
>
> Flink SQL 目前支持 Event time temporal join 任意表/视图,还不支持 Processing-time temporal join 任意表/视图(支持Processing-time join 实现了LookupTableSource的表)。
>
> Processing-time temporal join 任意表目前不支持的原因主要是语义问题,具体来说: 在Processing time关联时,Flink SQL 层面还没比较好的机制保证维表加载完后再关联。比如如用来做维表流的kafka中有 1000万 条数据,但目前没有办法实现将这 1000万条先记录全部加载完后主流过来的数据再去关联,在作业启动阶段,主流的数据预期能够关联上的数据可能因为维表还未加载完成而关联不上。
>
> 可以参考下 https://issues.apache.org/jira/browse/FLINK-19830 <https://issues.apache.org/jira/browse/FLINK-19830>
>
> 祝好
> Leonard
>
>
>
> > 在 2021年6月23日,17:03,op <520075694@qq.com.INVALID> 写道:
> >
> > Processing-time temporal join is not supported yet.
回复: Processing-time temporal join is not supported yet
Posted by op <52...@qq.com.INVALID>.
谢谢,Event time temporal join 会保存temporal每个的key的最新状态吗,官网文档说跟两边watermark有关,每太看明白。。。
------------------ 原始邮件 ------------------
发件人: "user-zh" <xbjtdcq@gmail.com>;
发送时间: 2021年6月23日(星期三) 下午5:40
收件人: "user-zh"<user-zh@flink.apache.org>;
主题: Re: Processing-time temporal join is not supported yet
Hi,
Flink SQL 目前支持 Event time temporal join 任意表/视图,还不支持 Processing-time temporal join 任意表/视图(支持Processing-time join 实现了LookupTableSource的表)。
Processing-time temporal join 任意表目前不支持的原因主要是语义问题,具体来说: 在Processing time关联时,Flink SQL 层面还没比较好的机制保证维表加载完后再关联。比如如用来做维表流的kafka中有 1000万 条数据,但目前没有办法实现将这 1000万条先记录全部加载完后主流过来的数据再去关联,在作业启动阶段,主流的数据预期能够关联上的数据可能因为维表还未加载完成而关联不上。
可以参考下 https://issues.apache.org/jira/browse/FLINK-19830 <https://issues.apache.org/jira/browse/FLINK-19830>
祝好
Leonard
> 在 2021年6月23日,17:03,op <520075694@qq.com.INVALID> 写道:
>
> Processing-time temporal join is not supported yet.
Re: Processing-time temporal join is not supported yet
Posted by Leonard Xu <xb...@gmail.com>.
Hi,
Flink SQL 目前支持 Event time temporal join 任意表/视图,还不支持 Processing-time temporal join 任意表/视图(支持Processing-time join 实现了LookupTableSource的表)。
Processing-time temporal join 任意表目前不支持的原因主要是语义问题,具体来说: 在Processing time关联时,Flink SQL 层面还没比较好的机制保证维表加载完后再关联。比如如用来做维表流的kafka中有 1000万 条数据,但目前没有办法实现将这 1000万条先记录全部加载完后主流过来的数据再去关联,在作业启动阶段,主流的数据预期能够关联上的数据可能因为维表还未加载完成而关联不上。
可以参考下 https://issues.apache.org/jira/browse/FLINK-19830 <https://issues.apache.org/jira/browse/FLINK-19830>
祝好
Leonard
> 在 2021年6月23日,17:03,op <52...@qq.com.INVALID> 写道:
>
> Processing-time temporal join is not supported yet.