@@ -23,7 +23,7 @@ import org.apache.flink.streaming.api.functions.ProcessFunction
2323import org .apache .flink .streaming .api .functions .async .{AsyncFunction , RichAsyncFunction }
2424import org .apache .flink .table .planner .codegen .CodeGenUtils ._
2525import org .apache .flink .table .planner .codegen .Indenter .toISC
26- import org .apache .flink .table .runtime .generated .{GeneratedFunction , GeneratedJoinCondition , JoinCondition }
26+ import org .apache .flink .table .runtime .generated .{FilterCondition , GeneratedFilterCondition , GeneratedFunction , GeneratedJoinCondition , JoinCondition }
2727import org .apache .flink .table .types .logical .LogicalType
2828
2929/**
@@ -180,33 +180,46 @@ object FunctionCodeGenerator {
180180 }
181181
182182 /**
183- * Generates a [[JoinCondition ]] that can be passed to Java compiler.
183+ * Generates a [[AbstractRichFunction ]] class code that can be passed to Java compiler. Including
184+ * [[JoinCondition ]] and [[FilterCondition ]] rich functions.
184185 *
185186 * @param ctx
186187 * The context of the code generator
187188 * @param name
188189 * Class name of the Function. Not must be unique but has to be a valid Java class identifier.
190+ * @param clazz
191+ * Function to be generated.
189192 * @param bodyCode
190193 * code contents of the SAM (Single Abstract Method).
191194 * @param input1Term
192195 * the first input term
193196 * @param input2Term
194197 * the second input term.
195198 * @return
196- * instance of GeneratedJoinCondition
199+ * the generated condition function name and code
197200 */
198- def generateJoinCondition (
201+ private def generateCondition [ F <: Function ] (
199202 ctx : CodeGeneratorContext ,
200203 name : String ,
204+ clazz : Class [F ],
201205 bodyCode : String ,
202206 input1Term : String = CodeGenUtils .DEFAULT_INPUT1_TERM ,
203- input2Term : String = CodeGenUtils .DEFAULT_INPUT2_TERM ): GeneratedJoinCondition = {
207+ input2Term : String = CodeGenUtils .DEFAULT_INPUT2_TERM ): ( String , String ) = {
204208 val funcName = newName(name)
205209
210+ val methodHeader = {
211+ if (clazz == classOf [JoinCondition ]) {
212+ s " apply( $ROW_DATA $input1Term, $ROW_DATA $input2Term) "
213+ } else if (clazz == classOf [FilterCondition ]) {
214+ s " apply( $ROW_DATA $input1Term) "
215+ } else {
216+ throw new CodeGenException (s " Unsupported Condition Function $clazz. " )
217+ }
218+ }
206219 val funcCode =
207220 j """
208221 public class $funcName extends ${className[AbstractRichFunction ]}
209- implements ${className[ JoinCondition ] } {
222+ implements ${clazz.getCanonicalName } {
210223
211224 ${ctx.reuseMemberCode()}
212225
@@ -222,7 +235,7 @@ object FunctionCodeGenerator {
222235 }
223236
224237 @Override
225- public boolean apply( $ROW_DATA $input1Term , $ROW_DATA $input2Term ) throws Exception {
238+ public boolean $methodHeader throws Exception {
226239 ${ctx.reusePerRecordCode()}
227240 ${ctx.reuseLocalVariableCode()}
228241 ${ctx.reuseInputUnboxingCode()}
@@ -237,6 +250,61 @@ object FunctionCodeGenerator {
237250 }
238251 """ .stripMargin
239252
253+ (funcName, funcCode)
254+ }
255+
256+ /**
257+ * Generates a [[JoinCondition ]] that can be passed to Java compiler.
258+ *
259+ * @param ctx
260+ * The context of the code generator
261+ * @param name
262+ * Class name of the Function. Not must be unique but has to be a valid Java class identifier.
263+ * @param bodyCode
264+ * code contents of the SAM (Single Abstract Method).
265+ * @param input1Term
266+ * the first input term
267+ * @param input2Term
268+ * the second input term.
269+ * @return
270+ * instance of GeneratedJoinCondition
271+ */
272+ def generateJoinCondition (
273+ ctx : CodeGeneratorContext ,
274+ name : String ,
275+ bodyCode : String ,
276+ input1Term : String = CodeGenUtils .DEFAULT_INPUT1_TERM ,
277+ input2Term : String = CodeGenUtils .DEFAULT_INPUT2_TERM ): GeneratedJoinCondition = {
278+
279+ val (funcName, funcCode) =
280+ generateCondition(ctx, name, classOf [JoinCondition ], bodyCode, input1Term, input2Term)
281+
240282 new GeneratedJoinCondition (funcName, funcCode, ctx.references.toArray, ctx.tableConfig)
241283 }
284+
285+ /**
286+ * Generates a [[FilterCondition ]] that can be passed to Java compiler.
287+ *
288+ * @param ctx
289+ * The context of the code generator
290+ * @param name
291+ * Class name of the Function. Not must be unique but has to be a valid Java class identifier.
292+ * @param bodyCode
293+ * code contents of the SAM (Single Abstract Method).
294+ * @param inputTerm
295+ * the input term
296+ * @return
297+ * instance of GeneratedFilterCondition
298+ */
299+ def generateFilterCondition (
300+ ctx : CodeGeneratorContext ,
301+ name : String ,
302+ bodyCode : String ,
303+ inputTerm : String = CodeGenUtils .DEFAULT_INPUT_TERM ): GeneratedFilterCondition = {
304+
305+ val (funcName, funcCode) =
306+ generateCondition(ctx, name, classOf [FilterCondition ], bodyCode, inputTerm)
307+
308+ new GeneratedFilterCondition (funcName, funcCode, ctx.references.toArray, ctx.tableConfig)
309+ }
242310}
0 commit comments