[
https://issues.apache.org/jira/browse/FLINK-7959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16265080#comment-16265080
]
ASF GitHub Bot commented on FLINK-7959:
---------------------------------------
Github user twalthr commented on a diff in the pull request:
https://github.com/apache/flink/pull/4940#discussion_r152923370
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala
---
@@ -21,75 +21,49 @@ import org.apache.flink.api.common.functions._
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.configuration.Configuration
import org.apache.flink.streaming.api.functions.ProcessFunction
-import org.apache.flink.table.api.TableConfig
import
org.apache.flink.table.codegen.CodeGenUtils.{boxedTypeTermForTypeInfo, newName}
import org.apache.flink.table.codegen.Indenter.toISC
/**
* A code generator for generating Flink
[[org.apache.flink.api.common.functions.Function]]s.
- * Including [[MapFunction]], [[FlatMapFunction]], [[FlatJoinFunction]],
[[ProcessFunction]].
- *
- * @param config configuration that determines runtime behavior
- * @param nullableInput input(s) can be null.
- * @param input1 type information about the first input of the Function
- * @param input2 type information about the second input if the Function
is binary
- * @param input1FieldMapping additional mapping information for input1
- * (e.g. POJO types have no deterministic field order and some input
fields might not be read)
- * @param input2FieldMapping additional mapping information for input2
- * (e.g. POJO types have no deterministic field order and some input
fields might not be read)
+ * Including [[MapFunction]], [[FlatMapFunction]], [[FlatJoinFunction]],
[[ProcessFunction]], and
+ * the corresponding rich version of the functions.
*/
-class FunctionCodeGenerator(
- config: TableConfig,
- nullableInput: Boolean,
- input1: TypeInformation[_ <: Any],
- input2: Option[TypeInformation[_ <: Any]] = None,
- input1FieldMapping: Option[Array[Int]] = None,
- input2FieldMapping: Option[Array[Int]] = None)
- extends CodeGenerator(
- config,
- nullableInput,
- input1,
- input2,
- input1FieldMapping,
- input2FieldMapping) {
-
- /**
- * A code generator for generating unary Flink
- * [[org.apache.flink.api.common.functions.Function]]s with one input.
- *
- * @param config configuration that determines runtime behavior
- * @param nullableInput input(s) can be null.
- * @param input type information about the input of the Function
- * @param inputFieldMapping additional mapping information necessary
for input
- * (e.g. POJO types have no deterministic field order and some input
fields might not be read)
- */
- def this(
- config: TableConfig,
- nullableInput: Boolean,
- input: TypeInformation[Any],
- inputFieldMapping: Array[Int]) =
- this(config, nullableInput, input, None, Some(inputFieldMapping))
+object FunctionCodeGenerator {
/**
* Generates a [[org.apache.flink.api.common.functions.Function]] that
can be passed to Java
* compiler.
*
+ * @param ctx The context of the code generator
* @param name Class name of the Function. Must not be unique but has
to be a valid Java class
* identifier.
* @param clazz Flink Function to be generated.
* @param bodyCode code contents of the SAM (Single Abstract Method).
Inputs, collector, or
* output record can be accessed via the given term
methods.
* @param returnType expected return type
+ * @param input1Type the first input type
+ * @param input1Term the first input term
+ * @param input2Type the second input type, optional.
+ * @param input2Term the second input term.
+ * @param collectorTerm the collector term
+ * @param contextTerm the context term
* @tparam F Flink Function to be generated.
* @tparam T Return type of the Flink Function.
* @return instance of GeneratedFunction
*/
def generateFunction[F <: Function, T <: Any](
- name: String,
- clazz: Class[F],
- bodyCode: String,
- returnType: TypeInformation[T])
- : GeneratedFunction[F, T] = {
+ ctx: CodeGeneratorContext,
+ name: String,
+ clazz: Class[F],
+ bodyCode: String,
+ returnType: TypeInformation[T],
+ input1Type: TypeInformation[_ <: Any],
--- End diff --
That's a lot of parameters. We should really think about moving input
related information to the context.
> Split CodeGenerator into CodeGeneratorContext and ExprCodeGenerator
> -------------------------------------------------------------------
>
> Key: FLINK-7959
> URL: https://issues.apache.org/jira/browse/FLINK-7959
> Project: Flink
> Issue Type: Improvement
> Components: Table API & SQL
> Reporter: Kurt Young
> Assignee: Kurt Young
>
> Right now {{CodeGenerator}} actually acts two roles, one is responsible for
> generating codes from RexNode, and the other one is keeping lots of reusable
> statements. It makes more sense to split these logic into two dedicated
> classes.
> The new {{CodeGeneratorContext}} will keep all the reusable statements, while
> the new {{ExprCodeGenerator}} will only do generating codes from RexNode.
> And for classes like {{AggregationCodeGenerator}} or
> {{FunctionCodeGenerator}}, I think the should not be the subclasses of the
> {{CodeGenerator}}, but should all as standalone classes. They can create
> {{ExprCodeGenerator}} when they need to generating codes from RexNode, and
> they can also generating codes by themselves. The {{CodeGeneratorContext}}
> can be passed around to collect all reusable statements, and list them in the
> final generated class.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)