[ https://issues.apache.org/jira/browse/FLINK-5792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15869395#comment-15869395 ]
Fabian Hueske commented on FLINK-5792: -------------------------------------- I agree, it makes sense to ship a serialized UDF object. Regular Flink function are also serialized and distributed to the workers. [~sunjincheng121] can you explain a bit how you want to distribute the UDF? I see two options: 1. make the UDF a member of wrapping function. It might be a bit tricky to pass the reference into the code-gen'd function. 2. add a final byte[] field into the code-gen'd function that holds the serialized UDF object and deserialize during initialization. This will blow up the code-gen'd string but might work well. Best, Fabian > Improve “UDF/UDTF" to support constructor with parameter. > --------------------------------------------------------- > > Key: FLINK-5792 > URL: https://issues.apache.org/jira/browse/FLINK-5792 > Project: Flink > Issue Type: Improvement > Components: Table API & SQL > Reporter: sunjincheng > Assignee: sunjincheng > > Currently UDF/UDTF in the codegen phase using a nonparametric constructor to > create the instance, causing the user can not include the state value in the > UDF/UDTF. The UDF/UDTF's codegen phase can use a serialized mechanism so that > the UDTF can contain state values. > 1. UserDefinedFunction inherits Serializable. > 2. Modify CodeGenerator about UDF/UDTF part. > 3. Modify TableAPI about UDF/UDTF > 4. Add Test. -- This message was sent by Atlassian JIRA (v6.3.15#6346)