incubator-hcatalog-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From khorg...@apache.org
Subject svn commit: r1210698 - in /incubator/hcatalog/trunk: CHANGES.txt storage-drivers/hbase/src/java/org/apache/hcatalog/hbase/HBaseHCatStorageHandler.java storage-drivers/hbase/src/java/org/apache/hcatalog/hbase/ImportSequenceFile.java
Date Mon, 05 Dec 2011 23:05:20 GMT
Author: khorgath
Date: Mon Dec  5 23:05:19 2011
New Revision: 1210698

URL: http://svn.apache.org/viewvc?rev=1210698&view=rev
Log:
HCATALOG-170 HBaseBulkOSD fails to launch ImportSequenceFile because of missing jars in dist
cache (toffer via khorgath)

Modified:
    incubator/hcatalog/trunk/CHANGES.txt
    incubator/hcatalog/trunk/storage-drivers/hbase/src/java/org/apache/hcatalog/hbase/HBaseHCatStorageHandler.java
    incubator/hcatalog/trunk/storage-drivers/hbase/src/java/org/apache/hcatalog/hbase/ImportSequenceFile.java

Modified: incubator/hcatalog/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/CHANGES.txt?rev=1210698&r1=1210697&r2=1210698&view=diff
==============================================================================
--- incubator/hcatalog/trunk/CHANGES.txt (original)
+++ incubator/hcatalog/trunk/CHANGES.txt Mon Dec  5 23:05:19 2011
@@ -79,6 +79,8 @@ Trunk (unreleased changes)
   OPTIMIZATIONS
 
   BUG FIXES
+  HCAT-170. HBaseBulkOSD fails to launch ImportSequenceFile because of missing jars in dist
cache (toffer via khorgath)
+
   HCAT-176. Class not found exception when running TestPigStorageDriver (daijy via khorgath)
 
   HCAT-175. HCat fail to build on trunk (daijy via khorgath)

Modified: incubator/hcatalog/trunk/storage-drivers/hbase/src/java/org/apache/hcatalog/hbase/HBaseHCatStorageHandler.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/storage-drivers/hbase/src/java/org/apache/hcatalog/hbase/HBaseHCatStorageHandler.java?rev=1210698&r1=1210697&r2=1210698&view=diff
==============================================================================
--- incubator/hcatalog/trunk/storage-drivers/hbase/src/java/org/apache/hcatalog/hbase/HBaseHCatStorageHandler.java
(original)
+++ incubator/hcatalog/trunk/storage-drivers/hbase/src/java/org/apache/hcatalog/hbase/HBaseHCatStorageHandler.java
Mon Dec  5 23:05:19 2011
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 
+import com.facebook.fb303.FacebookBase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.MasterNot
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hive.hbase.HBaseSerDe;
 import org.apache.hadoop.hive.metastore.HiveMetaHook;
@@ -46,11 +48,15 @@ import org.apache.hadoop.hive.ql.plan.Ta
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
 import org.apache.hadoop.hive.serde2.SerDe;
 import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hcatalog.mapreduce.HCatInputStorageDriver;
+import org.apache.hcatalog.mapreduce.HCatOutputFormat;
 import org.apache.hcatalog.mapreduce.HCatOutputStorageDriver;
 import org.apache.hcatalog.mapreduce.HCatTableInfo;
 import org.apache.hcatalog.storagehandler.HCatStorageHandler;
+import org.apache.thrift.TBase;
+import org.apache.zookeeper.ZooKeeper;
 
 /**
  * This class HBaseHCatStorageHandler provides functionality to create HBase
@@ -396,4 +402,37 @@ public class HBaseHCatStorageHandler ext
         return qualifiedName;
     }
 
+    /**
+     * Helper method for users to add the required depedency jars to distributed cache.
+     * @param conf
+     * @throws IOException
+     */
+    public static void addDependencyJars(Configuration conf) throws IOException {
+        //TODO provide a facility/interface for loading/specifying dependencies
+        //Ideally this method shouldn't be exposed to the user
+        TableMapReduceUtil.addDependencyJars(conf,
+                //hadoop-core
+                Writable.class,
+                //ZK
+                ZooKeeper.class,
+                //HBase
+                HTable.class,
+                //Hive
+                HiveException.class,
+                //HCatalog jar
+                HCatOutputFormat.class,
+                //hive hbase storage handler jar
+                HBaseSerDe.class,
+                //hcat hbase storage driver jar
+                HBaseOutputStorageDriver.class,
+                //hive jar
+                Table.class,
+                //libthrift jar
+                TBase.class,
+                //hbase jar
+                Bytes.class,
+                //thrift-fb303 .jar
+                FacebookBase.class);
+    }
+
 }

