diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java index 601ad08..899b373 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java @@ -24,11 +24,14 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Properties; import org.apache.commons.lang3.StringEscapeUtils; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.collections4.Predicate; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -378,7 +381,15 @@ public boolean doNext(WritableComparable key, Writable value) throws IOException inputSplits = splitSampling(work.getSplitSample(), inputSplits); } if (inputSplits.length > 0) { - return inputSplits; + LinkedList originSplits = new LinkedList(Arrays.asList(inputSplits)); + CollectionUtils.filter(originSplits, new Predicate() { + @Override + public boolean evaluate(FetchInputFormatSplit split) { + return split.getLength() > 0; + } + }); + + return originSplits.toArray(new FetchInputFormatSplit[originSplits.size()]); } } return null;