[jira] [Created] (FLINK-22680) An IndexOutOfBoundsException is thrown out after apply WatermarkAssignerChangelogNormalizeTransposeRule

classic Classic list List threaded Threaded
1 message Options
Reply | Threaded
Open this post in threaded view
|

[jira] [Created] (FLINK-22680) An IndexOutOfBoundsException is thrown out after apply WatermarkAssignerChangelogNormalizeTransposeRule

Shang Yuanchun (Jira)
Andy created FLINK-22680:
----------------------------

             Summary: An IndexOutOfBoundsException is thrown out after apply WatermarkAssignerChangelogNormalizeTransposeRule
                 Key: FLINK-22680
                 URL: https://issues.apache.org/jira/browse/FLINK-22680
             Project: Flink
          Issue Type: Bug
          Components: Table SQL / Planner
            Reporter: Andy
             Fix For: 1.14.0
         Attachments: image-2021-05-17-16-46-00-789.png

{code:java}
@Test
def testUnResolvedWindowAggregateOnUpsertSource(): Unit = {

  def localDateTime(epochSecond: Long): LocalDateTime = {
    LocalDateTime.ofEpochSecond(epochSecond, 0, ZoneOffset.UTC)
  }

  val upsertSourceCurrencyData = List(
    changelogRow("+U", "Euro", "no1", JLong.valueOf(114L), localDateTime(1L)),
    changelogRow("+U", "US Dollar", "no1", JLong.valueOf(102L), localDateTime(2L)),
    changelogRow("+U", "Yen", "no1", JLong.valueOf(1L), localDateTime(3L)),
    changelogRow("+U", "RMB", "no1", JLong.valueOf(702L), localDateTime(4L)),
    changelogRow("+U", "Euro",  "no1", JLong.valueOf(118L), localDateTime(6L)),
    changelogRow("+U", "US Dollar", "no1", JLong.valueOf(104L), localDateTime(4L)),
    changelogRow("-D", "RMB", "no1", JLong.valueOf(702L), localDateTime(4L)))

  val upsertSourceDataId = registerData(upsertSourceCurrencyData)
  tEnv.executeSql(
    s"""
       |CREATE TABLE upsert_currency (
       |  currency STRING,
       |  currency_no STRING,
       |  rate  BIGINT,
       |  currency_time TIMESTAMP(3),
       |  WATERMARK FOR currency_time AS currency_time - interval '5' SECOND,
       |  PRIMARY KEY(currency) NOT ENFORCED
       |) WITH (
       |  'connector' = 'values',
       |  'changelog-mode' = 'UA,D',
       |  'data-id' = '$upsertSourceDataId'
       |)
       |""".stripMargin)
  val sql =
    """
      |SELECT
      |TUMBLE_START(currency_time, INTERVAL '5' SECOND) as w_start,
      |TUMBLE_END(currency_time, INTERVAL '5' SECOND) as w_end,
      |MAX(rate) AS max_rate
      |FROM upsert_currency
      |GROUP BY TUMBLE(currency_time, INTERVAL '5' SECOND)
      |""".stripMargin
  val sink = new TestingAppendSink
  tEnv.sqlQuery(sql).toAppendStream[Row].addSink(sink)
  env.execute()
}
{code}
I add. the above ITCase for window aggregate process upsert input stream when resolved [FLINK-20487|https://issues.apache.org/jira/browse/FLINK-20487],  an `ArrayIndexOutOfBoundsException` is thrown out when apply `WatermarkAssignerChangelogNormalizeTransposeRule`, the detail information is as following,
{code:java}
java.lang.ArrayIndexOutOfBoundsException: 2java.lang.ArrayIndexOutOfBoundsException: 2
 at com.google.common.collect.RegularImmutableList.get(RegularImmutableList.java:75) at org.apache.calcite.util.Util$TransformingList.get(Util.java:2732) at org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalExchange$$anonfun$2.apply(CommonPhysicalExchange.scala:108) at org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalExchange$$anonfun$2.apply(CommonPhysicalExchange.scala:108) 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.nodes.physical.common.CommonPhysicalExchange.distributionToString(CommonPhysicalExchange.scala:108) at org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalExchange.explainTerms(CommonPhysicalExchange.scala:94) at org.apache.calcite.rel.AbstractRelNode.getDigestItems(AbstractRelNode.java:409) at org.apache.calcite.rel.AbstractRelNode.deepHashCode(AbstractRelNode.java:391) at org.apache.calcite.rel.AbstractRelNode$InnerRelDigest.hashCode(AbstractRelNode.java:443) at java.util.HashMap.hash(HashMap.java:339) at java.util.HashMap.get(HashMap.java:557) at org.apache.calcite.plan.hep.HepPlanner.addRelToGraph(HepPlanner.java:815) at org.apache.calcite.plan.hep.HepPlanner.addRelToGraph(HepPlanner.java:799) at org.apache.calcite.plan.hep.HepPlanner.applyTransformationResults(HepPlanner.java:734) at org.apache.calcite.plan.hep.HepPlanner.applyRule(HepPlanner.java:545) 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.FlinkGroupProgram$$anonfun$optimize$1$$anonfun$apply$1.apply(FlinkGroupProgram.scala:63) at org.apache.flink.table.planner.plan.optimize.program.FlinkGroupProgram$$anonfun$optimize$1$$anonfun$apply$1.apply(FlinkGroupProgram.scala:60) 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.FlinkGroupProgram$$anonfun$optimize$1.apply(FlinkGroupProgram.scala:60) at org.apache.flink.table.planner.plan.optimize.program.FlinkGroupProgram$$anonfun$optimize$1.apply(FlinkGroupProgram.scala:55) 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.immutable.Range.foreach(Range.scala:160) 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.FlinkGroupProgram.optimize(FlinkGroupProgram.scala:55) 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.StreamCommonSubGraphBasedOptimizer.optimizeTree(StreamCommonSubGraphBasedOptimizer.scala:163) at org.apache.flink.table.planner.plan.optimize.StreamCommonSubGraphBasedOptimizer.doOptimize(StreamCommonSubGraphBasedOptimizer.scala:79) at org.apache.flink.table.planner.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:77) at org.apache.flink.table.planner.delegation.PlannerBase.optimize(PlannerBase.scala:279) at org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:163) at org.apache.flink.table.api.bridge.scala.internal.StreamTableEnvironmentImpl.toStreamInternal(StreamTableEnvironmentImpl.scala:291) at org.apache.flink.table.api.bridge.scala.internal.StreamTableEnvironmentImpl.toAppendStream(StreamTableEnvironmentImpl.scala:325) at org.apache.flink.table.api.bridge.scala.TableConversions.toAppendStream(TableConversions.scala:78) at org.apache.flink.table.planner.runtime.stream.sql.GroupWindowITCase.testUnResolvedWindowAggregateOnUpsertSource(GroupWindowITCase.scala:470) 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.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:239) at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48) at org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45) at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55) 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.runners.Suite.runChild(Suite.java:128) at org.junit.runners.Suite.runChild(Suite.java:27) 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.rules.ExternalResource$1.evaluate(ExternalResource.java:48) at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48) at org.junit.rules.RunRules.evaluate(RunRules.java:20) 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}
The root cause is after transpose `Calc` and `Exchange` in the following `RelNode` Tree, the distribution keys of `Exchange` should be adjusted because the `Calc` already projects unuseless columns.

!image-2021-05-17-16-46-00-789.png!



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