diff --git ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java index 6eb51dd..0aaa250 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java @@ -34,6 +34,7 @@ import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -366,7 +367,7 @@ public class CombineHiveInputFormat 0) { // Processing files for (Path filterPath : poolSet) { - combine.createPool(job, new CombineFilter(filterPath)); + combine.createPool(job, new CombineFilter(job, filterPath)); } processPaths(job, combine, iss, inpFiles.toArray(new Path[0])); } @@ -544,11 +545,13 @@ public class CombineHiveInputFormat pStrings = new HashSet(); + private final Set filterPaths = new HashSet(); + private final Configuration conf; // store a path prefix in this TestFilter // PRECONDITION: p should always be a directory - public CombineFilter(Path p) { + public CombineFilter(Configuration conf, Path p) { + this.conf = conf; // we need to keep the path part only because the Hadoop CombineFileInputFormat will // pass the path part only to accept(). // Trailing the path with a separator to prevent partial matching. @@ -556,8 +559,7 @@ public class CombineHiveInputFormat filterPaths, Path path) { + for(Path filterPath : filterPaths){ + try { + //if path does not have scheme, use the scheme from filterPath + //for comparison + path = path.makeQualified(filterPath.getFileSystem(conf)); + } catch (IOException e) { + //this is not supposed to happen + //if it happens it can lead to incorrect splits being generated + // and files that should not be combined could end up being combied + throw new RuntimeException("Error making path " + path + " qualified ", e); + } + if(filterPath.equals(path)){ + return true; + } + } + return false; + } + @Override public String toString() { StringBuilder s = new StringBuilder(); s.append("PathFilter: "); - for (String pString : pStrings) { - s.append(pString + " "); + for (Path filterPath : filterPaths) { + s.append(filterPath + " "); } return s.toString(); }