Return-Path: X-Original-To: apmail-hbase-issues-archive@www.apache.org Delivered-To: apmail-hbase-issues-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id C916510939 for ; Tue, 5 Nov 2013 19:44:18 +0000 (UTC) Received: (qmail 1395 invoked by uid 500); 5 Nov 2013 19:44:18 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 1353 invoked by uid 500); 5 Nov 2013 19:44:18 -0000 Mailing-List: contact issues-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@hbase.apache.org Received: (qmail 1297 invoked by uid 99); 5 Nov 2013 19:44:18 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 05 Nov 2013 19:44:18 +0000 Date: Tue, 5 Nov 2013 19:44:18 +0000 (UTC) From: "Gary Helmling (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HBASE-9890) MR jobs are not working if started by a delegated user MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/HBASE-9890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13814188#comment-13814188 ] Gary Helmling commented on HBASE-9890: -------------------------------------- I've looked through the secure bulk load code in a little more detail, but I still can't say I understand why use of SecureBulkLoadClient in LoadIncrementalHFiles is conditioned on isHBaseSecurityEnabled(), instead of isHadoopSecurityEnabled(). It seems like they should be conditioned on isHadoopSecurityEnabled() instead, since this is all in place to pass through an HDFS delegation token for moving the HFiles on secure Hadoop. [~mbertozzi] Makes sense to me to change the LoadIncrementatlHFiles conditions here as well, assuming that doesn't cascade into broken tests. But I'm also okay with pushing that part into a separate JIRA, since it's somewhat independent of the original issue. The rest of the patch looks good to me. [~toffer] Any insights into why SecureBulkLoadClient usage is conditioned on HBase security being enabled instead of HDFS security? > 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)