[ 
https://issues.apache.org/jira/browse/PIG-2940?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Cheolsoo Park updated PIG-2940:
-------------------------------

    Description: 
To reproduce ths issue, please do the following in secure hadoop/hbase cluster:
# On a gateway node, run kinit to obtain kerberos credentials and run a Pig 
script that includes a HBaseStorage load/store.
# In the front-end, HBaseStorage obtains a delegation token from hbase server 
and adds it to the JobConf object.
# In the back-end, mappers connect to hbase using the delegation token w/o 
kerberos credentials.

While load-from-hbase works perfectly fine, store-to-hbase fails. This is 
because at step 3, mappers attempt to obtain a delegation token from hbase in 
the back-end.
{code:title=setStoreLocation()}
// Not setting a udf property and getting the hbase delegation token
// only once like in setLocation as setStoreLocation gets different Job
// objects for each call and the last Job passed is the one that is
// launched. So we end up getting multiple hbase delegation tokens.
addHBaseDelegationToken(m_conf, job);
{code}
The problem is that mappers in the back-end don't have kerberos credentials, so 
the call to addHBaseDelegationToken() fails with the following error:
{code}
2012-09-30 14:33:42,310 ERROR [main] 
org.apache.hadoop.security.UserGroupInformation: PriviledgedActionException 
as:testuser (auth:SIMPLE) 
cause:org.apache.hadoop.hbase.security.AccessDeniedException: 
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}
This is not an issue with load because a delegation token is only obtained in 
the front-end for the first time when HBASE_TOKEN_SET is not set.
{code:title=setLocation()}
String delegationTokenSet = udfProps.getProperty(HBASE_TOKEN_SET);
if (delegationTokenSet == null) {
    addHBaseDelegationToken(m_conf, job);
    udfProps.setProperty(HBASE_TOKEN_SET, "true");
}
{code}
The proposed fix is to modify addHBaseDelegationToken() so that tokens are 
obtained only if the current user has kerberos credentials, which is true in 
the front-end while false in the back-end.
{code}
--- a/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
+++ b/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
@@ -638,13 +638,15 @@ public class HBaseStorage extends LoadFunc implements 
StoreFuncInterface, LoadPu
                 // getCurrentUser method is not public in 0.20.2
                 Method m1 = 
UserGroupInformation.class.getMethod("getCurrentUser");
                 UserGroupInformation currentUser = (UserGroupInformation) 
m1.invoke(null,(Object[]) null);
                 if (currentUser.hasKerberosCredentials()) {
                     // Class and method are available only from 0.92 security 
release
                     Class tokenUtilClass = Class
                             
.forName("org.apache.hadoop.hbase.security.token.TokenUtil");
                     Method m2 = tokenUtilClass.getMethod("obtainTokenForJob",
                             new Class[] { Configuration.class, 
UserGroupInformation.class, Job.class });
                     m2.invoke(null,
                            new Object[] { hbaseConf, currentUser, job });
                 }
             } catch (ClassNotFoundException cnfe) {
                 throw new RuntimeException("Failure loading TokenUtil class, "
                         + "is secure RPC available?", cnfe);
{code}

  was:
To reproduce ths issue, please do the following in secure hadoop/hbase cluster:
# On a gateway node, run kinit to obtain kerberos credentials and run a Pig 
script that includes a HBaseStorage load/store.
# In the front-end, HBaseStorage obtains a delegation token from hbase server 
and adds it to the JobConf object.
# In the back-end, mappers connect to hbase using the delegation token w/o 
kerberos credentials.

While load-from-hbase works perfectly fine, store-to-hbase fails. This is 
because at step 3, mappers attempt to obtain a delegation token from hbase in 
the back-end.
{code:title=setStoreLocation()}
// Not setting a udf property and getting the hbase delegation token
// only once like in setLocation as setStoreLocation gets different Job
// objects for each call and the last Job passed is the one that is
// launched. So we end up getting multiple hbase delegation tokens.
addHBaseDelegationToken(m_conf, job);
{code}
The problem is that mappers in the back-end don't have kerberos credentials, so 
the call to addHBaseDelegationToken() fails with the following error:
{code}
2012-09-30 14:33:42,310 ERROR [main] 
org.apache.hadoop.security.UserGroupInformation: PriviledgedActionException 
as:testuser (auth:SIMPLE) 
cause:org.apache.hadoop.hbase.security.AccessDeniedException: 
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}
This is not an issue with load because a delegation token is only obtained in 
the front-end for the first time when HBASE_TOKEN_SET is not set.
{code}
String delegationTokenSet = udfProps.getProperty(HBASE_TOKEN_SET);
if (delegationTokenSet == null) {
    addHBaseDelegationToken(m_conf, job);
    udfProps.setProperty(HBASE_TOKEN_SET, "true");
}
{code}
The proposed fix is to modify addHBaseDelegationToken() so that tokens are 
obtained only if the current user has kerberos credentials, which is true in 
the front-end while false in the back-end.
{code}
--- a/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
+++ b/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
@@ -638,13 +638,15 @@ public class HBaseStorage extends LoadFunc implements 
StoreFuncInterface, LoadPu
                 // getCurrentUser method is not public in 0.20.2
                 Method m1 = 
