[ https://issues.apache.org/jira/browse/FLINK-7959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16265082#comment-16265082 ]
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_r152923905 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/InputFormatCodeGenerator.scala --- @@ -19,50 +19,47 @@ package org.apache.flink.table.codegen import org.apache.flink.api.common.io.GenericInputFormat import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.table.api.TableConfig import org.apache.flink.table.codegen.CodeGenUtils.newName import org.apache.flink.table.codegen.Indenter.toISC import org.apache.flink.types.Row /** * A code generator for generating Flink [[GenericInputFormat]]s. - * - * @param config configuration that determines runtime behavior */ -class InputFormatCodeGenerator( - config: TableConfig) - extends CodeGenerator(config, false, new RowTypeInfo(), None, None) { - +object InputFormatCodeGenerator { /** * Generates a values input format that can be passed to Java compiler. * + * @param ctx The code generator context * @param name Class name of the input format. Must not be unique but has to be a * valid Java class identifier. * @param records code for creating records * @param returnType expected return type + * @param outRecordTerm term of the output * @tparam T Return type of the Flink Function. * @return instance of GeneratedFunction */ def generateValuesInputFormat[T <: Row]( - name: String, - records: Seq[String], - returnType: TypeInformation[T]) - : GeneratedInput[GenericInputFormat[T], T] = { + ctx: CodeGeneratorContext, + name: String, + records: Seq[String], + returnType: TypeInformation[T], + outRecordTerm: String = CodeGeneratorContext.DEFAULT_OUT_RECORD_TERM) --- End diff -- Do we need this parameter? > 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)