Liu created FLINK-24871:
---------------------------

             Summary: Flink SQL hive reports IndexOutOfBoundsException when 
using trim in where clause
                 Key: FLINK-24871
                 URL: https://issues.apache.org/jira/browse/FLINK-24871
             Project: Flink
          Issue Type: Improvement
            Reporter: Liu


The problem can be reproduced as follow:

In class HiveDialectITCase, define the test testTrimError

 
{code:java}
@Test
public void testTrimError() {
    tableEnv.executeSql("create table src (x int,y string)");
    tableEnv.executeSql("select * from src where trim(y) != ''");
} {code}
Executing it will throw the following exception.

 
{panel}
java.lang.IndexOutOfBoundsException: index (2) must be less than size (1)

    at 
com.google.common.base.Preconditions.checkElementIndex(Preconditions.java:1345)
    at 
com.google.common.base.Preconditions.checkElementIndex(Preconditions.java:1327)
    at 
com.google.common.collect.SingletonImmutableList.get(SingletonImmutableList.java:43)
    at 
org.apache.calcite.rex.RexCallBinding.getOperandType(RexCallBinding.java:136)
    at 
org.apache.calcite.sql.type.OrdinalReturnTypeInference.inferReturnType(OrdinalReturnTypeInference.java:40)
    at 
org.apache.calcite.sql.type.SqlTypeTransformCascade.inferReturnType(SqlTypeTransformCascade.java:56)
    at 
org.apache.calcite.sql.type.SqlTypeTransformCascade.inferReturnType(SqlTypeTransformCascade.java:56)
    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.visitCall(SqlFunctionConverter.java:107)
    at 
org.apache.flink.table.planner.delegation.hive.SqlFunctionConverter.visitCall(SqlFunctionConverter.java:56)
    at org.apache.calcite.rex.RexCall.accept(RexCall.java:174)
    at org.apache.calcite.rex.RexShuttle.visitList(RexShuttle.java:158)
    at 
org.apache.flink.table.planner.delegation.hive.SqlFunctionConverter.visitCall(SqlFunctionConverter.java:107)
    at 
org.apache.flink.table.planner.delegation.hive.SqlFunctionConverter.visitCall(SqlFunctionConverter.java:56)
    at org.apache.calcite.rex.RexCall.accept(RexCall.java:174)
    at 
org.apache.flink.table.planner.delegation.hive.HiveParserCalcitePlanner.genFilterRelNode(HiveParserCalcitePlanner.java:914)
    at 
org.apache.flink.table.planner.delegation.hive.HiveParserCalcitePlanner.genFilterRelNode(HiveParserCalcitePlanner.java:1082)
    at 
org.apache.flink.table.planner.delegation.hive.HiveParserCalcitePlanner.genFilterLogicalPlan(HiveParserCalcitePlanner.java:1099)
    at 
org.apache.flink.table.planner.delegation.hive.HiveParserCalcitePlanner.genLogicalPlan(HiveParserCalcitePlanner.java:2736)
    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:290)
    at 
org.apache.flink.table.planner.delegation.hive.HiveParser.processCmd(HiveParser.java:238)
    at 
org.apache.flink.table.planner.delegation.hive.HiveParser.parse(HiveParser.java:208)
    at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:735)
    at 
org.apache.flink.connectors.hive.HiveDialectITCase.testTrimError(HiveDialectITCase.java:366)
    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)
{panel}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to