Dan Zou created FLINK-33313:
---
Summary: RexNodeExtractor fails to extract conditions with binary
literal
Key: FLINK-33313
URL: https://issues.apache.org/jira/browse/FLINK-33313
Project: Flink
Issue Type: Bug
Components: Table SQL / Planner
Reporter: Dan Zou
An exception will be thrown when we try to extract conditions with binary
literal in RexNodeExtractor. Here is a test I add in `RexNodeExtractorTest`
which could reproduce the case.
{code:java}
@Test
def testExtractConditionWithBinaryLiteral(): Unit = {
// blob
val t0 = rexBuilder.makeInputRef(allFieldTypes.get(5), 5)
// X'616263'
val t1 = rexBuilder.makeBinaryLiteral(ByteString.of("616263", 16))
// blob = X'616263'
val a = rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t0, t1)
val relBuilder: RexBuilder = new FlinkRexBuilder(typeFactory)
val (convertedExpressions, unconvertedRexNodes) =
extractConjunctiveConditions(a, -1, allFieldNames, relBuilder,
functionCatalog)
val expected: Array[Expression] = Array($"blob" === Array[Byte](97, 98, 99))
assertExpressionArrayEquals(expected, convertedExpressions)
assertEquals(0, unconvertedRexNodes.length)
}
{code}
And here is the exception stack:
{code:java}
org.apache.flink.table.api.ValidationException: Data type 'BINARY(3) NOT NULL'
with conversion class '[B' does not support a value literal of class
'org.apache.calcite.avatica.util.ByteString'.
at
org.apache.flink.table.expressions.ValueLiteralExpression.validateValueDataType(ValueLiteralExpression.java:294)
at
org.apache.flink.table.expressions.ValueLiteralExpression.(ValueLiteralExpression.java:79)
at
org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral(ApiExpressionUtils.java:251)
at
org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.visitLiteral(RexNodeExtractor.scala:503)
at
org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.visitLiteral(RexNodeExtractor.scala:393)
at org.apache.calcite.rex.RexLiteral.accept(RexLiteral.java:1217)
at
org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.$anonfun$visitCall$3(RexNodeExtractor.scala:509)
at
scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:233)
at scala.collection.Iterator.foreach(Iterator.scala:937)
at scala.collection.Iterator.foreach$(Iterator.scala:937)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1425)
at scala.collection.IterableLike.foreach(IterableLike.scala:70)
at scala.collection.IterableLike.foreach$(IterableLike.scala:69)
at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
at scala.collection.TraversableLike.map(TraversableLike.scala:233)
at scala.collection.TraversableLike.map$(TraversableLike.scala:226)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at
org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.visitCall(RexNodeExtractor.scala:509)
at
org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.visitCall(RexNodeExtractor.scala:393)
at org.apache.calcite.rex.RexCall.accept(RexCall.java:189)
at
org.apache.flink.table.planner.plan.utils.RexNodeExtractor$.$anonfun$extractConjunctiveConditions$2(RexNodeExtractor.scala:158)
at scala.collection.Iterator.foreach(Iterator.scala:937)
at scala.collection.Iterator.foreach$(Iterator.scala:937)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1425)
at scala.collection.IterableLike.foreach(IterableLike.scala:70)
at scala.collection.IterableLike.foreach$(IterableLike.scala:69)
at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
at
org.apache.flink.table.planner.plan.utils.RexNodeExtractor$.extractConjunctiveConditions(RexNodeExtractor.scala:157)
at
org.apache.flink.table.planner.plan.utils.RexNodeExtractor$.extractConjunctiveConditions(RexNodeExtractor.scala:119)
at
org.apache.flink.table.planner.plan.utils.RexNodeExtractorTest.extractConjunctiveConditions(RexNodeExtractorTest.scala:785)
at
org.apache.flink.table.planner.plan.utils.RexNodeExtractorTest.testExtractConditionWithBinaryLiteral(RexNodeExtractorTest.scala:160)
{code}
IMO, we should cast `ByteString` to `byte[]` in
RexNodeToExpressionConverter#visitLiteral
--
This message was sent by Atlassian Jira
(v8.20.10#820010)