Modified: incubator/hcatalog/trunk/storage-drivers/hbase/src/java/org/apache/hcatalog/hbase/ImportSequenceFile.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/storage-drivers/hbase/src/java/org/apache/hcatalog/hbase/ImportSequenceFile.java?rev=1210698&r1=1210697&r2=1210698&view=diff
==============================================================================
--- incubator/hcatalog/trunk/storage-drivers/hbase/src/java/org/apache/hcatalog/hbase/ImportSequenceFile.java
(original)
+++ incubator/hcatalog/trunk/storage-drivers/hbase/src/java/org/apache/hcatalog/hbase/ImportSequenceFile.java
Mon Dec  5 23:05:19 2011
@@ -35,7 +35,6 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -182,34 +181,39 @@ class ImportSequenceFile {
             job.getConfiguration().set(TotalOrderPartitioner.PARTITIONER_PATH,partitionFile.toString());
         }
 
-        //add hbase dependency jars
-        TableMapReduceUtil.addDependencyJars(job);
-        TableMapReduceUtil.addDependencyJars(job.getConfiguration());
         return job;
     }
 
     /**
      * Method to run the Importer MapReduce Job. Normally will be called by another MR job
-     * during OutputCommitter.commitJob().
-      * @param otherConf configuration of the parent job
+     * during OutputCommitter.commitJob(). It wil inherit
+      * @param parentConf configuration of the parent job
      * @param tableName name of table to bulk load data into
      * @param InputDir path of SequenceFile formatted data to read
      * @param scratchDir temporary path for the Importer MR job to build the HFiles which
will be imported
      * @return
      */
-    static boolean runJob(Configuration otherConf, String tableName, Path InputDir, Path
scratchDir) {
-        Configuration conf = HBaseConfiguration.create();
-        for(Map.Entry<String,String> el: otherConf) {
+    static boolean runJob(Configuration parentConf, String tableName, Path InputDir, Path
scratchDir) {
+        Configuration conf = new Configuration();
+        for(Map.Entry<String,String> el: parentConf) {
             if(el.getKey().startsWith("hbase."))
                 conf.set(el.getKey(),el.getValue());
+            if(el.getKey().startsWith("mapred.cache.archives"))
+                conf.set(el.getKey(),el.getValue());
         }
+
+        //Inherit jar dependencies added to distributed cache loaded by parent job
+        conf.set("mapred.job.classpath.archives",parentConf.get("mapred.job.classpath.archives",
""));
+        conf.set("mapreduce.job.cache.archives.visibilities",parentConf.get("mapreduce.job.cache.archives.visibilities",""));
+
         conf.set(HBaseConstants.PROPERTY_OUTPUT_TABLE_NAME_KEY, tableName);
 
         boolean localMode = "local".equals(conf.get("mapred.job.tracker"));
+
         boolean success = false;
         try {
-            FileSystem fs = FileSystem.get(conf);
-            Path workDir = new Path(new Job(otherConf).getWorkingDirectory(),IMPORTER_WORK_DIR);
+            FileSystem fs = FileSystem.get(parentConf);
+            Path workDir = new Path(new Job(parentConf).getWorkingDirectory(),IMPORTER_WORK_DIR);
             if(!fs.mkdirs(workDir))
                 throw new IOException("Importer work directory already exists: "+workDir);
             Job job = createSubmittableJob(conf, tableName, InputDir, scratchDir, localMode);



Mime
View raw message