Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-27741

Fix NPE when use dense_rank() and rank() in over aggregation

    XMLWordPrintableJSON

Details

    Description

      There has an 'NullPointException' when use RANK() and DENSE_RANK() in over window.

      @Test
        def testDenseRankOnOver(): Unit = {
          val t = failingDataSource(TestData.tupleData5)
            .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime.proctime)
          tEnv.registerTable("MyTable", t)
          val sqlQuery = "SELECT a, DENSE_RANK() OVER (PARTITION BY a ORDER BY proctime) FROM MyTable"
      
          val sink = new TestingAppendSink
          tEnv.sqlQuery(sqlQuery).toAppendStream[Row].addSink(sink)
          env.execute()
        }
      
      @Test
        def testRankOnOver(): Unit = {
          val t = failingDataSource(TestData.tupleData5)
            .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime.proctime)
          tEnv.registerTable("MyTable", t)
          val sqlQuery = "SELECT a, RANK() OVER (PARTITION BY a ORDER BY proctime) FROM MyTable"
      
          val sink = new TestingAppendSink
          tEnv.sqlQuery(sqlQuery).toAppendStream[Row].addSink(sink)
          env.execute()
        }
      

      Exception Info:

      java.lang.NullPointerException
      	at scala.collection.mutable.ArrayOps$ofInt$.length$extension(ArrayOps.scala:248)
      	at scala.collection.mutable.ArrayOps$ofInt.length(ArrayOps.scala:248)
      	at scala.collection.SeqLike.size(SeqLike.scala:104)
      	at scala.collection.SeqLike.size$(SeqLike.scala:104)
      	at scala.collection.mutable.ArrayOps$ofInt.size(ArrayOps.scala:242)
      	at scala.collection.IndexedSeqLike.sizeHintIfCheap(IndexedSeqLike.scala:95)
      	at scala.collection.IndexedSeqLike.sizeHintIfCheap$(IndexedSeqLike.scala:95)
      	at scala.collection.mutable.ArrayOps$ofInt.sizeHintIfCheap(ArrayOps.scala:242)
      	at scala.collection.mutable.Builder.sizeHint(Builder.scala:77)
      	at scala.collection.mutable.Builder.sizeHint$(Builder.scala:76)
      	at scala.collection.mutable.ArrayBuilder.sizeHint(ArrayBuilder.scala:21)
      	at scala.collection.TraversableLike.builder$1(TraversableLike.scala:229)
      	at scala.collection.TraversableLike.map(TraversableLike.scala:232)
      	at scala.collection.TraversableLike.map$(TraversableLike.scala:226)
      	at scala.collection.mutable.ArrayOps$ofInt.map(ArrayOps.scala:242)
      	at org.apache.flink.table.planner.plan.utils.AggFunctionFactory.createDenseRankAggFunction(AggFunctionFactory.scala:454)
      	at org.apache.flink.table.planner.plan.utils.AggFunctionFactory.createAggFunction(AggFunctionFactory.scala:94)
      	at org.apache.flink.table.planner.plan.utils.AggregateUtil$.$anonfun$transformToAggregateInfoList$1(AggregateUtil.scala:445)
      	at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:233)
      	at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:58)
      	at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:51)
      	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
      	at scala.collection.TraversableLike.map(TraversableLike.scala:233)
      	at scala.collection.TraversableLike.map$(TraversableLike.scala:226)
      	at scala.collection.AbstractTraversable.map(Traversable.scala:104)
      	at org.apache.flink.table.planner.plan.utils.AggregateUtil$.transformToAggregateInfoList(AggregateUtil.scala:435)
      	at org.apache.flink.table.planner.plan.utils.AggregateUtil$.transformToStreamAggregateInfoList(AggregateUtil.scala:381)
      	at org.apache.flink.table.planner.plan.utils.AggregateUtil$.transformToStreamAggregateInfoList(AggregateUtil.scala:361)
      	at org.apache.flink.table.planner.plan.utils.AggregateUtil.transformToStreamAggregateInfoList(AggregateUtil.scala)
      	at org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecOverAggregate.createUnboundedOverProcessFunction(StreamExecOverAggregate.java:279)
      	at org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecOverAggregate.translateToPlanInternal(StreamExecOverAggregate.java:198)
      	at org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:148)
      	at org.apache.flink.table.planner.plan.nodes.exec.ExecEdge.translateToPlan(ExecEdge.java:249)
      	at org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecCalc.translateToPlanInternal(CommonExecCalc.java:94)
      	at org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:148)
      	at org.apache.flink.table.planner.plan.nodes.exec.ExecEdge.translateToPlan(ExecEdge.java:249)
      	at org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecLegacySink.translateToTransformation(CommonExecLegacySink.java:185)
      	at org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecLegacySink.translateToPlanInternal(CommonExecLegacySink.java:154)
      	at org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:148)
      	at org.apache.flink.table.planner.delegation.StreamPlanner.$anonfun$translateToPlan$1(StreamPlanner.scala:79)
      	at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:233)
      	at scala.collection.Iterator.foreach(Iterator.scala:937)
      	at scala.collection.Iterator.foreach$(Iterator.scala:937)
      	at scala.collection.AbstractIterator.foreach(Iterator.scala:1425)
      	at scala.collection.IterableLike.foreach(IterableLike.scala:70)
      	at scala.collection.IterableLike.foreach$(IterableLike.scala:69)
      	at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
      	at scala.collection.TraversableLike.map(TraversableLike.scala:233)
      	at scala.collection.TraversableLike.map$(TraversableLike.scala:226)
      	at scala.collection.AbstractTraversable.map(Traversable.scala:104)
      	at org.apache.flink.table.planner.delegation.StreamPlanner.translateToPlan(StreamPlanner.scala:78)
      	at org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:181)
      	at org.apache.flink.table.api.bridge.internal.AbstractStreamTableEnvironmentImpl.toStreamInternal(AbstractStreamTableEnvironmentImpl.java:223)
      	at org.apache.flink.table.api.bridge.scala.internal.StreamTableEnvironmentImpl.toAppendStream(StreamTableEnvironmentImpl.scala:220)
      	at org.apache.flink.table.api.bridge.scala.TableConversions.toAppendStream(TableConversions.scala:199)
      	at org.apache.flink.table.planner.runtime.stream.sql.OverAggregateITCase.testDenseRankOnOver(OverAggregateITCase.scala:170)
      	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.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:258)
      	at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
      	at org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45)
      

      Attachments

        Issue Links

          Activity

            People

              chenzihao chenzihao
              chenzihao chenzihao
              Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: