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

Not able to add partition for views in hive when sentry is enabled

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • None
    • 3.0.0
    • Parser
    • None

    Description

      Repro:
      create table tesnit (a int) partitioned by (p int);
      insert into table tesnit partition (p = 1) values (1);
      insert into table tesnit partition (p = 2) values (1);
      create view test_view partitioned on (p) as select * from tesnit where p =1;

      alter view test_view add partition (p = 2);
      Error: Error while compiling statement: FAILED: SemanticException [Error 10056]: The query does not reference any valid partition. To run this query, set hive.mapred.mode=nonstrict (state=42000,code=10056)

      Attachments

        1. HIVE-16660.1.patch
          2 kB
          Yongzhi Chen

        Activity

          ychena Yongzhi Chen added a comment -

          The error is because hive fail to compile internal query generated for the alter operation:
          SELECT * FROM `default.test_view` WHERE (`p` = '2'); caused NPE when
          sentry enabled:

          HiveServer2-Handler-Pool: Thread-68]: FAILED: NullPointerException null
          java.lang.NullPointerException
          	at com.google.common.base.Preconditions.checkNotNull(Preconditions.java:187)
          	at com.google.common.cache.LocalCache.get(LocalCache.java:3964)
          	at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3969)
          	at com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4829)
          	at org.apache.hadoop.security.Groups.getGroups(Groups.java:215)
          	at org.apache.sentry.provider.common.HadoopGroupMappingService.getGroups(HadoopGroupMappingService.java:60)
          	at org.apache.sentry.binding.hive.authz.HiveAuthzBinding.getGroups(HiveAuthzBinding.java:372)
          	at org.apache.sentry.binding.hive.HiveAuthzBindingHook.postAnalyze(HiveAuthzBindingHook.java:395)
          	at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:449)
          	at org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer.analyzeAlterTableAddParts(DDLSemanticAnalyzer.java:2819)
          	at org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer.analyzeInternal(DDLSemanticAnalyzer.java:385)
          	at org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.analyze(BaseSemanticAnalyzer.java:223)
          	at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:446)
          	at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:312)
          	at org.apache.hadoop.hive.ql.Driver.compileInternal(Driver.java:1201)
          	at org.apache.hadoop.hive.ql.Driver.compileAndRespond(Driver.java:1188)
          	at org.apache.hive.service.cli.operation.SQLOperation.prepare(SQLOperation.java:134)
          	at org.apache.hive.service.cli.operation.SQLOperation.runInternal(SQLOperation.java:206)
          	at org.apache.hive.service.cli.operation.Operation.run(Operation.java:327)
          	at org.apache.hive.service.cli.session.HiveSessionImpl.executeStatementInternal(HiveSessionImpl.java:424)
          	at org.apache.hive.service.cli.session.HiveSessionImpl.executeStatementAsync(HiveSessionImpl.java:401)
          	at org.apache.hive.service.cli.CLIService.executeStatementAsync(CLIService.java:258)
          	at org.apache.hive.service.cli.thrift.ThriftCLIService.ExecuteStatement(ThriftCLIService.java:500)
          	at org.apache.hive.service.cli.thrift.TCLIService$Processor$ExecuteStatement.getResult(TCLIService.java:1313)
          	at org.apache.hive.service.cli.thrift.TCLIService$Processor$ExecuteStatement.getResult(TCLIService.java:1298)
          	at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
          	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
          	at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor.process(HadoopThriftAuthBridge.java:746)
          	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:286)
          	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
          	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
          	at java.lang.Thread.run(Thread.java:745)
          

          The NPE is because of null userName for the newly created Driver object for the query.
          The patch fix the issue by adding username information. Also fix the NPE when some user applicate call get partition path information from a view. The NPE like:

          Caused by: java.lang.NullPointerException
          	at org.apache.hadoop.hive.ql.metadata.Partition.getDataLocation(Partition.java:241)
          	at org.apache.hadoop.hive.ql.metadata.Partition.getPartitionPath(Partition.java:236)
          	....
          	at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:1648)
          	at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1318)
          	at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1127)
          	at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1120)
          	at org.apache.hive.service.cli.operation.SQLOperation.runQuery(SQLOperation.java:178)
          	... 11 more
          
          ychena Yongzhi Chen added a comment - The error is because hive fail to compile internal query generated for the alter operation: SELECT * FROM `default.test_view` WHERE (`p` = '2'); caused NPE when sentry enabled: HiveServer2-Handler-Pool: Thread-68]: FAILED: NullPointerException null java.lang.NullPointerException at com.google.common.base.Preconditions.checkNotNull(Preconditions.java:187) at com.google.common.cache.LocalCache.get(LocalCache.java:3964) at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3969) at com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4829) at org.apache.hadoop.security.Groups.getGroups(Groups.java:215) at org.apache.sentry.provider.common.HadoopGroupMappingService.getGroups(HadoopGroupMappingService.java:60) at org.apache.sentry.binding.hive.authz.HiveAuthzBinding.getGroups(HiveAuthzBinding.java:372) at org.apache.sentry.binding.hive.HiveAuthzBindingHook.postAnalyze(HiveAuthzBindingHook.java:395) at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:449) at org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer.analyzeAlterTableAddParts(DDLSemanticAnalyzer.java:2819) at org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer.analyzeInternal(DDLSemanticAnalyzer.java:385) at org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.analyze(BaseSemanticAnalyzer.java:223) at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:446) at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:312) at org.apache.hadoop.hive.ql.Driver.compileInternal(Driver.java:1201) at org.apache.hadoop.hive.ql.Driver.compileAndRespond(Driver.java:1188) at org.apache.hive.service.cli.operation.SQLOperation.prepare(SQLOperation.java:134) at org.apache.hive.service.cli.operation.SQLOperation.runInternal(SQLOperation.java:206) at org.apache.hive.service.cli.operation.Operation.run(Operation.java:327) at org.apache.hive.service.cli.session.HiveSessionImpl.executeStatementInternal(HiveSessionImpl.java:424) at org.apache.hive.service.cli.session.HiveSessionImpl.executeStatementAsync(HiveSessionImpl.java:401) at org.apache.hive.service.cli.CLIService.executeStatementAsync(CLIService.java:258) at org.apache.hive.service.cli.thrift.ThriftCLIService.ExecuteStatement(ThriftCLIService.java:500) at org.apache.hive.service.cli.thrift.TCLIService$Processor$ExecuteStatement.getResult(TCLIService.java:1313) at org.apache.hive.service.cli.thrift.TCLIService$Processor$ExecuteStatement.getResult(TCLIService.java:1298) at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor.process(HadoopThriftAuthBridge.java:746) at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:286) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) The NPE is because of null userName for the newly created Driver object for the query. The patch fix the issue by adding username information. Also fix the NPE when some user applicate call get partition path information from a view. The NPE like: Caused by: java.lang.NullPointerException at org.apache.hadoop.hive.ql.metadata.Partition.getDataLocation(Partition.java:241) at org.apache.hadoop.hive.ql.metadata.Partition.getPartitionPath(Partition.java:236) .... at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:1648) at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1318) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1127) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1120) at org.apache.hive.service.cli.operation.SQLOperation.runQuery(SQLOperation.java:178) ... 11 more
          ychena Yongzhi Chen added a comment -

          Need code review.

          ychena Yongzhi Chen added a comment - Need code review.
          hiveqa Hive QA added a comment -

          Here are the results of testing the latest attachment:
          https://issues.apache.org/jira/secure/attachment/12867782/HIVE-16660.1.patch

          ERROR: -1 due to no test(s) being added or modified.

          ERROR: -1 due to 2 failed/errored test(s), 10698 tests executed
          Failed tests:

          org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver[explainanalyze_3] (batchId=97)
          org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver[explainuser_3] (batchId=97)
          

          Test results: https://builds.apache.org/job/PreCommit-HIVE-Build/5234/testReport
          Console output: https://builds.apache.org/job/PreCommit-HIVE-Build/5234/console
          Test logs: http://104.198.109.242/logs/PreCommit-HIVE-Build-5234/

          Messages:

          Executing org.apache.hive.ptest.execution.TestCheckPhase
          Executing org.apache.hive.ptest.execution.PrepPhase
          Executing org.apache.hive.ptest.execution.ExecutionPhase
          Executing org.apache.hive.ptest.execution.ReportingPhase
          Tests exited with: TestsFailedException: 2 tests failed
          

          This message is automatically generated.

          ATTACHMENT ID: 12867782 - PreCommit-HIVE-Build

          hiveqa Hive QA added a comment - Here are the results of testing the latest attachment: https://issues.apache.org/jira/secure/attachment/12867782/HIVE-16660.1.patch ERROR: -1 due to no test(s) being added or modified. ERROR: -1 due to 2 failed/errored test(s), 10698 tests executed Failed tests: org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver[explainanalyze_3] (batchId=97) org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver[explainuser_3] (batchId=97) Test results: https://builds.apache.org/job/PreCommit-HIVE-Build/5234/testReport Console output: https://builds.apache.org/job/PreCommit-HIVE-Build/5234/console Test logs: http://104.198.109.242/logs/PreCommit-HIVE-Build-5234/ Messages: Executing org.apache.hive.ptest.execution.TestCheckPhase Executing org.apache.hive.ptest.execution.PrepPhase Executing org.apache.hive.ptest.execution.ExecutionPhase Executing org.apache.hive.ptest.execution.ReportingPhase Tests exited with: TestsFailedException: 2 tests failed This message is automatically generated. ATTACHMENT ID: 12867782 - PreCommit-HIVE-Build
          ychena Yongzhi Chen added a comment -

          The two failures are not related. They are known random failure tests.

          ychena Yongzhi Chen added a comment - The two failures are not related. They are known random failure tests.
          Yibing Yibing Shi added a comment -

          ychena, should we solve these 2 problems in 2 different JIRAs? They are not related.

          Yibing Yibing Shi added a comment - ychena , should we solve these 2 problems in 2 different JIRAs? They are not related.
          ychena Yongzhi Chen added a comment -

          Yibing, they are all related to partitions to the view. For example, If navigator hook is attached to HS2, add partition operation(after partition added) will trigger second issue. So in hue, user will see NPE even the first issue is solved.

          ychena Yongzhi Chen added a comment - Yibing , they are all related to partitions to the view. For example, If navigator hook is attached to HS2, add partition operation(after partition added) will trigger second issue. So in hue, user will see NPE even the first issue is solved.
          aihuaxu Aihua Xu added a comment -

          +1.

          aihuaxu Aihua Xu added a comment - +1.
          ychena Yongzhi Chen added a comment -

          Push to master. Thanks aihuaxu for reviewing the code.

          ychena Yongzhi Chen added a comment - Push to master. Thanks aihuaxu for reviewing the code.
          vgarg Vineet Garg added a comment -

          Hive 3.0.0 has been released so closing this jira.

          vgarg Vineet Garg added a comment - Hive 3.0.0 has been released so closing this jira.

          People

            ychena Yongzhi Chen
            ychena Yongzhi Chen
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: