You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@calcite.apache.org by "Sergey Nuyanzin (Jira)" <ji...@apache.org> on 2022/11/13 22:51:00 UTC
[jira] [Created] (CALCITE-5380) CompositeSingleOperandTypeChecker fails with index (1) must be less than size (1)
Sergey Nuyanzin created CALCITE-5380:
----------------------------------------
Summary: CompositeSingleOperandTypeChecker fails with index (1) must be less than size (1)
Key: CALCITE-5380
URL: https://issues.apache.org/jira/browse/CALCITE-5380
Project: Calcite
Issue Type: Improvement
Components: core
Affects Versions: 1.33.0
Reporter: Sergey Nuyanzin
I try to check main branch against existing Flink tests.
One of the things I faced is that existing Flink function
{code:java}
public class SqlListAggFunction extends SqlAggFunction {
public SqlListAggFunction() {
super(
"LISTAGG",
null,
SqlKind.LISTAGG,
ReturnTypes.ARG0_NULLABLE,
null,
OperandTypes.or(
OperandTypes.CHARACTER,
OperandTypes.sequence(
"'LISTAGG(<CHARACTER>, <CHARACTER_LITERAL>)'",
OperandTypes.CHARACTER,
OperandTypes.and(OperandTypes.CHARACTER, OperandTypes.LITERAL))),
SqlFunctionCategory.SYSTEM,
false,
false);
}
@Override
public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
return ImmutableList.of(
typeFactory.createTypeWithNullability(
typeFactory.createSqlType(SqlTypeName.VARCHAR), true));
}
@Override
public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
return typeFactory.createSqlType(SqlTypeName.VARCHAR);
}
}
{code}
started to fail with
{noformat}
org.apache.flink.table.api.ValidationException: SQL validation failed. index (1) must be less than size (1)
at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:186)
at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:113)
at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:263)
at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:106)
at org.apache.flink.table.api.internal.TableEnvironmentImpl.sqlQuery(TableEnvironmentImpl.java:734)
at org.apache.flink.table.api.stream.ExplainTest.testMiniBatchIntervalInfer(ExplainTest.scala:150)
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.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:258)
at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:61)
at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
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.run(ParentRunner.java:413)
at org.junit.runners.Suite.runChild(Suite.java:128)
at org.junit.runners.Suite.runChild(Suite.java:27)
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$1.execute(IdeaTestRunner.java:38)
at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11)
at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35)
at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:235)
at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:54)
Caused by: java.lang.IndexOutOfBoundsException: index (1) must be less than size (1)
at com.google.common.base.Preconditions.checkElementIndex(Preconditions.java:1355)
at com.google.common.base.Preconditions.checkElementIndex(Preconditions.java:1337)
at com.google.common.collect.SingletonImmutableList.get(SingletonImmutableList.java:44)
at org.apache.calcite.sql.type.FamilyOperandTypeChecker.checkSingleOperandType(FamilyOperandTypeChecker.java:68)
at org.apache.calcite.sql.type.CompositeSingleOperandTypeChecker.checkSingleOperandType(CompositeSingleOperandTypeChecker.java:70)
at org.apache.calcite.sql.type.CompositeOperandTypeChecker.check(CompositeOperandTypeChecker.java:312)
at org.apache.calcite.sql.type.CompositeOperandTypeChecker.checkOperandTypes(CompositeOperandTypeChecker.java:263)
at org.apache.calcite.sql.type.CompositeOperandTypeChecker.check(CompositeOperandTypeChecker.java:345)
at org.apache.calcite.sql.type.CompositeOperandTypeChecker.checkOperandTypes(CompositeOperandTypeChecker.java:263)
at org.apache.calcite.sql.SqlOperator.checkOperandTypes(SqlOperator.java:753)
at org.apache.calcite.sql.SqlOperator.validateOperands(SqlOperator.java:499)
at org.apache.calcite.sql.SqlFunction.deriveType(SqlFunction.java:335)
at org.apache.calcite.sql.SqlFunction.deriveType(SqlFunction.java:231)
at org.apache.calcite.sql.validate.SqlValidatorImpl$DeriveTypeVisitor.visit(SqlValidatorImpl.java:6369)
at org.apache.calcite.sql.validate.SqlValidatorImpl$DeriveTypeVisitor.visit(SqlValidatorImpl.java:6354)
at org.apache.calcite.sql.SqlCall.accept(SqlCall.java:161)
at org.apache.calcite.sql.validate.SqlValidatorImpl.deriveTypeImpl(SqlValidatorImpl.java:1880)
at org.apache.calcite.sql.validate.SqlValidatorImpl.deriveType(SqlValidatorImpl.java:1871)
at org.apache.calcite.sql.validate.SqlValidatorImpl.expandSelectItem(SqlValidatorImpl.java:461)
at org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelectList(SqlValidatorImpl.java:4456)
at org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelect(SqlValidatorImpl.java:3662)
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:1061)
at org.apache.calcite.sql.validate.SqlValidatorImpl.validateQuery(SqlValidatorImpl.java:1036)
at org.apache.calcite.sql.SqlSelect.validate(SqlSelect.java:248)
at org.apache.calcite.sql.validate.SqlValidatorImpl.validateScopedExpression(SqlValidatorImpl.java:1011)
at org.apache.calcite.sql.validate.SqlValidatorImpl.validate(SqlValidatorImpl.java:760)
at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:182)
... 44 more
{noformat}
The debugging shows that it starts failing after https://github.com/apache/calcite/commit/33f4ab40bbee26e06209061c35a422f2f1e05371
I noticed that {{iFormalOperand}} for {{org.apache.calcite.sql.type.SqlSingleOperandTypeChecker#checkSingleOperandType}} in {{org.apache.calcite.sql.type.CompositeOperandTypeChecker}} depends on rule class and for {{org.apache.calcite.sql.type.CompositeSingleOperandTypeChecker}} it does not.
After changing it in same way as for {{org.apache.calcite.sql.type.CompositeOperandTypeChecker}} it starts working again.
Not sure if it the right way (at least would like to double check).
Existing calcite tests are passing with this change and without.
I will see how easy it to retrieve Flink test to Calcite for that
--
This message was sent by Atlassian Jira
(v8.20.10#820010)