You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by "Caizhi Weng (Jira)" <ji...@apache.org> on 2021/03/30 02:47:00 UTC

[jira] [Created] (FLINK-22021) PushFilterIntoLegacyTableSourceScanRule fails to deal with INTERVAL types

Caizhi Weng created FLINK-22021:
-----------------------------------

             Summary: PushFilterIntoLegacyTableSourceScanRule fails to deal with INTERVAL types
                 Key: FLINK-22021
                 URL: https://issues.apache.org/jira/browse/FLINK-22021
             Project: Flink
          Issue Type: Bug
          Components: Table SQL / Planner
    Affects Versions: 1.13.0
            Reporter: Caizhi Weng
             Fix For: 1.13.0


Add the following test case to {{PushFilterIntoLegacyTableSourceScanRuleTest}} to reproduce this bug:

{code:scala}
@Test
  def testWithInterval(): Unit = {
    val schema = TableSchema
      .builder()
      .field("a", DataTypes.STRING)
      .field("b", DataTypes.STRING)
      .build()

    val data = List(Row.of("2021-03-30 10:00:00", "2021-03-30 11:00:00"))
    TestLegacyFilterableTableSource.createTemporaryTable(
      util.tableEnv,
      schema,
      "MTable",
      isBounded = true,
      data,
      List("a", "b"))

    util.verifyRelPlan(
      """
        |SELECT * FROM MTable
        |WHERE
        |  TIMESTAMPADD(HOUR, 1, TO_TIMESTAMP(a)) >= TO_TIMESTAMP(b)
        |  OR
        |  TIMESTAMPADD(YEAR, 1, TO_TIMESTAMP(b)) >= TO_TIMESTAMP(a)
        |""".stripMargin)
  }
{code}