UserGroupInformation.class.getMethod("getCurrentUser");
                 UserGroupInformation currentUser = (UserGroupInformation) 
m1.invoke(null,(Object[]) null);
                 if (currentUser.hasKerberosCredentials()) {
                     // Class and method are available only from 0.92 security 
release
                     Class tokenUtilClass = Class
                             
.forName("org.apache.hadoop.hbase.security.token.TokenUtil");
                     Method m2 = tokenUtilClass.getMethod("obtainTokenForJob",
                             new Class[] { Configuration.class, 
UserGroupInformation.class, Job.class });
                     m2.invoke(null,
                            new Object[] { hbaseConf, currentUser, job });
                 }
             } catch (ClassNotFoundException cnfe) {
                 throw new RuntimeException("Failure loading TokenUtil class, "
                         + "is secure RPC available?", cnfe);
{code}

    
> HBaseStorage store fails in secure cluster
> ------------------------------------------
>
>                 Key: PIG-2940
>                 URL: https://issues.apache.org/jira/browse/PIG-2940
>             Project: Pig
>          Issue Type: Bug
>            Reporter: Cheolsoo Park
>            Assignee: Cheolsoo Park
>             Fix For: 0.11
>
>
> To reproduce ths issue, please do the following in secure hadoop/hbase 
> cluster:
> # On a gateway node, run kinit to obtain kerberos credentials and run a Pig 
> script that includes a HBaseStorage load/store.
> # In the front-end, HBaseStorage obtains a delegation token from hbase server 
> and adds it to the JobConf object.
> # In the back-end, mappers connect to hbase using the delegation token w/o 
> kerberos credentials.
> While load-from-hbase works perfectly fine, store-to-hbase fails. This is 
> because at step 3, mappers attempt to obtain a delegation token from hbase in 
> the back-end.
> {code:title=setStoreLocation()}
> // Not setting a udf property and getting the hbase delegation token
> // only once like in setLocation as setStoreLocation gets different Job
> // objects for each call and the last Job passed is the one that is
> // launched. So we end up getting multiple hbase delegation tokens.
> addHBaseDelegationToken(m_conf, job);
> {code}
> The problem is that mappers in the back-end don't have kerberos credentials, 
> so the call to addHBaseDelegationToken() fails with the following error:
> {code}
> 2012-09-30 14:33:42,310 ERROR [main] 
> org.apache.hadoop.security.UserGroupInformation: PriviledgedActionException 
> as:testuser (auth:SIMPLE) 
> cause:org.apache.hadoop.hbase.security.AccessDeniedException: 
> 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}
> This is not an issue with load because a delegation token is only obtained in 
> the front-end for the first time when HBASE_TOKEN_SET is not set.
> {code:title=setLocation()}
> String delegationTokenSet = udfProps.getProperty(HBASE_TOKEN_SET);
> if (delegationTokenSet == null) {
>     addHBaseDelegationToken(m_conf, job);
>     udfProps.setProperty(HBASE_TOKEN_SET, "true");
> }
> {code}
> The proposed fix is to modify addHBaseDelegationToken() so that tokens are 
> obtained only if the current user has kerberos credentials, which is true in 
> the front-end while false in the back-end.
> {code}
> --- a/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
> +++ b/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
> @@ -638,13 +638,15 @@ public class HBaseStorage extends LoadFunc implements 
> StoreFuncInterface, LoadPu
>                  // getCurrentUser method is not public in 0.20.2
>                  Method m1 = 
> UserGroupInformation.class.getMethod("getCurrentUser");
>                  UserGroupInformation currentUser = (UserGroupInformation) 
> m1.invoke(null,(Object[]) null);
>                  if (currentUser.hasKerberosCredentials()) {
>                      // Class and method are available only from 0.92 
> security release
>                      Class tokenUtilClass = Class
>                              
> .forName("org.apache.hadoop.hbase.security.token.TokenUtil");
>                      Method m2 = tokenUtilClass.getMethod("obtainTokenForJob",
>                              new Class[] { Configuration.class, 
> UserGroupInformation.class, Job.class });
>                      m2.invoke(null,
>                             new Object[] { hbaseConf, currentUser, job });
>                  }
>              } catch (ClassNotFoundException cnfe) {
>                  throw new RuntimeException("Failure loading TokenUtil class, 
> "
>                          + "is secure RPC available?", cnfe);
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to