You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2012/09/27 02:35:01 UTC
svn commit: r1390793 -
/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java
Author: hashutosh
Date: Thu Sep 27 00:35:01 2012
New Revision: 1390793
URL: http://svn.apache.org/viewvc?rev=1390793&view=rev
Log:
HIVE-3483 :joins using partitioned table give incorrect results on windows (Thejas Nair via Ashutosh Chauhan)
Modified:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java?rev=1390793&r1=1390792&r2=1390793&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java Thu Sep 27 00:35:01 2012
@@ -33,6 +33,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<K ex
if (!done) {
if (f == null) {
- f = new CombineFilter(filterPath);
+ f = new CombineFilter(job, filterPath);
LOG.info("CombineHiveInputSplit creating pool for " + path +
"; using filter path " + filterPath);
combine.createPool(job, f);
@@ -393,7 +394,7 @@ public class CombineHiveInputFormat<K ex
if (inpFiles.size() > 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<K ex
}
static class CombineFilter implements PathFilter {
- private final Set<String> pStrings = new HashSet<String>();
+ private final Set<Path> filterPaths = new HashSet<Path>();
+ 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<K ex
}
public void addPath(Path p) {
- String pString = p.toUri().getPath().toString();
- pStrings.add(pString);
+ filterPaths.add(p);
}
// returns true if the specified path matches the prefix stored
@@ -565,7 +567,7 @@ public class CombineHiveInputFormat<K ex
public boolean accept(Path path) {
boolean find = false;
while (path != null && !find) {
- if(pStrings.contains(path.toString())) {
+ if(checkFilterPathContains(filterPaths, path)){
find = true;
break;
}
@@ -574,12 +576,31 @@ public class CombineHiveInputFormat<K ex
return find;
}
+ private boolean checkFilterPathContains(Set<Path> 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();
}