Author: gunther
Date: Thu Sep 25 20:08:56 2014
New Revision: 1627637
URL: http://svn.apache.org/r1627637
Log:
HIVE-8178: OrcNewInputFormat::getSplits() calls
OrcInputFormat.generateSplitsInfo twice (Gopal V via Gunther Hagleitner)
Modified:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcNewInputFormat.java
Modified:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcNewInputFormat.java
URL:
http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcNewInputFormat.java?rev=1627637&r1=1627636&r2=1627637&view=diff
==============================================================================
---
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcNewInputFormat.java
(original)
+++
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcNewInputFormat.java
Thu Sep 25 20:08:56 2014
@@ -118,13 +118,11 @@ public class OrcNewInputFormat extends I
public List<InputSplit> getSplits(JobContext jobContext)
throws IOException, InterruptedException {
perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ORC_GET_SPLITS);
- Configuration conf =
- ShimLoader.getHadoopShims().getConfiguration(jobContext);
List<OrcSplit> splits =
OrcInputFormat.generateSplitsInfo(ShimLoader.getHadoopShims()
.getConfiguration(jobContext));
- List<InputSplit> result = new ArrayList<InputSplit>();
- for(OrcSplit split: OrcInputFormat.generateSplitsInfo(conf)) {
+ List<InputSplit> result = new ArrayList<InputSplit>(splits.size());
+ for(OrcSplit split: splits) {
result.add(new OrcNewSplit(split));
}
perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.ORC_GET_SPLITS);