tartarus created FLINK-28158: -------------------------------- Summary: Flink supports all modes of Hive UDAF (PARTIAL1, PARTIAL2, FINAL, COMPLETE) Key: FLINK-28158 URL: https://issues.apache.org/jira/browse/FLINK-28158 Project: Flink Issue Type: Sub-task Components: Connectors / Hive Affects Versions: 1.15.0 Reporter: tartarus
Currently Flink UDAF only supports Hive UDAF's PARTIAL_1 and FINAL mode. When Flink uses Hive's UDAF percent_rank, it fails with the following exception message {code:java} org.apache.flink.table.api.TableException: Unexpected error in type inference logic of function 'percent_rank'. This is a bug. at org.apache.flink.table.types.inference.TypeInferenceUtil.createUnexpectedException(TypeInferenceUtil.java:206) at org.apache.flink.table.planner.functions.inference.TypeInferenceReturnInference.inferReturnType(TypeInferenceReturnInference.java:80) at org.apache.calcite.sql.SqlOperator.inferReturnType(SqlOperator.java:482) at org.apache.calcite.rex.RexBuilder.deriveReturnType(RexBuilder.java:283) at org.apache.calcite.rex.RexBuilder.makeCall(RexBuilder.java:257) at org.apache.flink.table.planner.delegation.hive.SqlFunctionConverter.visitOver(SqlFunctionConverter.java:121) at org.apache.flink.table.planner.delegation.hive.SqlFunctionConverter.visitOver(SqlFunctionConverter.java:56) at org.apache.calcite.rex.RexOver.accept(RexOver.java:121) at org.apache.flink.table.planner.delegation.hive.HiveParserCalcitePlanner.getWindowRexAndType(HiveParserCalcitePlanner.java:1859) at org.apache.flink.table.planner.delegation.hive.HiveParserCalcitePlanner.genSelectForWindowing(HiveParserCalcitePlanner.java:1913) at org.apache.flink.table.planner.delegation.hive.HiveParserCalcitePlanner.genSelectLogicalPlan(HiveParserCalcitePlanner.java:2002) at org.apache.flink.table.planner.delegation.hive.HiveParserCalcitePlanner.genLogicalPlan(HiveParserCalcitePlanner.java:2751) at org.apache.flink.table.planner.delegation.hive.HiveParserCalcitePlanner.logicalPlan(HiveParserCalcitePlanner.java:284) at org.apache.flink.table.planner.delegation.hive.HiveParserCalcitePlanner.genLogicalPlan(HiveParserCalcitePlanner.java:272) at org.apache.flink.table.planner.delegation.hive.HiveParser.analyzeSql(HiveParser.java:303) at org.apache.flink.table.planner.delegation.hive.HiveParser.processCmd(HiveParser.java:251) at org.apache.flink.table.planner.delegation.hive.HiveParser.parse(HiveParser.java:211) at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:695) at org.apache.flink.connectors.hive.HiveDialectITCase.testPercent_rank(HiveDialectITCase.java:800) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.ParentRunner.run(ParentRunner.java:413) at org.junit.runner.JUnitCore.run(JUnitCore.java:137) at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:33) at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:235) at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:54) Caused by: org.apache.flink.table.functions.hive.FlinkHiveUDFException: Failed to get Hive result type from org.apache.hadoop.hive.ql.udf.generic.GenericUDAFPercentRank at org.apache.flink.table.functions.hive.HiveGenericUDAF.inferReturnType(HiveGenericUDAF.java:249) at org.apache.flink.table.functions.hive.HiveFunction$HiveFunctionOutputStrategy.inferType(HiveFunction.java:122) at org.apache.flink.table.types.inference.TypeInferenceUtil.inferOutputType(TypeInferenceUtil.java:151) at org.apache.flink.table.planner.functions.inference.TypeInferenceReturnInference.inferReturnTypeOrError(TypeInferenceReturnInference.java:99) at org.apache.flink.table.planner.functions.inference.TypeInferenceReturnInference.inferReturnType(TypeInferenceReturnInference.java:76) ... 44 more Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Only COMPLETE mode supported for Rank function at org.apache.hadoop.hive.ql.udf.generic.GenericUDAFRank$GenericUDAFAbstractRankEvaluator.init(GenericUDAFRank.java:124) at org.apache.hadoop.hive.ql.udf.generic.GenericUDAFPercentRank$GenericUDAFPercentRankEvaluator.init(GenericUDAFPercentRank.java:59) at org.apache.flink.table.functions.hive.HiveGenericUDAF.init(HiveGenericUDAF.java:99) at org.apache.flink.table.functions.hive.HiveGenericUDAF.inferReturnType(HiveGenericUDAF.java:243) ... 48 more {code} According to the exception message, we can see that it is because the percentage_rank function requires COMPLETE Mode. We can reproduce it with ITCase: {code:java} @Test public void testPercent_rank() throws Exception { // automatically load hive module in hive-compatible mode HiveModule hiveModule = new HiveModule(hiveCatalog.getHiveVersion()); CoreModule coreModule = CoreModule.INSTANCE; for (String loaded : tableEnv.listModules()) { tableEnv.unloadModule(loaded); } tableEnv.loadModule("hive", hiveModule); tableEnv.loadModule("core", coreModule); // Flink UDAF only supports Hive UDAF's PARTIAL_1 and FINAL mode. tableEnv.executeSql( "create temporary function percent_rank as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFPercentRank'"); tableEnv.executeSql( "create table cbo_t1(key string, value string, c_int int, c_float float, c_boolean boolean)"); List<Row> results = CollectionUtil.iteratorToList( tableEnv.executeSql( "select percent_rank() over(partition by c_float order by key) from cbo_t1") .collect()); } {code} -- This message was sent by Atlassian Jira (v8.20.7#820007)