Return-Path: Delivered-To: apmail-hadoop-pig-commits-archive@www.apache.org Received: (qmail 79984 invoked from network); 28 Jun 2010 18:09:54 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 28 Jun 2010 18:09:54 -0000 Received: (qmail 82132 invoked by uid 500); 28 Jun 2010 18:09:54 -0000 Delivered-To: apmail-hadoop-pig-commits-archive@hadoop.apache.org Received: (qmail 82120 invoked by uid 500); 28 Jun 2010 18:09:53 -0000 Mailing-List: contact pig-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: pig-dev@hadoop.apache.org Delivered-To: mailing list pig-commits@hadoop.apache.org Received: (qmail 82113 invoked by uid 500); 28 Jun 2010 18:09:53 -0000 Delivered-To: apmail-incubator-pig-commits@incubator.apache.org Received: (qmail 82110 invoked by uid 99); 28 Jun 2010 18:09:53 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 28 Jun 2010 18:09:53 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 28 Jun 2010 18:09:51 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 22AC62388906; Mon, 28 Jun 2010 18:08:28 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r958666 - in /hadoop/pig/trunk: ./ src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/ src/org/apache/pig/backend/hadoop/executionengine/util/ Date: Mon, 28 Jun 2010 18:08:28 -0000 To: pig-commits@incubator.apache.org From: daijy@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20100628180828.22AC62388906@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org 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,