吴彦祖 created FLINK-23436: --------------------------- Summary: When implements the LookupFunction In SQL that indirect extends (Async)TableFunction, throw exception Key: FLINK-23436 URL: https://issues.apache.org/jira/browse/FLINK-23436 Project: Flink Issue Type: Bug Components: Table SQL / Planner Affects Versions: 1.13.0 Reporter: 吴彦祖 Attachments: image-2021-07-20-15-19-48-794.png, image-2021-07-20-15-21-59-569.png
When I implement the inner LookupTableFunctions, I use a abstract class to hold the common fields and common implements, like mini-batch, local cache, etc. the abstract class extends from (Async)TableFunction in Flink 1.13, and all the inner LookupTableFunctions extends from this class. !image-2021-07-20-15-19-48-794.png! I got the following exception when use lookup join in SQL: {noformat} Exception in thread "main" org.apache.flink.table.api.ValidationException: Could not determine a type inference for lookup function 'default_catalog.default_database.xxx_lookup'. Lookup functions support regular type inference. However, for convenience, the output class can simply be a Row or RowData class in which case the input and output types are derived from the table's schema with default conversion.Exception in thread "main" org.apache.flink.table.api.ValidationException: Could not determine a type inference for lookup function 'default_catalog.default_database.xxx_lookup'. Lookup functions support regular type inference. However, for convenience, the output class can simply be a Row or RowData class in which case the input and output types are derived from the table's schema with default conversion. at org.apache.flink.table.planner.codegen.LookupJoinCodeGenerator$.createLookupTypeInference(LookupJoinCodeGenerator.scala:270) at org.apache.flink.table.planner.codegen.LookupJoinCodeGenerator$.generateLookupFunction(LookupJoinCodeGenerator.scala:166) at org.apache.flink.table.planner.codegen.LookupJoinCodeGenerator$.generateSyncLookupFunction(LookupJoinCodeGenerator.scala:87) at org.apache.flink.table.planner.codegen.LookupJoinCodeGenerator.generateSyncLookupFunction(LookupJoinCodeGenerator.scala) at org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecLookupJoin.createSyncLookupJoin(CommonExecLookupJoin.java:440) at org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecLookupJoin.translateToPlanInternal(CommonExecLookupJoin.java:258) at org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:134) at org.apache.flink.table.planner.plan.nodes.exec.ExecEdge.translateToPlan(ExecEdge.java:247) at org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecCalc.translateToPlanInternal(CommonExecCalc.java:88) at org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:134) at org.apache.flink.table.planner.plan.nodes.exec.ExecEdge.translateToPlan(ExecEdge.java:247) at org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange.translateToPlanInternal(StreamExecExchange.java:75) at org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:134) at org.apache.flink.table.planner.plan.nodes.exec.ExecEdge.translateToPlan(ExecEdge.java:247) at org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate.translateToPlanInternal(StreamExecGroupWindowAggregate.java:192) at org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:134) at org.apache.flink.table.planner.plan.nodes.exec.ExecEdge.translateToPlan(ExecEdge.java:247) at org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecCalc.translateToPlanInternal(CommonExecCalc.java:88) at org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:134) at org.apache.flink.table.planner.plan.nodes.exec.ExecEdge.translateToPlan(ExecEdge.java:247) at org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange.translateToPlanInternal(StreamExecExchange.java:75) at org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:134) at org.apache.flink.table.planner.plan.nodes.exec.ExecEdge.translateToPlan(ExecEdge.java:247) at org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate.translateToPlanInternal(StreamExecGroupAggregate.java:146) at org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:134) at org.apache.flink.table.planner.plan.nodes.exec.ExecEdge.translateToPlan(ExecEdge.java:247) at org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecCalc.translateToPlanInternal(CommonExecCalc.java:88) at org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:134) at org.apache.flink.table.planner.plan.nodes.exec.ExecEdge.translateToPlan(ExecEdge.java:247) at org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecLegacySink.translateToTransformation(CommonExecLegacySink.java:172) at org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecLegacySink.translateToPlanInternal(CommonExecLegacySink.java:105) at org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:134) at org.apache.flink.table.planner.delegation.StreamPlanner$$anonfun$translateToPlan$1.apply(StreamPlanner.scala:70) at org.apache.flink.table.planner.delegation.StreamPlanner$$anonfun$translateToPlan$1.apply(StreamPlanner.scala:69) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.Iterator$class.foreach(Iterator.scala:891) at scala.collection.AbstractIterator.foreach(Iterator.scala:1334) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at scala.collection.AbstractIterable.foreach(Iterable.scala:54) at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) at scala.collection.AbstractTraversable.map(Traversable.scala:104) at org.apache.flink.table.planner.delegation.StreamPlanner.translateToPlan(StreamPlanner.scala:69) at org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:188) at org.apache.flink.table.api.internal.TableEnvironmentImpl.translate(TableEnvironmentImpl.java:1701) at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:761) at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:1039) at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:751) at org.apache.flink.table.examples.java.Jmq4SqlTest.main(Jmq4SqlTest.java:205)Caused by: org.apache.flink.table.api.ValidationException: Could not extract a valid type inference for function class 'org.apache.flink.streaming.connectors.xxx.dynamic.XxxDynamicLookupFunction'. Please check for implementation mistakes and/or provide a corresponding hint. at org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:516) at org.apache.flink.table.types.extraction.TypeInferenceExtractor.extractTypeInference(TypeInferenceExtractor.java:150) at org.apache.flink.table.types.extraction.TypeInferenceExtractor.forTableFunction(TypeInferenceExtractor.java:113) at org.apache.flink.table.functions.TableFunction.getTypeInference(TableFunction.java:207) at org.apache.flink.table.planner.codegen.LookupJoinCodeGenerator$.createLookupTypeInference(LookupJoinCodeGenerator.scala:251) ... 48 moreCaused by: org.apache.flink.table.api.ValidationException: Error in extracting a signature to output mapping. at org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:516) at org.apache.flink.table.types.extraction.FunctionMappingExtractor.extractOutputMapping(FunctionMappingExtractor.java:117) at org.apache.flink.table.types.extraction.TypeInferenceExtractor.extractTypeInferenceOrError(TypeInferenceExtractor.java:161) at org.apache.flink.table.types.extraction.TypeInferenceExtractor.extractTypeInference(TypeInferenceExtractor.java:148) ... 51 moreCaused by: org.apache.flink.table.api.ValidationException: Unable to extract a type inference from method:public void org.apache.flink.table.lookup.dynamic.BaseLookupTableFunction.eval(java.lang.Object[]) at org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:516) at org.apache.flink.table.types.extraction.FunctionMappingExtractor.extractResultMappings(FunctionMappingExtractor.java:183) at org.apache.flink.table.types.extraction.FunctionMappingExtractor.extractOutputMapping(FunctionMappingExtractor.java:114) ... 53 moreCaused by: org.apache.flink.table.api.ValidationException: Could not extract a data type from 'interface org.apache.flink.table.data.RowData' in generic class 'org.apache.flink.table.functions.TableFunction' in class org.apache.flink.streaming.connectors.xxx.dynamic.XxxDynamicLookupFunction. Please pass the required data type manually or allow RAW types. at org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:516) at org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeOrRawWithTemplate(DataTypeExtractor.java:241) at org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeOrRaw(DataTypeExtractor.java:219) at org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeWithClassContext(DataTypeExtractor.java:195) at org.apache.flink.table.types.extraction.DataTypeExtractor.extractFromGeneric(DataTypeExtractor.java:125) at org.apache.flink.table.types.extraction.FunctionMappingExtractor.lambda$createGenericResultExtraction$13(FunctionMappingExtractor.java:478) at org.apache.flink.table.types.extraction.FunctionMappingExtractor.putExtractedResultMappings(FunctionMappingExtractor.java:319) at org.apache.flink.table.types.extraction.FunctionMappingExtractor.collectMethodMappings(FunctionMappingExtractor.java:269) at org.apache.flink.table.types.extraction.FunctionMappingExtractor.extractResultMappings(FunctionMappingExtractor.java:169) ... 54 moreCaused by: org.apache.flink.table.api.ValidationException: Cannot extract a data type from an internal 'org.apache.flink.table.data.RowData' class without further information. Please use annotations to define the full logical type. at org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:516) at org.apache.flink.table.types.extraction.ExtractionUtils.extractionError(ExtractionUtils.java:511) at org.apache.flink.table.types.extraction.DataTypeExtractor.checkForCommonErrors(DataTypeExtractor.java:368) at org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeOrError(DataTypeExtractor.java:273) at org.apache.flink.table.types.extraction.DataTypeExtractor.extractDataTypeOrRawWithTemplate(DataTypeExtractor.java:233) ... 61 more {noformat} I debug the code, found the LookupJoinCodeGenerator use the method 'extractSimpleGeneric' in ExtractionUtils to extract output type of Tablefunctions. And the method as below: !image-2021-07-20-15-21-59-569.png! So, may be we should consider the situation that indirect extends. I modified the code as bellow, to traverse all the super classes to the base class until find the real class type that generic parameter in pos: {code:java} // code placeholder public static Optional<Class<?>> extractSimpleGeneric( Class<?> baseClass, Class<?> clazz, int pos) { try { if (null == clazz.getSuperclass() || !baseClass.isAssignableFrom(clazz.getSuperclass())) { return Optional.empty(); } return getParameterizedTypeClassSimple(clazz, baseClass, pos); } catch (Exception unused) { return Optional.empty(); } } /** * extract a generic parameter from a given class and the position of generic parameters * it will traverse all the super classes to the base class until find the generic parameter in pos has real class type * * * @param clazz subClass * @param baseClass target super class * @param pos the position of generic parameter * * Note: By default, we assume that the generic parameter of the subclass and the baseClass are in the same position. * * */ private static Optional<Class<?>> getParameterizedTypeClassSimple(Class<?> clazz, Class<?> baseClass, int pos) { do { if (clazz == baseClass) { break; } Type genericType= clazz.getGenericSuperclass(); if (genericType instanceof ParameterizedType) { Type[] actualTypes = ((ParameterizedType) genericType).getActualTypeArguments(); if (actualTypes.length > pos) { Class<?> typeClass = toClass(actualTypes[pos]); if (null != typeClass){ return Optional.of(typeClass); } } } } while(null != (clazz = clazz.getSuperclass())); return Optional.empty(); } {code} Of cause , by default, we assume that the position of generic parameter of the subclass is same as the baseClass. It is not applicable to the situation that diff generic parameter position like {code:java} // code placeholder UdfTableFunction extends BaseTableFunction<T, P> extends TableFunction<P> {code} I support this situation in a inelegant way that using package sun.* and reflection. Considering that this method should only be used for very specific use cases(currently, only used in lookup join), we can use the simple way. I can help to fixed it if necessary. -- This message was sent by Atlassian Jira (v8.3.4#803005)