You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ec...@apache.org on 2012/07/14 14:53:11 UTC
svn commit: r1361537 -
/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java
Author: ecapriolo
Date: Sat Jul 14 12:53:10 2012
New Revision: 1361537
URL: http://svn.apache.org/viewvc?rev=1361537&view=rev
Log:
HIVE-3257 Fix avro_joins.q testcase failure when building hive on hadoop0.23. Zhenxiao Luo (via egc)
Modified:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java?rev=1361537&r1=1361536&r2=1361537&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/avro/AvroGenericRecordReader.java Sat Jul 14 12:53:10 2012
@@ -92,7 +92,7 @@ public class AvroGenericRecordReader imp
// that matches our input split.
for (Map.Entry<String,PartitionDesc> pathsAndParts: mapRedWork.getPathToPartitionInfo().entrySet()){
String partitionPath = pathsAndParts.getKey();
- if(pathIsInPartition(split.getPath().makeQualified(fs), partitionPath)) {
+ if(pathIsInPartition(split.getPath(), partitionPath)) {
if(LOG.isInfoEnabled()) {
LOG.info("Matching partition " + partitionPath +
" with input split " + split);
@@ -121,7 +121,13 @@ public class AvroGenericRecordReader imp
}
private boolean pathIsInPartition(Path split, String partitionPath) {
- return split.toString().startsWith(partitionPath);
+ boolean schemeless = split.toUri().getScheme() == null;
+ if (schemeless) {
+ String schemelessPartitionPath = new Path(partitionPath).toUri().getPath();
+ return split.toString().startsWith(schemelessPartitionPath);
+ } else {
+ return split.toString().startsWith(partitionPath);
+ }
}