Uploaded image for project: 'Hive'
  1. Hive
  2. HIVE-24060

When the CBO is false, NPE is thrown by an EXCEPT or INTERSECT execution

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 3.1.0, 3.1.2
    • None
    • CBO, Hive
    • None

    Description

      set hive.cbo.enable=false;
      create table testtable(idx string, namex string) stored as orc;
      insert into testtable values('123', 'aaa'), ('234', 'bbb');
      explain select a.idx from (select idx,namex from testtable intersect select idx,namex from testtable) a
      

       The execution throws a NullPointException:

      2020-08-24 15:12:24,261 | WARN  | HiveServer2-Handler-Pool: Thread-345 | Error executing statement:  | org.apache.hive.service.cli.thrift.ThriftCLIService.executeNewStatement(ThriftCLIService.java:1155)
      org.apache.hive.service.cli.HiveSQLException: Error while compiling statement: FAILED: NullPointerException null
              at org.apache.hive.service.cli.operation.Operation.toSQLException(Operation.java:341) ~[hive-service-3.1.0.jar:3.1.0]
              at org.apache.hive.service.cli.operation.SQLOperation.prepare(SQLOperation.java:215) ~[hive-service-3.1.0.jar:3.1.0]
              at org.apache.hive.service.cli.operation.SQLOperation.runInternal(SQLOperation.java:316) ~[hive-service-3.1.0.jar:3.1.0]
              at org.apache.hive.service.cli.operation.Operation.run(Operation.java:253) ~[hive-service-3.1.0.jar:3.1.0]
              at org.apache.hive.service.cli.session.HiveSessionImpl.executeStatementInternal(HiveSessionImpl.java:684) ~[hive-service-3.1.0.jar:3.1.0]
              at org.apache.hive.service.cli.session.HiveSessionImpl.executeStatementAsync(HiveSessionImpl.java:670) ~[hive-service-3.1.0.jar:3.1.0]
              at org.apache.hive.service.cli.CLIService.executeStatementAsync(CLIService.java:342) ~[hive-service-3.1.0.jar:3.1.0]
              at org.apache.hive.service.cli.thrift.ThriftCLIService.executeNewStatement(ThriftCLIService.java:1144) ~[hive-service-3.1.0.jar:3.1.0]
              at org.apache.hive.service.cli.thrift.ThriftCLIService.ExecuteStatement(ThriftCLIService.java:1280) ~[hive-service-3.1.0.jar:3.1.0]
              at org.apache.hive.service.rpc.thrift.TCLIService$Processor$ExecuteStatement.getResult(TCLIService.java:1557) ~[hive-service-rpc-3.1.0.jar:3.1.0]
              at org.apache.hive.service.rpc.thrift.TCLIService$Processor$ExecuteStatement.getResult(TCLIService.java:1542) ~[hive-service-rpc-3.1.0.jar:3.1.0]
              at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) ~[libthrift-0.9.3.jar:0.9.3]
              at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) ~[libthrift-0.9.3.jar:0.9.3]
              at org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor.process(HadoopThriftAuthBridge.java:648) ~[hive-standalone-metastore-3.1.0.jar:3.1.0]
              at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:286) ~[libthrift-0.9.3.jar:0.9.3]
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_201]
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_201]
              at java.lang.Thread.run(Thread.java:748) [?:1.8.0_201]
      Caused by: java.lang.NullPointerException
              at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genSelectPlan(SemanticAnalyzer.java:4367) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genSelectPlan(SemanticAnalyzer.java:4346) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genPostGroupByBodyPlan(SemanticAnalyzer.java:10576) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genBodyPlan(SemanticAnalyzer.java:10515) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genPlan(SemanticAnalyzer.java:11434) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genPlan(SemanticAnalyzer.java:11291) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genPlan(SemanticAnalyzer.java:11318) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genPlan(SemanticAnalyzer.java:11304) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genOPTree(SemanticAnalyzer.java:12090) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.analyzeInternal(SemanticAnalyzer.java:12180) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.analyzeInternal(SemanticAnalyzer.java:11692) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.analyze(BaseSemanticAnalyzer.java:281) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.parse.ExplainSemanticAnalyzer.analyzeInternal(ExplainSemanticAnalyzer.java:164) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.analyze(BaseSemanticAnalyzer.java:281) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:712) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.Driver.compileInternal(Driver.java:2117) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.Driver.compileAndRespond(Driver.java:2064) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.Driver.compileAndRespond(Driver.java:2059) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hadoop.hive.ql.reexec.ReExecDriver.compileAndRespond(ReExecDriver.java:137) ~[hive-exec-3.1.0.jar:3.1.0]
              at org.apache.hive.service.cli.operation.SQLOperation.prepare(SQLOperation.java:204) ~[hive-service-3.1.0.jar:3.1.0]

      Attachments

        Activity

          People

            Unassigned Unassigned
            luguangming GuangMing Lu
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated: