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 2015/09/09 09:09:02 UTC

[32/50] [abbrv] hive git commit: HIVE-11689 : minor flow changes to ORC split generation (Sergey Shelukhin, reviewed by Prasanth Jayachandran and Swarnim Kulkarni) ADDENDUM

HIVE-11689 : minor flow changes to ORC split generation (Sergey Shelukhin, reviewed by Prasanth Jayachandran and Swarnim Kulkarni) ADDENDUM


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3ff3c6f1
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3ff3c6f1
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3ff3c6f1

Branch: refs/heads/beeline-cli
Commit: 3ff3c6f19ab82390f44c88cde5afea32a0299986
Parents: dbdd611
Author: Sergey Shelukhin <se...@apache.org>
Authored: Wed Sep 2 11:01:35 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Wed Sep 2 11:01:35 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java     | 18 ++++++++++--------
 1 file changed, 10 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3ff3c6f1/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
index 05efc5f..cf8694e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -373,7 +374,6 @@ public class OrcInputFormat  implements InputFormat<NullWritable, OrcStruct>,
     private final Configuration conf;
     private static Cache<Path, FileInfo> footerCache;
     private static ExecutorService threadPool = null;
-    private static ExecutorCompletionService<AcidDirInfo> ecs = null;
     private final int numBuckets;
     private final long maxSize;
     private final long minSize;
@@ -419,7 +419,6 @@ public class OrcInputFormat  implements InputFormat<NullWritable, OrcStruct>,
           threadPool = Executors.newFixedThreadPool(numThreads,
               new ThreadFactoryBuilder().setDaemon(true)
                   .setNameFormat("ORC_GET_SPLITS #%d").build());
-          ecs = new ExecutorCompletionService<AcidDirInfo>(threadPool);
         }
 
         if (footerCache == null && cacheStripeDetails) {
@@ -440,7 +439,7 @@ public class OrcInputFormat  implements InputFormat<NullWritable, OrcStruct>,
   /**
    * The full ACID directory information needed for splits; no more calls to HDFS needed.
    * We could just live with AcidUtils.Directory but...
-   * 1) That doesn't contain have base files.
+   * 1) That doesn't have base files for the base-directory case.
    * 2) We save fs for convenience to avoid getting it twice.
    */
   @VisibleForTesting
@@ -1031,17 +1030,18 @@ public class OrcInputFormat  implements InputFormat<NullWritable, OrcStruct>,
     // multi-threaded file statuses and split strategy
     boolean useFileIds = HiveConf.getBoolVar(conf, ConfVars.HIVE_ORC_INCLUDE_FILE_ID_IN_SPLITS);
     Path[] paths = getInputPaths(conf);
+    CompletionService<AcidDirInfo> ecs = new ExecutorCompletionService<>(Context.threadPool);
     for (Path dir : paths) {
       FileSystem fs = dir.getFileSystem(conf);
       FileGenerator fileGenerator = new FileGenerator(context, fs, dir, useFileIds);
-      pathFutures.add(Context.ecs.submit(fileGenerator));
+      pathFutures.add(ecs.submit(fileGenerator));
     }
 
     // complete path futures and schedule split generation
     try {
       for (int notIndex = 0; notIndex < paths.length; ++notIndex) {
-        AcidDirInfo adi = Context.ecs.take().get();
-        SplitStrategy splitStrategy = determineSplitStrategy(
+        AcidDirInfo adi = ecs.take().get();
+        SplitStrategy<?> splitStrategy = determineSplitStrategy(
             context, adi.fs, adi.splitPath, adi.acidInfo, adi.baseOrOriginalFiles);
 
         if (isDebugEnabled) {
@@ -1049,12 +1049,14 @@ public class OrcInputFormat  implements InputFormat<NullWritable, OrcStruct>,
         }
 
         if (splitStrategy instanceof ETLSplitStrategy) {
-          List<SplitInfo> splitInfos = splitStrategy.getSplits();
+          List<SplitInfo> splitInfos = ((ETLSplitStrategy)splitStrategy).getSplits();
           for (SplitInfo splitInfo : splitInfos) {
             splitFutures.add(Context.threadPool.submit(new SplitGenerator(splitInfo)));
           }
         } else {
-          splits.addAll(splitStrategy.getSplits());
+          @SuppressWarnings("unchecked")
+          List<OrcSplit> readySplits = (List<OrcSplit>)splitStrategy.getSplits();
+          splits.addAll(readySplits);
         }
       }