You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hcatalog-commits@incubator.apache.org by to...@apache.org on 2012/04/13 17:51:11 UTC

svn commit: r1325867 - in /incubator/hcatalog/branches/branch-0.4: ./ src/java/org/apache/hcatalog/common/ src/java/org/apache/hcatalog/mapreduce/ src/java/org/apache/hcatalog/pig/

Author: toffer
Date: Fri Apr 13 17:51:11 2012
New Revision: 1325867

URL: http://svn.apache.org/viewvc?rev=1325867&view=rev
Log:
merged from trunk: HCAT-69 Fix token usage from HCat (toffer)

Modified:
    incubator/hcatalog/branches/branch-0.4/   (props changed)
    incubator/hcatalog/branches/branch-0.4/CHANGES.txt
    incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/common/HCatConstants.java
    incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/common/HCatUtil.java
    incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java
    incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/HCatBaseOutputFormat.java
    incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/HCatOutputFormat.java
    incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/Security.java
    incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/pig/HCatStorer.java

Propchange: incubator/hcatalog/branches/branch-0.4/
------------------------------------------------------------------------------
  Merged /incubator/hcatalog/trunk:r1325864

Modified: incubator/hcatalog/branches/branch-0.4/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.4/CHANGES.txt?rev=1325867&r1=1325866&r2=1325867&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.4/CHANGES.txt (original)
+++ incubator/hcatalog/branches/branch-0.4/CHANGES.txt Fri Apr 13 17:51:11 2012
@@ -79,6 +79,8 @@ Release 0.4.0 - Unreleased
   HCAT-2 Support nested schema conversion between Hive an Pig (julienledem via hashutosh)
 
   IMPROVEMENTS
+  HCAT-69 Fix token usage from HCat (toffer)
+
   HCAT-357 e2e harness: allow to run again pre-installed packages (thw via gates)
 
   HCAT-352 Need tests for complex types (gates)

