回复: flink build-in 的 udf 的源码

2020-05-17 文章 venn
非常感谢大佬,耐心回复

-邮件原件-
发件人: user-zh-return-3567-wxchunjhyy=163@flink.apache.org 
 代表 Benchao Li
发送时间: 2020年5月16日 21:50
收件人: user-zh 
主题: Re: flink build-in 的 udf 的源码

Hi,

Flink内置函数的实现方式跟udf不太一样,很多函数是直接用的代码生成来做的。

下面是以blink planner为例,大概说下流程:
1. FlinkSqlOperatorTable 这个类里面放的是内置函数表,这个表会被calcite parse
SQL的时候用到,直接把这些函数识别为具体的某个函数定义。
2.
然后再代码生成阶段,会识别到这些函数,根据不同的函数定义,生成不同的函数实现调用。这部分你可以直接看下`org.apache.flink.table.planner.codegen.calls`这个package下的代码。
3. 上面第2条说的主要是scalar function的生成方式,agg
function还要特殊一点,这部分可以参考下`org.apache.flink.table.planner.functions.aggfunctions`这个package下的代码。


venn  于2020年5月16日周六 下午3:53写道:

> 各位大佬,请问下,flink 内置的 udf 的源码在什么位置,还有在哪里完成的函数注
> 册? 非常感谢各位大佬回复
>
>
>
> Thanks a lot !
>
>
>
>

-- 

Benchao Li
School of Electronics Engineering and Computer Science, Peking University
Tel:+86-15650713730
Email: libenc...@gmail.com; libenc...@pku.edu.cn


Re: flink build-in 的 udf 的源码

2020-05-16 文章 Benchao Li
Hi tison,

我不是很清楚我是否理解了你的问题,我先尝试解释一下,如果没有解释清楚,你可以再细化一下你的问题。

这个代码生成的过程整体是这样子的:
1. 不管是SQL,还是Table API,在经过前面的各种处理之后,都会转成逻辑执行计划;
2. 逻辑执行计划会经过优化,然后转成物理执行计划
3. 物理执行计划优化后,会翻译到Transformations(也就是DataStream里面的概念)
  3.1. 在翻译过程中,就是代码生成参与的地方。我理解这里最主要的生成其实是表达式,这里面就包含了各种内置函数、UDF等
  3.2.
如果从SQL的处理流程来讲,这里应该是就这一次代码生成的阶段;但是对于里面的各个函数来讲,他们分别属于不同的表达式,所以每个函数对应的call都会都一遍代码生成。
  3.3  然后如果是同一个函数,在多个不同的物理执行算子里面出现,他们是互相不相关的。也就是会被生成多次。


tison  于2020年5月16日周六 下午11:17写道:

> Hi Benchao,
>
> 我想搭车问一下这个代码生成是全局仅一次还是每个 call 都会走一遍流程?或者是其他策略。
>
> Best,
> tison.
>
>
> Benchao Li  于2020年5月16日周六 下午9:50写道:
>
> > Hi,
> >
> > Flink内置函数的实现方式跟udf不太一样,很多函数是直接用的代码生成来做的。
> >
> > 下面是以blink planner为例,大概说下流程:
> > 1. FlinkSqlOperatorTable 这个类里面放的是内置函数表,这个表会被calcite parse
> > SQL的时候用到,直接把这些函数识别为具体的某个函数定义。
> > 2.
> >
> >
> 然后再代码生成阶段,会识别到这些函数,根据不同的函数定义,生成不同的函数实现调用。这部分你可以直接看下`org.apache.flink.table.planner.codegen.calls`这个package下的代码。
> > 3. 上面第2条说的主要是scalar function的生成方式,agg
> >
> >
> function还要特殊一点,这部分可以参考下`org.apache.flink.table.planner.functions.aggfunctions`这个package下的代码。
> >
> >
> > venn  于2020年5月16日周六 下午3:53写道:
> >
> > > 各位大佬,请问下,flink 内置的 udf 的源码在什么位置,还有在哪里完成的函数注
> > > 册? 非常感谢各位大佬回复
> > >
> > >
> > >
> > > Thanks a lot !
> > >
> > >
> > >
> > >
> >
> > --
> >
> > Benchao Li
> > School of Electronics Engineering and Computer Science, Peking University
> > Tel:+86-15650713730
> > Email: libenc...@gmail.com; libenc...@pku.edu.cn
> >
>


