You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2014/05/20 20:27:32 UTC

svn commit: r1596354 - in /hbase/branches/0.98: hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java

Author: tedyu
Date: Tue May 20 18:27:32 2014
New Revision: 1596354

URL: http://svn.apache.org/r1596354
Log:
HBASE-11104 IntegrationTestImportTsv#testRunFromOutputCommitter misses credential initialization (Vandana)


Modified:
    hbase/branches/0.98/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java
    hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java

Modified: hbase/branches/0.98/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java?rev=1596354&r1=1596353&r2=1596354&view=diff
==============================================================================
--- hbase/branches/0.98/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java (original)
+++ hbase/branches/0.98/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java Tue May 20 18:27:32 2014
@@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.client.Sc
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.OutputCommitter;
@@ -60,6 +61,7 @@ import org.apache.hadoop.mapreduce.lib.i
 import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -257,7 +259,8 @@ public class IntegrationTestImportTsv im
       // look a lot like TestImportTsv#testMROnTable.
       final String table = format("%s-%s-child", NAME, context.getJobID());
       final String cf = "FAM";
-
+      String fileLocation = System.getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION);
+      conf.set(ImportTsv.CREDENTIALS_LOCATION, fileLocation);
       String[] args = {
           "-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B",
           "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b",
@@ -271,7 +274,9 @@ public class IntegrationTestImportTsv im
       } catch (Exception e) {
         throw new IOException("Underlying MapReduce job failed. Aborting commit.", e);
       } finally {
-        util.deleteTable(table);
+        if (util.getHBaseAdmin().tableExists(table)) {
+          util.deleteTable(table);
+        }
       }
     }
   }
@@ -362,6 +367,10 @@ public class IntegrationTestImportTsv im
     JobLaunchingOutputFormat.setOutputPath(job, outputPath);
     TableMapReduceUtil.addDependencyJars(job);
     addTestDependencyJars(job.getConfiguration());
+    TableMapReduceUtil.initCredentials(job);
+    JobClient jc = new JobClient(job.getConfiguration());
+    job.getCredentials().addToken(new Text("my_mr_token"),
+      jc.getDelegationToken(new Text("renewer")));
 
     // Job launched by the OutputCommitter will fail if dependency jars are
     // not shipped properly.

Modified: hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java?rev=1596354&r1=1596353&r2=1596354&view=diff
==============================================================================
--- hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java (original)
+++ hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java Tue May 20 18:27:32 2014
@@ -20,11 +20,13 @@ package org.apache.hadoop.hbase.mapreduc
 
 import static java.lang.String.format;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.Set;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -49,6 +51,7 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -77,6 +80,9 @@ public class ImportTsv extends Configure
   public final static String BULK_OUTPUT_CONF_KEY = "importtsv.bulk.output";
   public final static String TIMESTAMP_CONF_KEY = "importtsv.timestamp";
   public final static String JOB_NAME_CONF_KEY = "mapred.job.name";
+  //This config is used to propagate credentials from parent MR jobs which launch
+  //ImportTSV jobs. SEE IntegrationTestImportTsv.
+  public final static String CREDENTIALS_LOCATION = "credentials_location";
   // TODO: the rest of these configs are used exclusively by TsvImporterMapper.
   // Move them out of the tool and let the mapper handle its own validation.
   public final static String SKIP_LINES_CONF_KEY = "importtsv.skip.bad.lines";
@@ -394,7 +400,6 @@ public class ImportTsv extends Configure
       throws IOException, ClassNotFoundException {
 
     HBaseAdmin admin = new HBaseAdmin(conf);
-
     // Support non-XML supported characters
     // by re-encoding the passed separator as a Base64 string.
     String actualSeparator = conf.get(SEPARATOR_CONF_KEY);
@@ -416,9 +421,14 @@ public class ImportTsv extends Configure
     FileInputFormat.setInputPaths(job, inputDir);
     job.setInputFormatClass(TextInputFormat.class);
     job.setMapperClass(mapperClass);
-
     String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
     String columns[] = conf.getStrings(COLUMNS_CONF_KEY);
+    if(StringUtils.isNotEmpty(conf.get(CREDENTIALS_LOCATION))) {
+      String fileLoc = conf.get(CREDENTIALS_LOCATION);
+      Credentials cred = Credentials.readTokenStorageFile(new File(fileLoc), conf);
+      job.getCredentials().addAll(cred);
+    }
+
     if (hfileOutPath != null) {
       if (!admin.tableExists(tableName)) {
         LOG.warn(format("Table '%s' does not exist.", tableName));