hive-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "David Mollitor (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HIVE-21748) HBase Operations Can Fail When Using MAPREDLOCAL
Date Fri, 17 May 2019 23:52:00 GMT

    [ https://issues.apache.org/jira/browse/HIVE-21748?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16842811#comment-16842811
] 

David Mollitor commented on HIVE-21748:
---------------------------------------

OK.  I figured it out.

When performing a query which launches a MAPREDLOCAL, that MAPREDLOCAL job uses the Hive service
user's kerberos ticket cache when attempting to communicate with HBase.  This is not proper.

* In the case of {{Failed to find any Kerberos tgt}}, I had an expired tgt in my ticket cache
* In the case of {{Insufficient permissions for user 'yarn'}}, I had mistakenly put a tgt
from the yarn principal in the 'hive' user ticket cache: {{sudo -u hive kinit -kt yarn.keytab
yarn}}

The MAPREDLOCAL process should be passed an HBase delegation (auth) token from the HS2 service
principal and should not be relying on the kerberos ticket cache.  The Hive service does not
use the ticket cache, instead caching the tgt in JVM memory, therefore, there is no expectation
that there should be a valid tgt in the ticket cache.

> HBase Operations Can Fail When Using MAPREDLOCAL
> ------------------------------------------------
>
>                 Key: HIVE-21748
>                 URL: https://issues.apache.org/jira/browse/HIVE-21748
>             Project: Hive
>          Issue Type: Bug
>          Components: HBase Handler
>    Affects Versions: 4.0.0, 3.2.0
>            Reporter: David Mollitor
>            Priority: Major
>         Attachments: HBaseMapredLocalExplain.txt
>
>
> https://github.com/apache/hive/blob/5634140b2beacdac20ceec8c73ff36bce5675ef8/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java#L258-L262
> {code:java|title=HBaseStorageHandler.java}
>     if (this.configureInputJobProps) {
>       LOG.info("Configuring input job properties");
> ...
>       try {
>         addHBaseDelegationToken(jobConf);
>       } catch (IOException | MetaException e) {
>         throw new IllegalStateException("Error while configuring input job properties",
e);
>       }
>    }
>   else {
>     LOG.info("Configuring output job properties");
>     ...
>   }
> {code}
> What we can see here is that the HBase Delegation Token is only created when there is
an input job (reading from an HBase source).  For a particular stage of a query, if there
is no HBASE input, only HBASE output, then the delegation token is not created and may cause
a failure if the subsequent MapReduceLocal is unable to connect to HBase without the token.
> {code:none|title=Error Message in HS2 Log}
> 2019-05-17 10:24:55,036 ERROR org.apache.hive.service.cli.operation.Operation: [HiveServer2-Background-Pool:
Thread-388]: Error running hive query:
> org.apache.hive.service.cli.HiveSQLException: Error while processing statement: FAILED:
Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask
>         at org.apache.hive.service.cli.operation.Operation.toSQLException(Operation.java:400)
>         at org.apache.hive.service.cli.operation.SQLOperation.runQuery(SQLOperation.java:238)
>         at org.apache.hive.service.cli.operation.SQLOperation.access$300(SQLOperation.java:89)
>         at org.apache.hive.service.cli.operation.SQLOperation$3$1.run(SQLOperation.java:301)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at javax.security.auth.Subject.doAs(Subject.java:415)
>         at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1924)
>         at org.apache.hive.service.cli.operation.SQLOperation$3.run(SQLOperation.java:314)
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>         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)
> {code}
> You can tell it will fail because an HDFS Token will be created, but it will not report
an HBASE token in the HS2 logs.  The following is an example of a proper setup.  If it is
missing the HBASE_AUTH_TOKEN it will probably fail..
> {code:none|title=Logging of a Proper Run}
> 2019-05-17 10:36:15,593 INFO  org.apache.hadoop.mapreduce.JobSubmitter: [HiveServer2-Background-Pool:
Thread-455]: Submitting tokens for job: job_1557858663665_0048
> 2019-05-17 10:36:15,593 INFO  org.apache.hadoop.mapreduce.JobSubmitter: [HiveServer2-Background-Pool:
Thread-455]: Kind: HDFS_DELEGATION_TOKEN, Service: 10.17.101.237:8020, Ident: (token for hive:
HDFS_DELEGATION_TOKEN owner=hive/host-10-17-102-135.coe.cloudera.com@EXAMPLE.COM, renewer=yarn,
realUser=, issueDate=1558114574357, maxDate=1558719374357, sequenceNumber=75, masterKeyId=4)
> 2019-05-17 10:36:15,593 INFO  org.apache.hadoop.mapreduce.JobSubmitter: [HiveServer2-Background-Pool:
Thread-455]: Kind: HBASE_AUTH_TOKEN, Service: 9b282733-7927-4785-92ea-dad419f6f055, Ident:
(org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier@b1)
> 2019-05-17 10:36:15,859 INFO  org.apache.hadoop.yarn.client.api.impl.YarnClientImpl:
[HiveServer2-Background-Pool: Thread-455]: Submitted application application_1557858663665_0048
> {code}
> Error message in the Local MapReduce log.
> {code:none|title=Error message}
> Caused by: org.apache.hadoop.hbase.ipc.RemoteWithExtrasException(org.apache.hadoop.hbase.security.AccessDeniedException):
org.apache.hadoop.hbase.security.AccessDeniedException: Insufficient permissions for user
'xxxx' (table=xyz, action=READ)
>         at org.apache.hadoop.hbase.security.access.AccessController.internalPreRead(AccessController.java:1611)
>         at org.apache.hadoop.hbase.security.access.AccessController.preScannerOpen(AccessController.java:2080)
>         at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$50.call(RegionCoprocessorHost.java:1300)
>         at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$RegionOperation.call(RegionCoprocessorHos
> or perhaps...
> 2019-05-10 07:43:24,875 WARN  [htable-pool2-t1]: security.UserGroupInformation (UserGroupInformation.java:doAs(1927))
- PriviledgedActionException as:hive (auth:KERBEROS) cause:javax.security.sasl.SaslException:
GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level:
Failed to find any Kerberos tgt)]
> 2019-05-10 07:43:24,876 WARN  [htable-pool2-t1]: ipc.RpcClientImpl (RpcClientImpl.java:run(675))
- Exception encountered while connecting to the server : javax.security.sasl.SaslException:
GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level:
Failed to find any Kerberos tgt)]
> 2019-05-10 07:43:24,876 ERROR [htable-pool2-t1]: ipc.RpcClientImpl (RpcClientImpl.java:run(685))
- SASL authentication failed. The most likely cause is missing or invalid credentials. Consider
'kinit'.
> javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid
credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
> 	at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:211)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message