Hi Gabor,

the Table API / SQL translator generates the code as String and ships the
String as member of the function object.
The code is compiled with Janino when the function is initialized in the
open() method. So we are not shipping classes but compile the code at the
worker.

Not sure if this approach would work for serializers and comparators as
well.

Best, Fabian

2016-04-05 16:47 GMT+02:00 Timo Walther <twal...@apache.org>:

> Hi Gábor,
>
> the code generation in the Table API is in a very early stage and contains
> not much optimization logic so far. Currently we extend the functionality
> to support the most important SQL operations. It will need some time until
> we can further optimize the generated code (e.g. for tracking nulls).
>
> We are using the Janino compiler [1] for in-memory compilation and class
> loading. You can have a look at the code generation here [2].
>
> Regards,
> Timo
>
> [1] http://unkrig.de/w/Janino
> [2]
> https://github.com/apache/flink/blob/master/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala
>
>
>
> On 05.04.2016 16:25, Gábor Horváth wrote:
>
>> Hi!
>>
>> During this summer I plan to introduce runtime code generation in the
>> serializers [1]
>> to improve the performance of Flink.
>>
>> As Stephan Ewen pointed in Table API / SQL effort code generation will
>> also
>> be used to
>> generate functions and data types. In order to share as much code as
>> possible and
>> align the code generation efforts I would like to cooperate with the
>> authors of that project.
>> Who are they, what libraries and approach are they planning to use? Is
>> there a design
>> document or a requirement list somewhere?
>>
>> I know about one document [2], but that did not contain the answers I was
>> looking for.
>>
>> Thanks in advance,
>> Gábor Horváth
>>
>> [1]
>>
>> https://docs.google.com/document/d/1VC8lCeErx9kI5lCMPiUn625PO0rxR-iKlVqtt3hkVnk
>> [2]
>>
>> https://docs.google.com/document/d/1sITIShmJMGegzAjGqFuwiN_iw1urwykKsLiacokxSw0
>>
>>
>

Reply via email to