Author: daijy Date: Mon Jun 28 18:08:27 2010 New Revision: 958666 URL: http://svn.apache.org/viewvc?rev=958666&view=rev Log: PIG-1467: order by fail when set "fs.file.impl.disable.cache" to true
Modified: hadoop/pig/trunk/CHANGES.txt hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java Modified: hadoop/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=958666&r1=958665&r2=958666&view=diff ============================================================================== --- hadoop/pig/trunk/CHANGES.txt (original) +++ hadoop/pig/trunk/CHANGES.txt Mon Jun 28 18:08:27 2010 @@ -95,6 +95,8 @@ PIG-1309: Map-side Cogroup (ashutoshc) BUG FIXES +PIG-1467: order by fail when set "fs.file.impl.disable.cache" to true (daijy) + PIG-1463: Replace "bz" with ".bz" in setStoreLocation in PigStorage (zjffdu) PIG-1221: Filter equality does not work for tuples (zjffdu) Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java?rev=958666&r1=958665&r2=958666&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java Mon Jun 28 18:08:27 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/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java?rev=958666&r1=958665&r2=958666&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java Mon Jun 28 18:08:27 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/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java?rev=958666&r1=958665&r2=958666&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java Mon Jun 28 18:08:27 2010 @@ -38,6 +38,7 @@ import org.apache.pig.backend.executione import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore; +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; @@ -78,6 +79,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,