You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by da...@apache.org on 2010/06/28 20:05:43 UTC

svn commit: r958665 - in /hadoop/pig/branches/branch-0.7: ./ src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/ src/org/apache/pig/backend/hadoop/executionengine/util/

Author: daijy
Date: Mon Jun 28 18:05:43 2010
New Revision: 958665

URL: http://svn.apache.org/viewvc?rev=958665&view=rev
Log:
PIG-1467: order by fail when set "fs.file.impl.disable.cache" to true

Modified:
    hadoop/pig/branches/branch-0.7/CHANGES.txt
    hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java
    hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java
    hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java

Modified: hadoop/pig/branches/branch-0.7/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.7/CHANGES.txt?rev=958665&r1=958664&r2=958665&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.7/CHANGES.txt (original)
+++ hadoop/pig/branches/branch-0.7/CHANGES.txt Mon Jun 28 18:05:43 2010
@@ -194,6 +194,8 @@ OPTIMIZATIONS
 
 BUG FIXES
 
+PIG-1467: order by fail when set "fs.file.impl.disable.cache" to true (daijy)
+
 PIG-1443: DefaultTuple underestimate the memory footprint for string (daijy)
 
 PIG-1446: https://issues.apache.org/jira/browse/PIG-1446 (hashutosh)

Modified: hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java?rev=958665&r1=958664&r2=958665&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java (original)
+++ hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java Mon Jun 28 18:05:43 2010
@@ -35,6 +35,7 @@ import org.apache.pig.impl.io.NullablePa
 import org.apache.pig.impl.util.Pair;
 import org.apache.pig.data.DefaultTupleFactory;
 import org.apache.pig.data.DataType;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce;
 import org.apache.pig.backend.hadoop.executionengine.util.MapRedUtil;
 
 
@@ -101,6 +102,7 @@ public class SkewedPartitioner extends P
     @Override
     public void setConf(Configuration job) {
         conf = job;
+        PigMapReduce.sJobConf = conf;
         String keyDistFile = job.get("pig.keyDistFile", "");
         if (keyDistFile.length() == 0)
             throw new RuntimeException(this.getClass().getSimpleName() + " used but no key distribution found");

Modified: hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java?rev=958665&r1=958664&r2=958665&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java (original)
+++ hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java Mon Jun 28 18:05:43 2010
@@ -102,7 +102,11 @@ public class WeightedRangePartitioner ex
             
             
             // use local file system to get the quantilesFile
-            Configuration conf = new Configuration(false);            
+            Configuration conf = new Configuration(false);
+            if (configuration.get("fs.file.impl")!=null)
+                conf.set("fs.file.impl", configuration.get("fs.file.impl"));
+            if (configuration.get("fs.hdfs.impl")!=null)
+                conf.set("fs.hdfs.impl", configuration.get("fs.hdfs.impl"));
             conf.set(MapRedUtil.FILE_SYSTEM_NAME, "file:///");
             
             ReadToEndLoader loader = new ReadToEndLoader(new BinStorage(),

Modified: hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java?rev=958665&r1=958664&r2=958665&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java (original)
+++ hadoop/pig/branches/branch-0.7/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java Mon Jun 28 18:05:43 2010
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce;
 import org.apache.pig.builtin.BinStorage;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataType;
@@ -67,6 +68,11 @@ public class MapRedUtil {
 
         // use local file system to get the keyDistFile
         Configuration conf = new Configuration(false);            
+        
+        if (PigMapReduce.sJobConf.get("fs.file.impl")!=null)
+            conf.set("fs.file.impl", PigMapReduce.sJobConf.get("fs.file.impl"));
+        if (PigMapReduce.sJobConf.get("fs.hdfs.impl")!=null)
+            conf.set("fs.hdfs.impl", PigMapReduce.sJobConf.get("fs.hdfs.impl"));
         conf.set(MapRedUtil.FILE_SYSTEM_NAME, "file:///");
 
         ReadToEndLoader loader = new ReadToEndLoader(new BinStorage(), conf,