hive-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From sunc...@apache.org
Subject svn commit: r1671964 - /hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SecureCmdDoAs.java
Date Tue, 07 Apr 2015 22:39:27 GMT
Author: sunchao
Date: Tue Apr  7 22:39:27 2015
New Revision: 1671964

URL: http://svn.apache.org/r1671964
Log:
HIVE-10098 - HS2 local task for map join fails in KMS encrypted cluster (Yongzhi via Chao)

Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SecureCmdDoAs.java

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SecureCmdDoAs.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SecureCmdDoAs.java?rev=1671964&r1=1671963&r2=1671964&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SecureCmdDoAs.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SecureCmdDoAs.java Tue Apr  7 22:39:27
2015
@@ -45,14 +45,14 @@ public class SecureCmdDoAs {
     // metastore tokens into a file
     String uname = UserGroupInformation.getLoginUser().getShortUserName();
     FileSystem fs = FileSystem.get(conf);
-    Token<?> fsToken = fs.getDelegationToken(uname);
+    Credentials cred = new Credentials();
+    // Use method addDelegationTokens instead of getDelegationToken to get all the tokens
including KMS.
+    fs.addDelegationTokens(uname, cred);
 
     tokenFile = File.createTempFile("hive_hadoop_delegation_token", null);
     tokenPath = new Path(tokenFile.toURI());
 
     //write credential with token to file
-    Credentials cred = new Credentials();
-    cred.addToken(fsToken.getService(), fsToken);
     cred.writeTokenStorageFile(tokenPath, conf);
   }
 



Mime
View raw message