[ https://issues.apache.org/jira/browse/FLINK-15592?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17017788#comment-17017788 ]
Rui Li commented on FLINK-15592: -------------------------------- Following is the list of functions that are in {{FlinkSqlOperatorTable}} and *not* in {{BuiltInFunctionDefinitions}}. [~lzljs3620320] any suggestions which of them should be included in the black list? {code} [$literalchain, $scalar_query, $sum0, *, +, -, /, /int, <, <=, <>, =, >, >=, ascii, auxiliary_group, between asymmetric, between symmetric, case, char_length, character_length, chr, classifier, coalesce, convert_tz, current_date, current_time, current_timestamp, date_format, dayofmonth, dayofweek, dayofyear, decode, dense_rank, dot, encode, except, except all, exists, final, first, first_value, from_base64, from_unixtime, group_id, grouping, grouping_id, hash_code, hop, hop_end, hop_proctime, hop_rowtime, hop_start, hour, if, initcap, instr, intersect, intersect all, is distinct from, is false, is not distinct from, is not false, is not null, is not true, is not unknown, is null, is true, is unknown, is_alpha, is_decimal, is_digit, item, lag, last, last_value, lead, left, listagg, localtime, localtimestamp, locate, lower, match_proctime, match_rowtime, minute, month, multiset, next, not between asymmetric, not between symmetric, not in, not like, not similar to, now, nullif, nulls first, nulls last, overlaps, parse_url, prev, print, proctime_materialize, quarter, rand_integer, rank, regexp, regexp_extract, regexp_replace, reinterpret, reverse, right, row_number, running, second, session, session_end, session_proctime, session_rowtime, session_start, similar to, single_value, split_index, stddev, stddev_pop, stddev_samp, str_to_map, streamrecord_timestamp, substr, timestampadd, timestampdiff, to_base64, to_date, to_timestamp, tumble, tumble_end, tumble_proctime, tumble_rowtime, tumble_start, union, union all, unix_timestamp, upper, var_pop, var_samp, variance, week, year, ||] {code} Or maybe we can take a conservative approach and ban all of them in {{HiveModule}}? > Streaming sql throw hive exception when it doesn't use any hive table > --------------------------------------------------------------------- > > Key: FLINK-15592 > URL: https://issues.apache.org/jira/browse/FLINK-15592 > Project: Flink > Issue Type: Bug > Components: Table SQL / Planner > Affects Versions: 1.10.0 > Reporter: Jeff Zhang > Priority: Critical > Fix For: 1.10.0 > > > I use the following streaming sql to query a kafka table whose metadata is > store in hive metastore via HiveCatalog. But it will throw hive related > exception which is very confusing. > SQL > {code} > SELECT * > FROM ( > SELECT *, > ROW_NUMBER() OVER( > ORDER BY event_ts) AS rownum > FROM source_kafka) > WHERE rownum <= 10 > {code} > Exception > {code} > Caused by: org.apache.flink.table.api.ValidationException: SQL validation > failed. java.lang.reflect.InvocationTargetException > at > org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:130) > at > org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:105) > at > org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:127) > at > org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:66) > at > org.apache.flink.table.api.internal.TableEnvironmentImpl.sqlQuery(TableEnvironmentImpl.java:464) > at > org.apache.zeppelin.flink.sql.AbstractStreamSqlJob.run(AbstractStreamSqlJob.java:103) > ... 13 more > Caused by: java.lang.RuntimeException: > java.lang.reflect.InvocationTargetException > at > org.apache.flink.table.planner.functions.utils.HiveFunctionUtils.invokeGetResultType(HiveFunctionUtils.java:77) > at > org.apache.flink.table.planner.functions.utils.HiveAggSqlFunction.lambda$createReturnTypeInference$0(HiveAggSqlFunction.java:82) > at > org.apache.calcite.sql.SqlOperator.inferReturnType(SqlOperator.java:470) > at > org.apache.calcite.sql.SqlOperator.validateOperands(SqlOperator.java:437) > at org.apache.calcite.sql.SqlFunction.deriveType(SqlFunction.java:303) > at org.apache.calcite.sql.SqlFunction.deriveType(SqlFunction.java:219) > at > org.apache.calcite.sql.validate.SqlValidatorImpl$DeriveTypeVisitor.visit(SqlValidatorImpl.java:5600) > at > org.apache.calcite.sql.validate.SqlValidatorImpl$DeriveTypeVisitor.visit(SqlValidatorImpl.java:5587) > at org.apache.calcite.sql.SqlCall.accept(SqlCall.java:139) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.deriveTypeImpl(SqlValidatorImpl.java:1691) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.deriveType(SqlValidatorImpl.java:1676) > at > org.apache.calcite.sql.SqlCallBinding.getOperandType(SqlCallBinding.java:237) > at > org.apache.calcite.sql.type.OrdinalReturnTypeInference.inferReturnType(OrdinalReturnTypeInference.java:40) > at > org.apache.calcite.sql.type.SqlTypeTransformCascade.inferReturnType(SqlTypeTransformCascade.java:54) > at > org.apache.calcite.sql.SqlOperator.inferReturnType(SqlOperator.java:470) > at > org.apache.calcite.sql.SqlOperator.validateOperands(SqlOperator.java:437) > at > org.apache.calcite.sql.SqlOverOperator.deriveType(SqlOverOperator.java:86) > at > org.apache.calcite.sql.validate.SqlValidatorImpl$DeriveTypeVisitor.visit(SqlValidatorImpl.java:5600) > at > org.apache.calcite.sql.validate.SqlValidatorImpl$DeriveTypeVisitor.visit(SqlValidatorImpl.java:5587) > at org.apache.calcite.sql.SqlCall.accept(SqlCall.java:139) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.deriveTypeImpl(SqlValidatorImpl.java:1691) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.deriveType(SqlValidatorImpl.java:1676) > at > org.apache.calcite.sql.SqlAsOperator.deriveType(SqlAsOperator.java:133) > at > org.apache.calcite.sql.validate.SqlValidatorImpl$DeriveTypeVisitor.visit(SqlValidatorImpl.java:5600) > at > org.apache.calcite.sql.validate.SqlValidatorImpl$DeriveTypeVisitor.visit(SqlValidatorImpl.java:5587) > at org.apache.calcite.sql.SqlCall.accept(SqlCall.java:139) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.deriveTypeImpl(SqlValidatorImpl.java:1691) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.deriveType(SqlValidatorImpl.java:1676) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.expandSelectItem(SqlValidatorImpl.java:479) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelectList(SqlValidatorImpl.java:4105) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelect(SqlValidatorImpl.java:3389) > at > org.apache.calcite.sql.validate.SelectNamespace.validateImpl(SelectNamespace.java:60) > at > org.apache.calcite.sql.validate.AbstractNamespace.validate(AbstractNamespace.java:84) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.validateNamespace(SqlValidatorImpl.java:1008) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.validateQuery(SqlValidatorImpl.java:968) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3122) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3104) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelect(SqlValidatorImpl.java:3376) > at > org.apache.calcite.sql.validate.SelectNamespace.validateImpl(SelectNamespace.java:60) > at > org.apache.calcite.sql.validate.AbstractNamespace.validate(AbstractNamespace.java:84) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.validateNamespace(SqlValidatorImpl.java:1008) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.validateQuery(SqlValidatorImpl.java:968) > at org.apache.calcite.sql.SqlSelect.validate(SqlSelect.java:216) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.validateScopedExpression(SqlValidatorImpl.java:943) > at > org.apache.calcite.sql.validate.SqlValidatorImpl.validate(SqlValidatorImpl.java:650) > at > org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:126) > ... 18 more > Caused by: java.lang.reflect.InvocationTargetException > 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.apache.flink.table.planner.functions.utils.HiveFunctionUtils.invokeGetResultType(HiveFunctionUtils.java:73) > ... 63 more > Caused by: org.apache.flink.table.functions.hive.FlinkHiveUDFException: > Failed to get Hive result type from > org.apache.hadoop.hive.ql.udf.generic.GenericUDAFRowNumber > at > org.apache.flink.table.functions.hive.HiveGenericUDAF.getHiveResultType(HiveGenericUDAF.java:202) > ... 68 more > Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Only COMPLETE > mode supported for row_number function > at > org.apache.hadoop.hive.ql.udf.generic.GenericUDAFRowNumber$GenericUDAFAbstractRowNumberEvaluator.init(GenericUDAFRowNumber.java:100) > at > org.apache.flink.table.functions.hive.HiveGenericUDAF.init(HiveGenericUDAF.java:93) > at > org.apache.flink.table.functions.hive.HiveGenericUDAF.getHiveResultType(HiveGenericUDAF.java:196) > ... 68 more > ERROR > Took 2 sec. Last updated by anonymous at January 15 2020, 11:12:16 AM. > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005)