Uploaded image for project: 'Tajo (Retired)'
  1. Tajo (Retired)
  2. TAJO-1552

NPE occurs when GreedyHeuristicJoinOrderAlgorithm.getCost() returns infinity.

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Minor
    • Resolution: Fixed
    • None
    • None
    • None
    • None

    Description

      I tested TPC-DS Query28 but NPE occurs in GreedyHeuristicJoinOrderAlgorithm.

      select *
      from (select avg(ss_list_price) B1_LP
                  ,count(ss_list_price) B1_CNT
                  ,count(distinct ss_list_price) B1_CNTD
            from store_sales
            where ss_quantity between 0 and 5
              and (ss_list_price between 11 and 11+10
                   or ss_coupon_amt between 460 and 460+1000
                   or ss_wholesale_cost between 14 and 14+20)) B1,
           (select avg(ss_list_price) B2_LP
                  ,count(ss_list_price) B2_CNT
                  ,count(distinct ss_list_price) B2_CNTD
            from store_sales
            where ss_quantity between 6 and 10
              and (ss_list_price between 91 and 91+10
                or ss_coupon_amt between 1430 and 1430+1000
                or ss_wholesale_cost between 32 and 32+20)) B2,
           (select avg(ss_list_price) B3_LP
                  ,count(ss_list_price) B3_CNT
                  ,count(distinct ss_list_price) B3_CNTD
            from store_sales
            where ss_quantity between 11 and 15
              and (ss_list_price between 66 and 66+10
                or ss_coupon_amt between 920 and 920+1000
                or ss_wholesale_cost between 4 and 4+20)) B3,
           (select avg(ss_list_price) B4_LP
                  ,count(ss_list_price) B4_CNT
                  ,count(distinct ss_list_price) B4_CNTD
            from store_sales
            where ss_quantity between 16 and 20
              and (ss_list_price between 142 and 142+10
                or ss_coupon_amt between 3054 and 3054+1000
                or ss_wholesale_cost between 80 and 80+20)) B4,
           (select avg(ss_list_price) B5_LP
                  ,count(ss_list_price) B5_CNT
                  ,count(distinct ss_list_price) B5_CNTD
            from store_sales
            where ss_quantity between 21 and 25
              and (ss_list_price between 135 and 135+10
                or ss_coupon_amt between 14180 and 14180+1000
                or ss_wholesale_cost between 38 and 38+20)) B5,
           (select avg(ss_list_price) B6_LP
                  ,count(ss_list_price) B6_CNT
                  ,count(distinct ss_list_price) B6_CNTD
            from store_sales
            where ss_quantity between 26 and 30
              and (ss_list_price between 28 and 28+10
                or ss_coupon_amt between 2513 and 2513+1000
                or ss_wholesale_cost between 42 and 42+20)) B6
      limit 100;
      
      Stack Trace:
      java.lang.NullPointerException
      	at java.util.AbstractCollection.addAll(AbstractCollection.java:341)
      	at org.apache.tajo.plan.joinorder.GreedyHeuristicJoinOrderAlgorithm.getBestPair(GreedyHeuristicJoinOrderAlgorithm.java:222)
      	at org.apache.tajo.plan.joinorder.GreedyHeuristicJoinOrderAlgorithm.findBestOrder(GreedyHeuristicJoinOrderAlgorithm.java:109)
      	at org.apache.tajo.plan.LogicalOptimizer.optimizeJoinOrder(LogicalOptimizer.java:117)
      	at org.apache.tajo.plan.LogicalOptimizer.optimize(LogicalOptimizer.java:97)
      	at org.apache.tajo.master.GlobalEngine.createLogicalPlan(GlobalEngine.java:278)
      	at org.apache.tajo.master.GlobalEngine.executeQuery(GlobalEngine.java:188)
      	at org.apache.tajo.master.TajoMasterClientService$TajoMasterClientProtocolServiceHandler.submitQuery(TajoMasterClientService.java:287)
      	at org.apache.tajo.ipc.TajoMasterClientProtocol$TajoMasterClientProtocolService$2.callBlockingMethod(TajoMasterClientProtocol.java:543)
      	at org.apache.tajo.rpc.BlockingRpcServer$ServerHandler.channelRead(BlockingRpcServer.java:113)
      	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
      	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
      	at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
      	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
      	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
      	at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:182)
      	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
      	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
      	at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
      	at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:130)
      	at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
      	at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
      	at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
      	at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
      	at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116)
      	at java.lang.Thread.run(Thread.java:744)
      

      I researched GreedyHeuristicJoinOrderAlgorithm and found that NPE occurs when getCost() returns infinity value.

      Attachments

        Issue Links

          Activity

            People

              hjkim Hyoungjun Kim
              hjkim Hyoungjun Kim
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: