[
https://issues.apache.org/jira/browse/FLINK-8274?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16380928#comment-16380928
]
ASF GitHub Bot commented on FLINK-8274:
---------------------------------------
Github user walterddr commented on a diff in the pull request:
https://github.com/apache/flink/pull/5174#discussion_r171357748
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
---
@@ -171,4 +186,47 @@ trait CommonCalc {
rowCnt
}
}
+
+ /**
+ * split origin generated code to split function calls, only used for
calc.
+ * @param generator
+ * @param codeBuffer
+ * @param maxLength
+ * @return (method definitions, method bodies, method callings) of
split function calls
+ */
+ private def generateCalcSplitFunctions(
--- End diff --
Seems like in order to avoid the out of compilation limit issue, you need
to put this in all Common nodes.
Any changes we can put it in `plan/util` and make it more generic? will
probably imagine same thing could happen to any plan node that requires to
generate a row of outputs.
> Fix Java 64K method compiling limitation for CommonCalc
> -------------------------------------------------------
>
> Key: FLINK-8274
> URL: https://issues.apache.org/jira/browse/FLINK-8274
> Project: Flink
> Issue Type: Bug
> Components: Table API & SQL
> Affects Versions: 1.5.0
> Reporter: Ruidong Li
> Assignee: Ruidong Li
> Priority: Critical
>
> For complex SQL Queries, the generated code for {code}DataStreamCalc{code},
> {code}DataSetCalc{code} may exceed Java's method length limitation 64kb.
>
> This issue will split long method to several sub method calls.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)