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/01/22 19:45:43 UTC
svn commit: r1653999 - in /hive/trunk/shims:
0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java
0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
Author: xuefu
Date: Thu Jan 22 18:45:42 2015
New Revision: 1653999
URL: http://svn.apache.org/r1653999
Log:
HIVE-9440: Folders may not be pruned for Hadoop 2 (Jimmy via Xuefu)
Modified:
hive/trunk/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java
hive/trunk/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
Modified: hive/trunk/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java?rev=1653999&r1=1653998&r2=1653999&view=diff
==============================================================================
--- hive/trunk/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java (original)
+++ hive/trunk/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java Thu Jan 22 18:45:42 2015
@@ -24,6 +24,7 @@ import java.net.InetSocketAddress;
import java.net.MalformedURLException;
import java.net.URI;
import java.net.URL;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.HashMap;
@@ -47,10 +48,13 @@ import org.apache.hadoop.fs.permission.F
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.mapred.ClusterStatus;
+import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.JobInProgress;
import org.apache.hadoop.mapred.JobTracker;
import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.TaskLogServlet;
import org.apache.hadoop.mapred.WebHCatJTShim20S;
import org.apache.hadoop.mapred.lib.TotalOrderPartitioner;
@@ -75,6 +79,39 @@ import org.apache.hadoop.util.VersionInf
public class Hadoop20SShims extends HadoopShimsSecure {
@Override
+ public HadoopShims.CombineFileInputFormatShim getCombineFileInputFormat() {
+ return new CombineFileInputFormatShim() {
+ @Override
+ public RecordReader getRecordReader(InputSplit split,
+ JobConf job, Reporter reporter) throws IOException {
+ throw new IOException("CombineFileInputFormat.getRecordReader not needed.");
+ }
+
+ @Override
+ protected FileStatus[] listStatus(JobConf job) throws IOException {
+ FileStatus[] result = super.listStatus(job);
+ boolean foundDir = false;
+ for (FileStatus stat: result) {
+ if (stat.isDir()) {
+ foundDir = true;
+ break;
+ }
+ }
+ if (!foundDir) {
+ return result;
+ }
+ ArrayList<FileStatus> files = new ArrayList<FileStatus>();
+ for (FileStatus stat: result) {
+ if (!stat.isDir()) {
+ files.add(stat);
+ }
+ }
+ return files.toArray(new FileStatus[files.size()]);
+ }
+ };
+ }
+
+ @Override
public String getTaskAttemptLogUrl(JobConf conf,
String taskTrackerHttpAddress, String taskAttemptId)
throws MalformedURLException {
Modified: hive/trunk/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java?rev=1653999&r1=1653998&r2=1653999&view=diff
==============================================================================
--- hive/trunk/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java (original)
+++ hive/trunk/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java Thu Jan 22 18:45:42 2015
@@ -29,7 +29,7 @@ import java.security.NoSuchAlgorithmExce
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
-import java.util.HashSet;
+import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -68,8 +68,10 @@ import org.apache.hadoop.hdfs.client.Hdf
import org.apache.hadoop.hdfs.protocol.EncryptionZone;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.mapred.ClusterStatus;
+import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.WebHCatJTShim23;
import org.apache.hadoop.mapred.lib.TotalOrderPartitioner;
@@ -129,6 +131,30 @@ public class Hadoop23Shims extends Hadoo
}
@Override
+ public HadoopShims.CombineFileInputFormatShim getCombineFileInputFormat() {
+ return new CombineFileInputFormatShim() {
+ @Override
+ public RecordReader getRecordReader(InputSplit split,
+ JobConf job, Reporter reporter) throws IOException {
+ throw new IOException("CombineFileInputFormat.getRecordReader not needed.");
+ }
+
+ @Override
+ protected List<FileStatus> listStatus(JobContext job) throws IOException {
+ List<FileStatus> result = super.listStatus(job);
+ Iterator<FileStatus> it = result.iterator();
+ while (it.hasNext()) {
+ FileStatus stat = it.next();
+ if (!stat.isFile()) {
+ it.remove();
+ }
+ }
+ return result;
+ }
+ };
+ }
+
+ @Override
public String getTaskAttemptLogUrl(JobConf conf,
String taskTrackerHttpAddress, String taskAttemptId)
throws MalformedURLException {
Modified: hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java?rev=1653999&r1=1653998&r2=1653999&view=diff
==============================================================================
--- hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java (original)
+++ hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java Thu Jan 22 18:45:42 2015
@@ -58,17 +58,6 @@ public abstract class HadoopShimsSecure
static final Log LOG = LogFactory.getLog(HadoopShimsSecure.class);
- @Override
- public HadoopShims.CombineFileInputFormatShim getCombineFileInputFormat() {
- return new CombineFileInputFormatShim() {
- @Override
- public RecordReader getRecordReader(InputSplit split,
- JobConf job, Reporter reporter) throws IOException {
- throw new IOException("CombineFileInputFormat.getRecordReader not needed.");
- }
- };
- }
-
public static class InputSplitShim extends CombineFileSplit {
long shrinkedLength;
boolean _isShrinked;
@@ -342,28 +331,6 @@ public abstract class HadoopShimsSecure
CombineFileSplit cfSplit = split;
return new CombineFileRecordReader(job, cfSplit, reporter, rrClass);
}
-
- @Override
- protected FileStatus[] listStatus(JobConf job) throws IOException {
- FileStatus[] result = super.listStatus(job);
- boolean foundDir = false;
- for (FileStatus stat: result) {
- if (stat.isDir()) {
- foundDir = true;
- break;
- }
- }
- if (!foundDir) {
- return result;
- }
- ArrayList<FileStatus> files = new ArrayList<FileStatus>();
- for (FileStatus stat: result) {
- if (!stat.isDir()) {
- files.add(stat);
- }
- }
- return files.toArray(new FileStatus[files.size()]);
- }
}
@Override