You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by xu...@apache.org on 2014/12/29 18:44:43 UTC

svn commit: r1648398 - in /hive/branches/spark: ./ hbase-handler/ ql/src/java/org/apache/hadoop/hive/ql/io/

Author: xuefu
Date: Mon Dec 29 17:44:42 2014
New Revision: 1648398

URL: http://svn.apache.org/r1648398
Log:
HIVE-9153: Perf enhancement on CombineHiveInputFormat and HiveInputFormat (Rui via Xuefu)
merged from trunk, r1648397

Modified:
    hive/branches/spark/   (props changed)
    hive/branches/spark/hbase-handler/pom.xml   (props changed)
    hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java
    hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveRecordReader.java
    hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java

Propchange: hive/branches/spark/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Mon Dec 29 17:44:42 2014
@@ -2,4 +2,4 @@
 /hive/branches/cbo:1605012-1627125
 /hive/branches/tez:1494760-1622766
 /hive/branches/vectorization:1466908-1527856
-/hive/trunk:1608589-1648226
+/hive/trunk:1608589-1648226,1648397

Propchange: hive/branches/spark/hbase-handler/pom.xml
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Mon Dec 29 17:44:42 2014
@@ -2,4 +2,4 @@
 /hive/branches/cbo/hbase-handler/pom.xml:1605012-1627125
 /hive/branches/tez/hbase-handler/pom.xml:1494760-1622766
 /hive/branches/vectorization/hbase-handler/pom.xml:1466908-1527856
-/hive/trunk/hbase-handler/pom.xml:1494760-1537575,1608589-1633422,1633911,1634262,1634442,1634636,1634946,1636885,1636888,1637521,1641875,1642127,1642148,1643125,1644171,1644717,1644764,1644780,1646994
+/hive/trunk/hbase-handler/pom.xml:1494760-1537575,1608589-1633422,1633911,1634262,1634442,1634636,1634946,1636885,1636888,1637521,1641875,1642127,1642148,1643125,1644171,1644717,1644764,1644780,1646994,1648397

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java?rev=1648398&r1=1648397&r2=1648398&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java Mon Dec 29 17:44:42 2014
@@ -275,7 +275,8 @@ public class CombineHiveInputFormat<K ex
   /**
    * Create Hive splits based on CombineFileSplit.
    */
-  private InputSplit[] getCombineSplits(JobConf job, int numSplits, Map<String, PartitionDesc> pathToPartitionInfo)
+  private InputSplit[] getCombineSplits(JobConf job, int numSplits,
+      Map<String, PartitionDesc> pathToPartitionInfo)
       throws IOException {
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
     perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.GET_SPLITS);
@@ -512,7 +513,8 @@ public class CombineHiveInputFormat<K ex
     if (combinablePaths.size() > 0) {
       FileInputFormat.setInputPaths(job, combinablePaths.toArray
           (new Path[combinablePaths.size()]));
-      Map<String, PartitionDesc> pathToPartitionInfo = Utilities.getMapWork(job).getPathToPartitionInfo();
+      Map<String, PartitionDesc> pathToPartitionInfo = this.pathToPartitionInfo != null ?
+          this.pathToPartitionInfo : Utilities.getMapWork(job).getPathToPartitionInfo();
       InputSplit[] splits = getCombineSplits(job, numSplits, pathToPartitionInfo);
       for (InputSplit split : splits) {
         result.add(split);
@@ -658,7 +660,7 @@ public class CombineHiveInputFormat<K ex
 
     return ShimLoader.getHadoopShims().getCombineFileInputFormat()
         .getRecordReader(job,
-        ((CombineHiveInputSplit) split).getInputSplitShim(), reporter,
+        (CombineFileSplit) split, reporter,
         CombineHiveRecordReader.class);
   }
 

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveRecordReader.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveRecordReader.java?rev=1648398&r1=1648397&r2=1648398&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveRecordReader.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveRecordReader.java Mon Dec 29 17:44:42 2014
@@ -44,8 +44,9 @@ public class CombineHiveRecordReader<K e
   public CombineHiveRecordReader(InputSplit split, Configuration conf,
       Reporter reporter, Integer partition) throws IOException {
     super((JobConf)conf);
-    CombineHiveInputSplit hsplit = new CombineHiveInputSplit(jobConf,
-        (CombineFileSplit) split);
+    CombineHiveInputSplit hsplit = split instanceof CombineHiveInputSplit ?
+        (CombineHiveInputSplit) split :
+        new CombineHiveInputSplit(jobConf, (CombineFileSplit) split);
     String inputFormatClassName = hsplit.inputFormatClassName();
     Class inputFormatClass = null;
     try {

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java?rev=1648398&r1=1648397&r2=1648398&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java Mon Dec 29 17:44:42 2014
@@ -258,15 +258,17 @@ public class HiveInputFormat<K extends W
   }
 
   protected void init(JobConf job) {
-    if (HiveConf.getVar(job, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) {
-      mrwork = (MapWork) Utilities.getMergeWork(job);
-      if (mrwork == null) {
+    if (mrwork == null || pathToPartitionInfo == null) {
+      if (HiveConf.getVar(job, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) {
+        mrwork = (MapWork) Utilities.getMergeWork(job);
+        if (mrwork == null) {
+          mrwork = Utilities.getMapWork(job);
+        }
+      } else {
         mrwork = Utilities.getMapWork(job);
       }
-    } else {
-      mrwork = Utilities.getMapWork(job);
+      pathToPartitionInfo = mrwork.getPathToPartitionInfo();
     }
-    pathToPartitionInfo = mrwork.getPathToPartitionInfo();
   }
 
   /*