Modified: incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/common/HCatConstants.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/common/HCatConstants.java?rev=1325867&r1=1325866&r2=1325867&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/common/HCatConstants.java (original)
+++ incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/common/HCatConstants.java Fri Apr 13 17:51:11 2012
@@ -72,15 +72,11 @@ public final class HCatConstants {
   public static final String HCAT_KEY_OUTPUT_INFO = HCAT_KEY_OUTPUT_BASE + ".info";
   public static final String HCAT_KEY_HIVE_CONF = HCAT_KEY_OUTPUT_BASE + ".hive.conf";
   public static final String HCAT_KEY_TOKEN_SIGNATURE = HCAT_KEY_OUTPUT_BASE + ".token.sig";
-  public static final String HCAT_KEY_JOBCLIENT_TOKEN_SIGNATURE = HCAT_KEY_OUTPUT_BASE + ".jobclient.token.sig";
-  public static final String HCAT_KEY_JOBCLIENT_TOKEN_STRFORM = HCAT_KEY_OUTPUT_BASE + ".jobclient.token.strform";
 
   public static final String[] OUTPUT_CONFS_TO_SAVE = {
     HCAT_KEY_OUTPUT_INFO,
     HCAT_KEY_HIVE_CONF,
-    HCAT_KEY_TOKEN_SIGNATURE,
-    HCAT_KEY_JOBCLIENT_TOKEN_SIGNATURE,
-    HCAT_KEY_JOBCLIENT_TOKEN_STRFORM
+    HCAT_KEY_TOKEN_SIGNATURE
     };
 
 

Modified: incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/common/HCatUtil.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/common/HCatUtil.java?rev=1325867&r1=1325866&r2=1325867&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/common/HCatUtil.java (original)
+++ incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/common/HCatUtil.java Fri Apr 13 17:51:11 2012
@@ -358,22 +358,6 @@ public class HCatUtil {
         // return null;
     }
 
-    public static void cancelJobTrackerDelegationToken(String tokenStrForm,
-            String tokenSignature) throws Exception {
-        // LOG.info("cancelJobTrackerDelegationToken("+tokenStrForm+","+tokenSignature+")");
-        JobClient jcl = new JobClient(new JobConf(new Configuration(),
-                HCatOutputFormat.class));
-        Token<org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier> t = extractJobTrackerToken(
-                tokenStrForm, tokenSignature);
-        // LOG.info("canceling "+t);
-        try {
-            jcl.cancelDelegationToken(t);
-        } catch (Exception e) {
-            // HCatUtil.logToken(LOG, "jcl token to cancel", t);
-            // ignore if token has already been invalidated.
-        }
-    }
-
     public static Token<? extends AbstractDelegationTokenIdentifier> extractThriftToken(
             String tokenStrForm, String tokenSignature) throws MetaException,
             TException, IOException {
@@ -385,17 +369,6 @@ public class HCatUtil {
         return t;
     }
 
-    public static Token<org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier> extractJobTrackerToken(
-            String tokenStrForm, String tokenSignature) throws MetaException,
-            TException, IOException {
-        // LOG.info("extractJobTrackerToken("+tokenStrForm+","+tokenSignature+")");
-        Token<org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier> t = new Token<org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier>();
-        t.decodeFromUrlString(tokenStrForm);
-        t.setService(new Text(tokenSignature));
-        // LOG.info("returning "+t);
-        return t;
-    }
-
     /**
      * Logging stack trace
      *

Modified: incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java?rev=1325867&r1=1325866&r2=1325867&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java (original)
+++ incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java Fri Apr 13 17:51:11 2012
@@ -187,15 +187,6 @@ class FileOutputCommitterContainer exten
                     (HCatConstants.HCAT_KEY_TOKEN_SIGNATURE) != null) {
                 client.cancelDelegationToken(tokenStrForm);
             }
-
-            if (harProcessor.isEnabled()) {
-                String jcTokenStrForm = jobContext.getConfiguration().get(HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_STRFORM);
-                String jcTokenSignature = jobContext.getConfiguration().get(HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_SIGNATURE);
-                if(jcTokenStrForm != null && jcTokenSignature != null) {
-                    HCatUtil.cancelJobTrackerDelegationToken(tokenStrForm,jcTokenSignature);
-                }
-            }
-
         } catch(Exception e) {
             if( e instanceof HCatException ) {
                 throw (HCatException) e;
@@ -387,23 +378,8 @@ class FileOutputCommitterContainer exten
                 getBaseOutputCommitter().cleanupJob(HCatMapRedUtil.createJobContext(context));
             }
 
-            //Cancel HCat and JobTracker tokens
-            // cancel the deleg. tokens that were acquired for this job now that
-            // we are done - we should cancel if the tokens were acquired by
-            // HCatOutputFormat and not if they were supplied by Oozie. In the latter
-            // case the HCAT_KEY_TOKEN_SIGNATURE property in the conf will not be set
-            String tokenStrForm = client.getTokenStrForm();
-            if(tokenStrForm != null && context.getConfiguration().get(HCatConstants.HCAT_KEY_TOKEN_SIGNATURE) != null) {
-              client.cancelDelegationToken(tokenStrForm);
-            }
-            if(harProcessor.isEnabled()) {
-                String jcTokenStrForm =
-                  context.getConfiguration().get(HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_STRFORM);
-                String jcTokenSignature =
-                  context.getConfiguration().get(HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_SIGNATURE);
-                if(jcTokenStrForm != null && jcTokenSignature != null) {
-                  HCatUtil.cancelJobTrackerDelegationToken(tokenStrForm,jcTokenSignature);
-                }
+            if(Security.getInstance().isSecurityEnabled()) {
+                Security.getInstance().cancelToken(client, context);
             }
         } catch (Exception e) {
 

Modified: incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/HCatBaseOutputFormat.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/HCatBaseOutputFormat.java?rev=1325867&r1=1325866&r2=1325867&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/HCatBaseOutputFormat.java (original)
+++ incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/HCatBaseOutputFormat.java Fri Apr 13 17:51:11 2012
@@ -221,25 +221,4 @@ public abstract class HCatBaseOutputForm
     jobInfo.setPosOfDynPartCols(posOfDynPartCols);
     jobInfo.setOutputSchema(schemaWithoutParts);
   }
-
-  static void cancelDelegationTokens(JobContext context, OutputJobInfo outputJobInfo) throws Exception {
-    HiveConf hiveConf = HCatUtil.getHiveConf(context.getConfiguration());
-    HiveMetaStoreClient client = HCatUtil.createHiveClient(hiveConf);
-    // cancel the deleg. tokens that were acquired for this job now that
-    // we are done - we should cancel if the tokens were acquired by
-    // HCatOutputFormat and not if they were supplied by Oozie. In the latter
-    // case the HCAT_KEY_TOKEN_SIGNATURE property in the conf will not be set
-    String tokenStrForm = client.getTokenStrForm();
-    if(tokenStrForm != null && context.getConfiguration().get(HCatConstants.HCAT_KEY_TOKEN_SIGNATURE) != null) {
-      client.cancelDelegationToken(tokenStrForm);
-    }
-
-    String jcTokenStrForm =
-      context.getConfiguration().get(HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_STRFORM);
-    String jcTokenSignature =
-      context.getConfiguration().get(HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_SIGNATURE);
-    if(jcTokenStrForm != null && jcTokenSignature != null) {
-      HCatUtil.cancelJobTrackerDelegationToken(tokenStrForm,jcTokenSignature);
-    }
-  }
 }

Modified: incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/HCatOutputFormat.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/HCatOutputFormat.java?rev=1325867&r1=1325866&r2=1325867&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/HCatOutputFormat.java (original)
+++ incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/HCatOutputFormat.java Fri Apr 13 17:51:11 2012
@@ -186,11 +186,8 @@ public class HCatOutputFormat extends HC
         FsPermission.setUMask(conf, FsPermission.getDefault().applyUMask(
             tblPath.getFileSystem(conf).getFileStatus(tblPath).getPermission()));
 
-        try {
-          UserGroupInformation.class.getMethod("isSecurityEnabled");
-          Security.getInstance().handleSecurity(job, outputJobInfo, client, conf, harRequested);
-        } catch (NoSuchMethodException e) {
-          LOG.info("Security is not supported by this version of hadoop.");
+        if(Security.getInstance().isSecurityEnabled()) {
+            Security.getInstance().handleSecurity(job, outputJobInfo, client, conf, harRequested);
         }
       } catch(Exception e) {
         if( e instanceof HCatException ) {

Modified: incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/Security.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/Security.java?rev=1325867&r1=1325866&r2=1325867&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/Security.java (original)
+++ incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/mapreduce/Security.java Fri Apr 13 17:51:11 2012
@@ -19,16 +19,22 @@
 package org.apache.hcatalog.mapreduce;
 
 import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.thrift.DelegationTokenSelector;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobTracker;
 import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -39,21 +45,39 @@ import org.apache.hcatalog.common.HCatUt
 import org.apache.thrift.TException;
 
 final class Security {
+
+  private static final Log LOG = LogFactory.getLog(HCatOutputFormat.class);
   
   // making sure this is not initialized unless needed
   private static final class LazyHolder {
     public static final Security INSTANCE = new Security();
   }
 
-  private static Map<String, Token<? extends AbstractDelegationTokenIdentifier>> tokenMap = new HashMap<String, Token<? extends AbstractDelegationTokenIdentifier>>();
-  
   public static Security getInstance() {
     return LazyHolder.INSTANCE;
   }
 
+  boolean isSecurityEnabled() {
+      try {
+          Method m = UserGroupInformation.class.getMethod("isSecurityEnabled");
+          return (Boolean)m.invoke(null, (Object[])null);
+      } catch (NoSuchMethodException e) {
+          LOG.info("Security is not supported by this version of hadoop.", e);
+      } catch (InvocationTargetException e) {
+          String msg = "Failed to call isSecurityEnabled()";
+          LOG.info(msg, e);
+          throw new IllegalStateException(msg,e);
+      } catch (IllegalAccessException e) {
+          String msg = "Failed to call isSecurityEnabled()";
+          LOG.info(msg, e);
+          throw new IllegalStateException(msg,e);
+      }
+      return false;
+  }
+
   // a signature string to associate with a HCatTableInfo - essentially
   // a concatenation of dbname, tablename and partition keyvalues.
-  private String getTokenSignature(OutputJobInfo outputJobInfo) {
+  String getTokenSignature(OutputJobInfo outputJobInfo) {
     StringBuilder result = new StringBuilder("");
     String dbName = outputJobInfo.getDatabaseName();
     if(dbName != null) {
@@ -61,13 +85,17 @@ final class Security {
     }
     String tableName = outputJobInfo.getTableName();
     if(tableName != null) {
-      result.append("+" + tableName);
+      result.append("." + tableName);
     }
     Map<String, String> partValues = outputJobInfo.getPartitionValues();
     if(partValues != null) {
       for(Entry<String, String> entry: partValues.entrySet()) {
-        result.append("+" + entry.getKey() + "=" + entry.getValue());
+        result.append("/");
+        result.append(entry.getKey());
+        result.append("=");
+        result.append(entry.getValue());
       }
+
     }
     return result.toString();
   }
@@ -82,22 +110,14 @@ final class Security {
     if(UserGroupInformation.isSecurityEnabled()){
       UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
       // check if oozie has set up a hcat deleg. token - if so use it
-      TokenSelector<? extends TokenIdentifier> tokenSelector = new DelegationTokenSelector();
-      // TODO: will oozie use a "service" called "oozie" - then instead of
-      // new Text() do new Text("oozie") below - if this change is made also
-      // remember to do:
-      //  job.getConfiguration().set(HCAT_KEY_TOKEN_SIGNATURE, "oozie");
-      // Also change code in OutputCommitter.cleanupJob() to cancel the
-      // token only if token.service is not "oozie" - remove the condition of
-      // HCAT_KEY_TOKEN_SIGNATURE != null in that code.
-      Token<? extends TokenIdentifier> token = tokenSelector.selectToken(
-          new Text(), ugi.getTokens());
-      if(token != null) {
-  
-        job.getCredentials().addToken(new Text(ugi.getUserName()),token);
-  
-      } else {
-  
+      TokenSelector<? extends TokenIdentifier> hiveTokenSelector = new DelegationTokenSelector();
+      //Oozie does not change the service field of the token
+      //hence by default token generation will have a value of "new Text("")"
+      //HiveClient will look for a use TokenSelector.selectToken() with service
+      //set to empty "Text" if hive.metastore.token.signature property is set to null
+      Token<? extends TokenIdentifier> hiveToken = hiveTokenSelector.selectToken(
+        new Text(), ugi.getTokens());
+      if(hiveToken == null) {
         // we did not get token set up by oozie, let's get them ourselves here.
         // we essentially get a token per unique Output HCatTableInfo - this is
         // done because through Pig, setOutput() method is called multiple times
@@ -110,47 +130,51 @@ final class Security {
         // TableInfo, we can have as many tokens as there are stores and the TokenSelector
         // will correctly pick the right tokens which the committer will use and
         // cancel.
-        
         String tokenSignature = getTokenSignature(outputJobInfo);
-        if(tokenMap.get(tokenSignature) == null) {
-          // get delegation tokens from hcat server and store them into the "job"
-          // These will be used in to publish partitions to
-          // hcat normally in OutputCommitter.commitJob()
-          // when the JobTracker in Hadoop MapReduce starts supporting renewal of 
-          // arbitrary tokens, the renewer should be the principal of the JobTracker
-          tokenMap.put(tokenSignature, HCatUtil.extractThriftToken(
-              client.getDelegationToken(ugi.getUserName()),
-              tokenSignature));
-        }
-  
-        String jcTokenSignature = "jc."+tokenSignature;
+        // get delegation tokens from hcat server and store them into the "job"
+        // These will be used in to publish partitions to
+        // hcat normally in OutputCommitter.commitJob()
+        // when the JobTracker in Hadoop MapReduce starts supporting renewal of
+        // arbitrary tokens, the renewer should be the principal of the JobTracker
+        hiveToken = HCatUtil.extractThriftToken(client.getDelegationToken(ugi.getUserName()), tokenSignature);
+
         if (harRequested){
-          if(tokenMap.get(jcTokenSignature) == null) {
-            tokenMap.put(jcTokenSignature,
+          TokenSelector<? extends TokenIdentifier> jtTokenSelector =
+            new org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenSelector();
+          Token jtToken =
+            jtTokenSelector.selectToken(org.apache.hadoop.security.SecurityUtil.buildTokenService(JobTracker.getAddress(conf)),
+                                            ugi.getTokens());
+          if(jtToken == null) {
+            //we don't need to cancel this token as the TokenRenewer for JT tokens
+            //takes care of cancelling them
+            job.getCredentials().addToken(new Text("hcat jt token"),
                 HCatUtil.getJobTrackerDelegationToken(conf,ugi.getUserName()));
           }
         }
         
-        job.getCredentials().addToken(new Text(ugi.getUserName() + tokenSignature),
-            tokenMap.get(tokenSignature));
+        job.getCredentials().addToken(new Text(ugi.getUserName() +"_"+ tokenSignature), hiveToken);
         // this will be used by the outputcommitter to pass on to the metastore client
         // which in turn will pass on to the TokenSelector so that it can select
         // the right token.
         job.getConfiguration().set(HCatConstants.HCAT_KEY_TOKEN_SIGNATURE, tokenSignature);
-  
-        if (harRequested){
-          job.getCredentials().addToken(new Text(ugi.getUserName() + jcTokenSignature),
-              tokenMap.get(jcTokenSignature));
-  
-          job.getConfiguration().set(
-              HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_SIGNATURE, jcTokenSignature);
-          job.getConfiguration().set(
-              HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_STRFORM, 
-              tokenMap.get(jcTokenSignature).encodeToUrlString());
-          //          LOG.info("Set hive dt["+tokenSignature+"]");
-          //          LOG.info("Set jt dt["+jcTokenSignature+"]");
-        }
       }
+    }
   }
+
+  // we should cancel hcat token if it was acquired by hcat
+  // and not if it was supplied (ie Oozie). In the latter
+  // case the HCAT_KEY_TOKEN_SIGNATURE property in the conf will not be set
+  void cancelToken(HiveMetaStoreClient client, JobContext context) throws IOException, MetaException {
+    String tokenStrForm = client.getTokenStrForm();
+    if(tokenStrForm != null && context.getConfiguration().get(HCatConstants.HCAT_KEY_TOKEN_SIGNATURE) != null) {
+      try {
+        client.cancelDelegationToken(tokenStrForm);
+      } catch (TException e) {
+        String msg = "Failed to cancel delegation token";
+        LOG.error(msg,e);
+        throw new IOException(msg,e);
+      }
+    }
   }
+
 }

Modified: incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/pig/HCatStorer.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/pig/HCatStorer.java?rev=1325867&r1=1325866&r2=1325867&view=diff
==============================================================================
--- incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/pig/HCatStorer.java (original)
+++ incubator/hcatalog/branches/branch-0.4/src/java/org/apache/hcatalog/pig/HCatStorer.java Fri Apr 13 17:51:11 2012
@@ -124,8 +124,6 @@ public class HCatStorer extends HCatBase
       PigHCatUtil.saveConfigIntoUDFProperties(p, config,HCatConstants.HCAT_KEY_HIVE_CONF);
       PigHCatUtil.saveConfigIntoUDFProperties(p, config,HCatConstants.HCAT_DYNAMIC_PTN_JOBID);
       PigHCatUtil.saveConfigIntoUDFProperties(p, config,HCatConstants.HCAT_KEY_TOKEN_SIGNATURE);
-      PigHCatUtil.saveConfigIntoUDFProperties(p, config,HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_SIGNATURE);
-      PigHCatUtil.saveConfigIntoUDFProperties(p, config,HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_STRFORM);
       PigHCatUtil.saveConfigIntoUDFProperties(p, config,HCatConstants.HCAT_KEY_OUTPUT_INFO);
 
       p.setProperty(COMPUTED_OUTPUT_SCHEMA,ObjectSerializer.serialize(computedSchema));
@@ -136,9 +134,6 @@ public class HCatStorer extends HCatBase
       PigHCatUtil.getConfigFromUDFProperties(p, config, HCatConstants.HCAT_KEY_HIVE_CONF);
       PigHCatUtil.getConfigFromUDFProperties(p, config, HCatConstants.HCAT_DYNAMIC_PTN_JOBID);
       PigHCatUtil.getConfigFromUDFProperties(p, config, HCatConstants.HCAT_KEY_TOKEN_SIGNATURE);
-      PigHCatUtil.getConfigFromUDFProperties(p, config, HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_SIGNATURE);
-      PigHCatUtil.getConfigFromUDFProperties(p, config, HCatConstants.HCAT_KEY_JOBCLIENT_TOKEN_STRFORM);
-
     }
   }