Ferenc Csaky created FLINK-38682:
------------------------------------
Summary: Cannot validate SQL if a dynamic parameter points to a
RAW typed field
Key: FLINK-38682
URL: https://issues.apache.org/jira/browse/FLINK-38682
Project: Flink
Issue Type: Bug
Affects Versions: 2.1.1
Reporter: Ferenc Csaky
Assignee: Ferenc Csaky
Fix For: 2.2.0
I have the following SQL:
{code:sql}
SELECT * FROM VectorEmbed ORDER BY cosine_similarity(?, name_vector) DESC;
{code}
Here, {{cosine_similarity}} is a UDF, which takes 2 parameters, both are the
same custom type, which means RAW. Currently, the above query fails to
validate, cause the validator tries to construct a CAST for the dynamic SQL
param, but {{RawRelDataType}} in Flink uses {{SqlTypeName.OTHER}}, and the
existing logic does not accept {{OTHER}} types, so it blows up with the
following exception:
{code}
org.apache.flink.table.api.TableException: Unexpected error in type inference
logic of function 'cosine_similarity'. This is a bug.
at
org.apache.flink.table.types.inference.TypeInferenceUtil.createUnexpectedException(TypeInferenceUtil.java:227)
at
org.apache.flink.table.planner.functions.inference.TypeInferenceOperandChecker.checkOperandTypes(TypeInferenceOperandChecker.java:110)
at
org.apache.calcite.sql.SqlOperator.checkOperandTypes(SqlOperator.java:759)
at
org.apache.calcite.sql.SqlOperator.validateOperands(SqlOperator.java:499)
at org.apache.calcite.sql.SqlFunction.deriveType(SqlFunction.java:373)
at org.apache.calcite.sql.SqlFunction.deriveType(SqlFunction.java:245)
at
org.apache.calcite.sql.validate.SqlValidatorImpl$DeriveTypeVisitor.visit(SqlValidatorImpl.java:6504)
at
org.apache.calcite.sql.validate.SqlValidatorImpl$DeriveTypeVisitor.visit(SqlValidatorImpl.java:6486)
at org.apache.calcite.sql.SqlCall.accept(SqlCall.java:161)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.deriveTypeImpl(SqlValidatorImpl.java:1929)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.deriveType(SqlValidatorImpl.java:1920)
at org.apache.calcite.sql.SqlNode.validateExpr(SqlNode.java:276)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateExpr(SqlValidatorImpl.java:4616)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateOrderItem(SqlValidatorImpl.java:4361)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateOrderItem(SqlValidatorImpl.java:4354)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateOrderList(SqlValidatorImpl.java:4306)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelect(SqlValidatorImpl.java:3736)
at
org.apache.calcite.sql.validate.SelectNamespace.validateImpl(SelectNamespace.java:61)
at
org.apache.calcite.sql.validate.AbstractNamespace.validate(AbstractNamespace.java:88)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateNamespace(SqlValidatorImpl.java:1104)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateQuery(SqlValidatorImpl.java:1079)
at org.apache.calcite.sql.SqlSelect.validate(SqlSelect.java:282)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateScopedExpression(SqlValidatorImpl.java:1054)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validate(SqlValidatorImpl.java:803)
...
Caused by: java.lang.UnsupportedOperationException: Unsupported type when
convertTypeToSpec: OTHER
at
org.apache.calcite.sql.type.SqlTypeUtil.convertTypeToSpec(SqlTypeUtil.java:1120)
at
org.apache.calcite.sql.type.SqlTypeUtil.convertTypeToSpec(SqlTypeUtil.java:1142)
at
org.apache.flink.table.planner.functions.inference.TypeInferenceOperandChecker.castTo(TypeInferenceOperandChecker.java:246)
at
org.apache.flink.table.planner.functions.inference.TypeInferenceOperandChecker.insertImplicitCasts(TypeInferenceOperandChecker.java:234)
at
org.apache.flink.table.planner.functions.inference.TypeInferenceOperandChecker.checkOperandTypesOrError(TypeInferenceOperandChecker.java:213)
at
org.apache.flink.table.planner.functions.inference.TypeInferenceOperandChecker.checkOperandTypes(TypeInferenceOperandChecker.java:103)
... 167 more
{code}
The problem occurs in {{TypeInferenceOperandChecker}}, which calls
{{SqlTypeUtil#convertTypeToSpec}} which does not handle the {{OTHER}} type.
As a solution, I suggest to construct the {{SqlRawTypeNameSpec}} by hand in
case we encouter a {{RawRelDataType}}.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)