hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Gary Helmling (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-9890) MR jobs are not working if started by a delegated user
Date Tue, 05 Nov 2013 22:11:18 GMT

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

Gary Helmling commented on HBASE-9890:
--------------------------------------

bq. I think this is a question of practicality – LoadIncrementalHFiles can only use the
SecureBulkLoadClient when the appropriate coprocessor is available on the RS. It's only available
when HBase security is enabled.

Whether {{hbase.security.authentication == "kerberos"}} has nothing to do with whether SecureBulkLoadEndpoint
is loaded on a table's regions.  The coprocessor needs to be configured independently (via
hbase.coprocessor.region.classes, hbase.coprocessor.user.region.classes, or directly on the
table).  It does also assume that the AccessController coprocessor is enabled, but that again
can be independent of authentication.

I may be missing something, but it seems like the main use of SecureBulkLoadEndpoint is to
move the bulk load HFiles to a staging directory, proxying to HDFS as the end user.  Even
the AccessController checks (which should only happen if AccessController is enabled), can
be done independently of whether HBase requires kerberos authentication (you can do access
control without kerberos auth).  So the secure bulk loading seems to me to only be required
when HDFS secure auth is enabled, and should be usable in that case regardless of the value
of hbase.security.authentication.

There is a bigger issue here, in that we are amassing a pile of security configurations that
are all exposed (and must be put together) by end users.  But I think that is solvable by
providing a simpler end user configuration, while still retaining the correct granularity
of configuration checks within the code itself. HBASE-4817 is a long standing issue to simplify
the end user configuration.

> MR jobs are not working if started by a delegated user
> ------------------------------------------------------
>
>                 Key: HBASE-9890
>                 URL: https://issues.apache.org/jira/browse/HBASE-9890
>             Project: HBase
>          Issue Type: Bug
>          Components: mapreduce, security
>    Affects Versions: 0.98.0, 0.94.12, 0.96.0
>            Reporter: Matteo Bertozzi
>            Assignee: Matteo Bertozzi
>             Fix For: 0.98.0, 0.94.13, 0.96.1
>
>         Attachments: HBASE-9890-94-v0.patch, HBASE-9890-v0.patch, HBASE-9890-v1.patch
>
>
> If Map-Reduce jobs are started with by a proxy user that has already the delegation tokens,
we get an exception on "obtain token" since the proxy user doesn't have the kerberos auth.
> For example:
>  * If we use oozie to execute RowCounter - oozie will get the tokens required (HBASE_AUTH_TOKEN)
and it will start the RowCounter. Once the RowCounter tries to obtain the token, it will get
an exception.
>  * If we use oozie to execute LoadIncrementalHFiles - oozie will get the tokens required
(HDFS_DELEGATION_TOKEN) and it will start the LoadIncrementalHFiles. Once the LoadIncrementalHFiles
tries to obtain the token, it will get an exception.
> {code}
>  org.apache.hadoop.hbase.security.AccessDeniedException: Token generation only allowed
for Kerberos authenticated clients
>     at org.apache.hadoop.hbase.security.token.TokenProvider.getAuthenticationToken(TokenProvider.java:87)
> {code}
> {code}
> org.apache.hadoop.ipc.RemoteException(java.io.IOException): Delegation Token can be issued
only with kerberos or web authentication
> 	at org.apache.hadoop.hdfs.DFSClient.getDelegationToken(DFSClient.java:783)
> 	at org.apache.hadoop.hdfs.DistributedFileSystem.getDelegationToken(DistributedFileSystem.java:868)
> 	at org.apache.hadoop.fs.FileSystem.collectDelegationTokens(FileSystem.java:509)
> 	at org.apache.hadoop.fs.FileSystem.addDelegationTokens(FileSystem.java:487)
> 	at org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodesInternal(TokenCache.java:130)
> 	at org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodesInternal(TokenCache.java:111)
> 	at org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodes(TokenCache.java:85)
> 	at org.apache.hadoop.filecache.TrackerDistributedCacheManager.getDelegationTokens(TrackerDistributedCacheManager.java:949)
> 	at org.apache.hadoop.mapred.JobClient.copyAndConfigureFiles(JobClient.java:854)
> 	at org.apache.hadoop.mapred.JobClient.copyAndConfigureFiles(JobClient.java:743)
> 	at org.apache.hadoop.mapred.JobClient.submitJobInternal(JobClient.java:945)
> 	at org.apache.hadoop.mapreduce.Job.submit(Job.java:566)
> 	at org.apache.hadoop.mapreduce.Job.waitForCompletion(Job.java:596)
> 	at org.apache.hadoop.hbase.mapreduce.RowCounter.main(RowCounter.java:173)
> {code}



--
This message was sent by Atlassian JIRA
(v6.1#6144)

Mime
View raw message