Timo Walther created FLINK-7698: ----------------------------------- Summary: Join with null literals leads to NPE Key: FLINK-7698 URL: https://issues.apache.org/jira/browse/FLINK-7698 Project: Flink Issue Type: Bug Components: Table API & SQL Reporter: Timo Walther
The following query fails: {code} @Test def testProcessTimeInnerJoin(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) env.setStateBackend(getStateBackend) StreamITCase.clear env.setParallelism(1) val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on t1.a = t2.a and t1.nullField = t2.nullField and " + "t1.proctime between t2.proctime - interval '5' second and t2.proctime + interval '5' second" val data1 = new mutable.MutableList[(Int, Long, String)] data1.+=((1, 1L, "Hi1")) data1.+=((1, 2L, "Hi2")) data1.+=((1, 5L, "Hi3")) data1.+=((2, 7L, "Hi5")) data1.+=((1, 9L, "Hi6")) data1.+=((1, 8L, "Hi8")) data1.+=((1, 8L, "Hi8")) val data2 = new mutable.MutableList[(Int, Long, String)] data2.+=((1, 1L, "HiHi")) data2.+=((2, 2L, "HeHe")) val t1 = env.fromCollection(data1).toTable(tEnv, 'a, 'b, 'c, 'proctime.proctime) .select('a, 'b, 'c, 'proctime, Null(Types.LONG) as 'nullField) val t2 = env.fromCollection(data2).toTable(tEnv, 'a, 'b, 'c, 'proctime.proctime) .select('a, 'b, 'c, 'proctime, 12L as 'nullField) tEnv.registerTable("T1", t1) tEnv.registerTable("T2", t2) val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] result.addSink(new StreamITCase.StringSink[Row]) env.execute() } {code} It leads to: {code} java.lang.NullPointerException at org.apache.calcite.rex.RexUtil.gatherConstraint(RexUtil.java:437) at org.apache.calcite.rex.RexUtil.gatherConstraints(RexUtil.java:399) at org.apache.calcite.rex.RexUtil.predicateConstants(RexUtil.java:336) at org.apache.calcite.plan.RelOptPredicateList.of(RelOptPredicateList.java:144) at org.apache.calcite.rel.metadata.RelMdPredicates$JoinConditionBasedPredicateInference.inferPredicates(RelMdPredicates.java:654) at org.apache.calcite.rel.metadata.RelMdPredicates.getPredicates(RelMdPredicates.java:326) at GeneratedMetadataHandler_Predicates.getPredicates_$(Unknown Source) at GeneratedMetadataHandler_Predicates.getPredicates(Unknown Source) at GeneratedMetadataHandler_Predicates.getPredicates_$(Unknown Source) at GeneratedMetadataHandler_Predicates.getPredicates(Unknown Source) at org.apache.calcite.rel.metadata.RelMetadataQuery.getPulledUpPredicates(RelMetadataQuery.java:803) at org.apache.calcite.rel.rules.ReduceExpressionsRule$ProjectReduceExpressionsRule.onMatch(ReduceExpressionsRule.java:264) at org.apache.calcite.plan.AbstractRelOptPlanner.fireRule(AbstractRelOptPlanner.java:317) at org.apache.calcite.plan.hep.HepPlanner.applyRule(HepPlanner.java:506) at org.apache.calcite.plan.hep.HepPlanner.applyRules(HepPlanner.java:385) at org.apache.calcite.plan.hep.HepPlanner.executeInstruction(HepPlanner.java:251) at org.apache.calcite.plan.hep.HepInstruction$RuleInstance.execute(HepInstruction.java:125) at org.apache.calcite.plan.hep.HepPlanner.executeProgram(HepPlanner.java:210) at org.apache.calcite.plan.hep.HepPlanner.findBestExp(HepPlanner.java:197) at org.apache.flink.table.api.TableEnvironment.runHepPlanner(TableEnvironment.scala:257) at org.apache.flink.table.api.StreamTableEnvironment.optimize(StreamTableEnvironment.scala:663) at org.apache.flink.table.api.StreamTableEnvironment.translate(StreamTableEnvironment.scala:728) at org.apache.flink.table.api.scala.StreamTableEnvironment.toAppendStream(StreamTableEnvironment.scala:219) at org.apache.flink.table.api.scala.StreamTableEnvironment.toAppendStream(StreamTableEnvironment.scala:195) at org.apache.flink.table.api.scala.TableConversions.toAppendStream(TableConversions.scala:121) at org.apache.flink.table.runtime.stream.sql.JoinITCase.testProcessTimeInnerJoin(JoinITCase.scala:67) 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.rules.ExternalResource$1.evaluate(ExternalResource.java:48) 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.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) 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.execution.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:51) at com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:242) at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:70) {code} Seems to be a Calcite bug, but we have to investigate this first. Replacing {{Null(Types.LONG)}} with a value works. -- This message was sent by Atlassian JIRA (v6.4.14#64029)