The exception stack is
{code}
org.apache.flink.table.api.ValidationException: Data type 'INTERVAL SECOND(3) NOT NULL' with conversion class 'java.time.Duration' does not support a value literal of class 'java.math.BigDecimal'.

	at org.apache.flink.table.expressions.ValueLiteralExpression.validateValueDataType(ValueLiteralExpression.java:286)
	at org.apache.flink.table.expressions.ValueLiteralExpression.<init>(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:451)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.visitLiteral(RexNodeExtractor.scala:359)
	at org.apache.calcite.rex.RexLiteral.accept(RexLiteral.java:1173)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter$$anonfun$8.apply(RexNodeExtractor.scala:459)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter$$anonfun$8.apply(RexNodeExtractor.scala:459)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.Iterator$class.foreach(Iterator.scala:891)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
	at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
	at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
	at scala.collection.AbstractTraversable.map(Traversable.scala:104)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.visitCall(RexNodeExtractor.scala:458)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.visitCall(RexNodeExtractor.scala:359)
	at org.apache.calcite.rex.RexCall.accept(RexCall.java:174)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter$$anonfun$8.apply(RexNodeExtractor.scala:459)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter$$anonfun$8.apply(RexNodeExtractor.scala:459)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.Iterator$class.foreach(Iterator.scala:891)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
	at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
	at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
	at scala.collection.AbstractTraversable.map(Traversable.scala:104)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.visitCall(RexNodeExtractor.scala:458)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.visitCall(RexNodeExtractor.scala:359)
	at org.apache.calcite.rex.RexCall.accept(RexCall.java:174)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter$$anonfun$8.apply(RexNodeExtractor.scala:459)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter$$anonfun$8.apply(RexNodeExtractor.scala:459)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.Iterator$class.foreach(Iterator.scala:891)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
	at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
	at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
	at scala.collection.AbstractTraversable.map(Traversable.scala:104)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.visitCall(RexNodeExtractor.scala:458)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.visitCall(RexNodeExtractor.scala:359)
	at org.apache.calcite.rex.RexCall.accept(RexCall.java:174)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter$$anonfun$8.apply(RexNodeExtractor.scala:459)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter$$anonfun$8.apply(RexNodeExtractor.scala:459)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.Iterator$class.foreach(Iterator.scala:891)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
	at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
	at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
	at scala.collection.AbstractTraversable.map(Traversable.scala:104)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.visitCall(RexNodeExtractor.scala:458)
	at org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter.visitCall(RexNodeExtractor.scala:359)
	at org.apache.calcite.rex.RexCall.accept(RexCall.java:174)
	at org.apache.flink.table.planner.plan.utils.RexNodeExtractor$$anonfun$extractConjunctiveConditions$1.apply(RexNodeExtractor.scala:136)
	at org.apache.flink.table.planner.plan.utils.RexNodeExtractor$$anonfun$extractConjunctiveConditions$1.apply(RexNodeExtractor.scala:135)
	at scala.collection.Iterator$class.foreach(Iterator.scala:891)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
	at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
	at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
	at org.apache.flink.table.planner.plan.utils.RexNodeExtractor$.extractConjunctiveConditions(RexNodeExtractor.scala:135)
	at org.apache.flink.table.planner.plan.utils.RexNodeExtractor$.extractConjunctiveConditions(RexNodeExtractor.scala:101)
	at org.apache.flink.table.planner.plan.rules.logical.PushFilterIntoLegacyTableSourceScanRule.pushFilterIntoScan(PushFilterIntoLegacyTableSourceScanRule.scala:90)
	at org.apache.flink.table.planner.plan.rules.logical.PushFilterIntoLegacyTableSourceScanRule.onMatch(PushFilterIntoLegacyTableSourceScanRule.scala:77)
	at org.apache.calcite.plan.AbstractRelOptPlanner.fireRule(AbstractRelOptPlanner.java:333)
	at org.apache.calcite.plan.hep.HepPlanner.applyRule(HepPlanner.java:542)
	at org.apache.calcite.plan.hep.HepPlanner.applyRules(HepPlanner.java:407)
	at org.apache.calcite.plan.hep.HepPlanner.executeInstruction(HepPlanner.java:271)
	at org.apache.calcite.plan.hep.HepInstruction$RuleCollection.execute(HepInstruction.java:74)
	at org.apache.calcite.plan.hep.HepPlanner.executeProgram(HepPlanner.java:202)
	at org.apache.calcite.plan.hep.HepPlanner.findBestExp(HepPlanner.java:189)
	at org.apache.flink.table.planner.plan.optimize.program.FlinkHepProgram.optimize(FlinkHepProgram.scala:69)
	at org.apache.flink.table.planner.plan.optimize.program.FlinkHepRuleSetProgram.optimize(FlinkHepRuleSetProgram.scala:87)
	at org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram$$anonfun$optimize$1.apply(FlinkChainedProgram.scala:62)
	at org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram$$anonfun$optimize$1.apply(FlinkChainedProgram.scala:58)
	at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
	at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
	at scala.collection.Iterator$class.foreach(Iterator.scala:891)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
	at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
	at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
	at scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157)
	at scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104)
	at org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram.optimize(FlinkChainedProgram.scala:57)
	at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.optimizeTree(BatchCommonSubGraphBasedOptimizer.scala:87)
	at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.org$apache$flink$table$planner$plan$optimize$BatchCommonSubGraphBasedOptimizer$$optimizeBlock(BatchCommonSubGraphBasedOptimizer.scala:58)
	at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:46)
	at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:46)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.doOptimize(BatchCommonSubGraphBasedOptimizer.scala:46)
	at org.apache.flink.table.planner.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:81)
	at org.apache.flink.table.planner.delegation.PlannerBase.optimize(PlannerBase.scala:304)
	at org.apache.flink.table.planner.utils.TableTestUtilBase.assertPlanEquals(TableTestBase.scala:927)
	at org.apache.flink.table.planner.utils.TableTestUtilBase.doVerifyPlan(TableTestBase.scala:797)
	at org.apache.flink.table.planner.utils.TableTestUtilBase.verifyRelPlan(TableTestBase.scala:402)
	at org.apache.flink.table.planner.plan.rules.logical.PushFilterIntoLegacyTableSourceScanRuleTest.testWithInterval(PushFilterIntoLegacyTableSourceScanRuleTest.scala:188)
	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:50)
	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
	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:239)
	at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
	at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
	at org.junit.rules.RunRules.evaluate(RunRules.java:20)
	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
	at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
	at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68)
	at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:33)
	at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:230)
	at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:58)
{code}

This is because {{RexNodeExtractor#visitLiteral}} does not deal with interval types.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)