-- 

Benchao Li
School of Electronics Engineering and Computer Science, Peking University
Tel:+86-15650713730
Email: libenc...@gmail.com; libenc...@pku.edu.cn


Re: flink build-in 的 udf 的源码

2020-05-16 文章 tison
Hi Benchao,

我想搭车问一下这个代码生成是全局仅一次还是每个 call 都会走一遍流程?或者是其他策略。

Best,
tison.


Benchao Li  于2020年5月16日周六 下午9:50写道:

> Hi,
>
> Flink内置函数的实现方式跟udf不太一样,很多函数是直接用的代码生成来做的。
>
> 下面是以blink planner为例,大概说下流程:
> 1. FlinkSqlOperatorTable 这个类里面放的是内置函数表,这个表会被calcite parse
> SQL的时候用到,直接把这些函数识别为具体的某个函数定义。
> 2.
>
> 然后再代码生成阶段,会识别到这些函数,根据不同的函数定义,生成不同的函数实现调用。这部分你可以直接看下`org.apache.flink.table.planner.codegen.calls`这个package下的代码。
> 3. 上面第2条说的主要是scalar function的生成方式,agg
>
> function还要特殊一点,这部分可以参考下`org.apache.flink.table.planner.functions.aggfunctions`这个package下的代码。
>
>
> venn  于2020年5月16日周六 下午3:53写道:
>
> > 各位大佬,请问下,flink 内置的 udf 的源码在什么位置,还有在哪里完成的函数注
> > 册? 非常感谢各位大佬回复
> >
> >
> >
> > Thanks a lot !
> >
> >
> >
> >
>
> --
>
> Benchao Li
> School of Electronics Engineering and Computer Science, Peking University
> Tel:+86-15650713730
> Email: libenc...@gmail.com; libenc...@pku.edu.cn
>


Re: flink build-in 的 udf 的源码

2020-05-16 文章 Benchao Li
Hi,

Flink内置函数的实现方式跟udf不太一样,很多函数是直接用的代码生成来做的。

下面是以blink planner为例,大概说下流程:
1. FlinkSqlOperatorTable 这个类里面放的是内置函数表,这个表会被calcite parse
SQL的时候用到,直接把这些函数识别为具体的某个函数定义。
2.
然后再代码生成阶段,会识别到这些函数,根据不同的函数定义,生成不同的函数实现调用。这部分你可以直接看下`org.apache.flink.table.planner.codegen.calls`这个package下的代码。
3. 上面第2条说的主要是scalar function的生成方式,agg
function还要特殊一点,这部分可以参考下`org.apache.flink.table.planner.functions.aggfunctions`这个package下的代码。


venn  于2020年5月16日周六 下午3:53写道:

> 各位大佬,请问下,flink 内置的 udf 的源码在什么位置,还有在哪里完成的函数注
> 册? 非常感谢各位大佬回复
>
>
>
> Thanks a lot !
>
>
>
>

-- 

Benchao Li
School of Electronics Engineering and Computer Science, Peking University
Tel:+86-15650713730
Email: libenc...@gmail.com; libenc...@pku.edu.cn


flink build-in 的 udf 的源码

2020-05-16 文章 venn
各位大佬,请问下,flink 内置的 udf 的源码在什么位置,还有在哪里完成的函数注
册? 非常感谢各位大佬回复

 

Thanks a lot !