svn commit: r711158 - in /hadoop/pig/branches/types/src/org/apache/pig: backend/hadoop/executionengine/mapReduceLayer/ backend/hadoop/executionengine/physicalLayer/ backend/hadoop/executionengine/phys
Author: gates Date: Mon Nov 3 17:45:47 2008 New Revision: 711158 URL: http://svn.apache.org/viewvc?rev=711158view=rev Log: PIG-350 Patch that redoes join to stream one side of the join instead of materialize keys from all inputs into memory. Patch by Pradeep that built on earlier work done by Daniel. Added: hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POJoinPackage.java hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POOptimizedForEach.java Modified: hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalLauncher.java hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/PhysicalOperator.java hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhyPlanVisitor.java hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java hadoop/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt Modified: hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalLauncher.java URL: http://svn.apache.org/viewvc/hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalLauncher.java?rev=711158r1=711157r2=711158view=diff == --- hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalLauncher.java (original) +++ hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalLauncher.java Mon Nov 3 17:45:47 2008 @@ -18,11 +18,13 @@ import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil; import org.apache.pig.impl.PigContext; import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil; +import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler.CoGroupStreamingOptimizerVisitor; import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.MROperPlan; import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.MRPrinter; import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.MRStreamHandler; import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.POPackageAnnotator; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan; +import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POJoinPackage; import org.apache.pig.impl.plan.PlanException; import org.apache.pig.impl.plan.VisitorException; import org.apache.pig.impl.util.ConfigurationValidator; @@ -127,7 +129,12 @@ // check whether stream operator is present MRStreamHandler checker = new MRStreamHandler(plan); checker.visit(); - + +// optimize joins +CoGroupStreamingOptimizerVisitor cgso = new MRCompiler.CoGroupStreamingOptimizerVisitor(plan, +pc.getProperties().getProperty(join.biggest.input.chunksize, POJoinPackage.DEFAULT_CHUNK_SIZE)); +cgso.visit(); + // figure out the type of the key for the map plan // this is needed when the key is null to create // an appropriate NullableXXXWritable object Modified: hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java URL: http://svn.apache.org/viewvc/hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java?rev=711158r1=711157r2=711158view=diff == --- hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java (original) +++ hadoop/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java Mon Nov 3 17:45:47 2008 @@ -51,6 +51,7 @@ import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.PODistinct; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFilter; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POGlobalRearrange; +import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POJoinPackage; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLimit; import
svn commit: r709206 - in /hadoop/pig/trunk: ./ contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/storage/apachelog/ contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/stora
Author: gates Date: Thu Oct 30 09:56:48 2008 New Revision: 709206 URL: http://svn.apache.org/viewvc?rev=709206view=rev Log: PIG-509: Added CombinedLogLoader, loads logs that were created using Apache's combined log format. Added: hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/storage/apachelog/CombinedLogLoader.java hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/storage/TestCombinedLogLoader.java Modified: hadoop/pig/trunk/CHANGES.txt Modified: hadoop/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=709206r1=709205r2=709206view=diff == --- hadoop/pig/trunk/CHANGES.txt (original) +++ hadoop/pig/trunk/CHANGES.txt Thu Oct 30 09:56:48 2008 @@ -379,3 +379,6 @@ gates). move to hadoop + + PIG-509: Added CombinedLogLoader, loads logs that were created using + Apache's combined log format (spackest via gates). Added: hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/storage/apachelog/CombinedLogLoader.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/storage/apachelog/CombinedLogLoader.java?rev=709206view=auto == --- hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/storage/apachelog/CombinedLogLoader.java (added) +++ hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/storage/apachelog/CombinedLogLoader.java Thu Oct 30 09:56:48 2008 @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the + * NOTICE file distributed with this work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the License); you may not use this file + * except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is + * distributed on an AS IS BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and limitations under the License. + */ + +package org.apache.pig.piggybank.storage.apachelog; + +import java.util.regex.Pattern; + +import org.apache.pig.piggybank.storage.RegExLoader; + +/** + * CombinedLogLoader is used to load logs based on Apache's combined log format, based on a format like + * + * LogFormat %h %l %u %t \%r\ %s %b \%{Referer}i\ \%{User-Agent}i\ combined + * + * The log filename ends up being access_log from a line like + * + * CustomLog logs/combined_log combined + * + * Example: + * + * raw = LOAD 'combined_log' USING org.apache.pig.piggybank.storage.apachelog.CombinedLogLoader AS + * (remoteAddr, remoteLogname, user, time, method, uri, proto, status, bytes, referer, userAgent); + * + */ + +public class CombinedLogLoader extends RegExLoader { +// 1.2.3.4 - - [30/Sep/2008:15:07:53 -0400] GET / HTTP/1.1 200 3190 - +// Mozilla/5.0 (Macintosh; U; Intel Mac OS X 10_5_4; en-us) AppleWebKit/525.18 (KHTML, like Gecko) Version/3.1.2 Safari/525.20.1 +private final static Pattern combinedLogPattern = Pattern + .compile(^(\\S+)\\s+(\\S+)\\s+(\\S+)\\s+.(\\S+\\s+\\S+).\\s+\(\\S+)\\s+(.+?)\\s+(HTTP[^\]+)\\\s+(\\S+)\\s+(\\S+)\\s+\([^\]*)\\\s+\(.*)\$); + +public Pattern getPattern() { +return combinedLogPattern; +} +} Added: hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/storage/TestCombinedLogLoader.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/storage/TestCombinedLogLoader.java?rev=709206view=auto == --- hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/storage/TestCombinedLogLoader.java (added) +++ hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/storage/TestCombinedLogLoader.java Thu Oct 30 09:56:48 2008 @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the + * NOTICE file distributed with this work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the License); you may not use this file + * except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed
svn commit: r709222 - in /hadoop/pig/branches/types: src/org/apache/pig/builtin/DIFF.java test/org/apache/pig/test/TestBuiltin.java
Author: gates Date: Thu Oct 30 11:33:43 2008 New Revision: 709222 URL: http://svn.apache.org/viewvc?rev=709222view=rev Log: PIG-511 Fixed flaws in builtin UDF diff pointed out by Crisitan Ivascu. Modified: hadoop/pig/branches/types/src/org/apache/pig/builtin/DIFF.java hadoop/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java Modified: hadoop/pig/branches/types/src/org/apache/pig/builtin/DIFF.java URL: http://svn.apache.org/viewvc/hadoop/pig/branches/types/src/org/apache/pig/builtin/DIFF.java?rev=709222r1=709221r2=709222view=diff == --- hadoop/pig/branches/types/src/org/apache/pig/builtin/DIFF.java (original) +++ hadoop/pig/branches/types/src/org/apache/pig/builtin/DIFF.java Thu Oct 30 11:33:43 2008 @@ -18,7 +18,9 @@ package org.apache.pig.builtin; import java.io.IOException; +import java.util.HashSet; import java.util.Iterator; +import java.util.Set; import org.apache.pig.EvalFunc; import org.apache.pig.backend.executionengine.ExecException; @@ -33,8 +35,6 @@ * will emit any Tuples that are in on of the DataBags but not the other. If the * fields are values, it will emit tuples with values that do not match. * - * @author breed - * */ public class DIFF extends EvalFuncDataBag { TupleFactory mTupleFactory = TupleFactory.getInstance(); @@ -78,48 +78,20 @@ DataBag bag1, DataBag bag2, DataBag emitTo) { -// Create two distinct versions of the bag. This will speed up -// comparison, and provide us a sorted order so we don't have to do -// an n^2 lookup. -DataBag d1 = mBagFactory.newDistinctBag(); -DataBag d2 = mBagFactory.newDistinctBag(); -IteratorTuple i1 = d1.iterator(); -IteratorTuple i2 = d2.iterator(); -while (i1.hasNext()) d1.add(i1.next()); -while (i2.hasNext()) d2.add(i2.next()); - -i1 = d1.iterator(); -i2 = d2.iterator(); - -Tuple t1 = i1.next(); -Tuple t2 = i2.next(); - -while (i1.hasNext() i2.hasNext()) { -int c = t1.compareTo(t2); - -if (c 0) { -// place t1 in the result bag and advance i1 -emitTo.add(t1); -t1 = i1.next(); -} else if (c 0) { -// place t2 in the result bag and advance i2 -emitTo.add(t2); -t2 = i2.next(); -} else if (c == 0) { -// put neither in the result bag, advance both iterators -t1 = i1.next(); -t2 = i2.next(); -} -} +// Build two hash tables and probe with first one, then the other. +// This does make the assumption that the distinct set of keys from +// each bag will fit in memory. +SetTuple s1 = new HashSetTuple(); +IteratorTuple i1 = bag1.iterator(); +while (i1.hasNext()) s1.add(i1.next()); + +SetTuple s2 = new HashSetTuple(); +IteratorTuple i2 = bag2.iterator(); +while (i2.hasNext()) s2.add(i2.next()); + +for (Tuple t : s1) if (!s2.contains(t)) emitTo.add(t); +for (Tuple t : s2) if (!s1.contains(t)) emitTo.add(t); -// One ran out, put all the rest of the other (if there are any) in -// the result bag. -while (i1.hasNext()) { -emitTo.add(i1.next()); -} -while (i2.hasNext()) { -emitTo.add(i2.next()); -} } Modified: hadoop/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java URL: http://svn.apache.org/viewvc/hadoop/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java?rev=709222r1=709221r2=709222view=diff == --- hadoop/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java (original) +++ hadoop/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java Thu Oct 30 11:33:43 2008 @@ -19,6 +19,7 @@ import java.io.File; import java.io.PrintWriter; +import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -890,6 +891,44 @@ assertTrue(f1.equals(f2)); } + +@Test +public void testDIFF() throws Exception { +// Test it in the case with two bags. +BagFactory bf = BagFactory.getInstance(); +TupleFactory tf = TupleFactory.getInstance(); + +DataBag b1 = bf.newDefaultBag(); +DataBag b2 = bf.newDefaultBag(); +for (int i = 0; i 10; i++) b1.add(tf.newTuple(new Integer(i))); +for (int i = 0; i 10; i += 2) b2.add(tf.newTuple(new Integer(i))); +Tuple t = tf.newTuple(2); +t.set(0, b1); +t.set(1, b2); +DIFF d = new DIFF(); +DataBag result = d.exec(t); + +assertEquals(5, result.size()); +IteratorTuple i = result.iterator(); +int
svn commit: r707420 - in /incubator/pig/trunk: ./ contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/ contrib/piggybank/java/src/test/java/org/apache/pig/pig
Author: gates Date: Thu Oct 23 10:31:19 2008 New Revision: 707420 URL: http://svn.apache.org/viewvc?rev=707420view=rev Log: PIG-503: Changed default date format for DateExtractor. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/DateExtractor.java incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/apachelogparser/TestDateExtractor.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=707420r1=707419r2=707420view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Thu Oct 23 10:31:19 2008 @@ -374,3 +374,6 @@ PIG-476: Added DateExtractor, a piggybank eval func that extracts a date from a string (spackest via gates). + + PIG-503: Changed default date format for DateExtractor (spackest via + gates). Modified: incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/DateExtractor.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/DateExtractor.java?rev=707420r1=707419r2=707420view=diff == --- incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/DateExtractor.java (original) +++ incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/DateExtractor.java Thu Oct 23 10:31:19 2008 @@ -23,7 +23,7 @@ /** * DateExtractor has three different constructors which each allow for different functionality. The - * incomingDateFormat (MM-dd- by default) is used to match the date string that gets passed in from the + * incomingDateFormat (-MM-dd by default) is used to match the date string that gets passed in from the * log. The outgoingDateFormat (dd/MMM/:HH:mm:ss Z by default) is used to format the returned string. * * Different constructors exist for each combination; please use the appropriate respective constructor. @@ -34,13 +34,13 @@ * * From pig latin you will need to use aliases to use a non-default format, like * - * define MyDateExtractor org.apache.pig.piggybank.evaluation.util.apachelogparser.DateExtractor(MM-); + * define MyDateExtractor org.apache.pig.piggybank.evaluation.util.apachelogparser.DateExtractor(-MM); * * A = FOREACH row GENERATE DateExtractor(dayTime); */ public class DateExtractor extends EvalFuncDataAtom { private static SimpleDateFormat DEFAULT_INCOMING_DATE_FORMAT = new SimpleDateFormat(dd/MMM/:HH:mm:ss Z); -private static SimpleDateFormat DEFAULT_OUTGOING_DATE_FORMAT = new SimpleDateFormat(MM-dd-); +private static SimpleDateFormat DEFAULT_OUTGOING_DATE_FORMAT = new SimpleDateFormat(-MM-dd); private SimpleDateFormat incomingDateFormat; private SimpleDateFormat outgoingDateFormat; @@ -85,8 +85,7 @@ try { date = incomingDateFormat.parse(incomingDateString); } catch (ParseException e) { -System.err.println(Unable to parse incoming date string + -incomingDateString + , + e.getMessage()); +System.err.println(Unable to parse incoming date string + incomingDateString + , + e.getMessage()); return; } Modified: incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/apachelogparser/TestDateExtractor.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/apachelogparser/TestDateExtractor.java?rev=707420r1=707419r2=707420view=diff == --- incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/apachelogparser/TestDateExtractor.java (original) +++ incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/apachelogparser/TestDateExtractor.java Thu Oct 23 10:31:19 2008 @@ -36,7 +36,7 @@ Tuple input = new Tuple(new DataAtom(20/Sep/2008:23:53:04 -0600)); DataAtom output = new DataAtom(); dayExtractor.exec(input, output); -assertEquals(09-20-2008, output.toString()); +assertEquals(2008-09-20, output.toString()); } @Test @@ -51,7 +51,7 @@ input = new Tuple(new DataAtom(20/Sep/2008:23:53:04 -0600)); output = new DataAtom(); dayExtractor.exec(input, output); -assertEquals(09-20-2008, output.toString
svn commit: r705004 - /incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOLoad.java
Author: gates Date: Wed Oct 15 12:36:58 2008 New Revision: 705004 URL: http://svn.apache.org/viewvc?rev=705004view=rev Log: Removed bogus import from LOLoad. Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOLoad.java Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOLoad.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOLoad.java?rev=705004r1=705003r2=705004view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOLoad.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOLoad.java Wed Oct 15 12:36:58 2008 @@ -35,8 +35,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import sun.awt.motif.MDataTransferer; - public class LOLoad extends LogicalOperator { private static final long serialVersionUID = 2L; protected boolean splittable = true;
svn commit: r704589 - in /incubator/pig/trunk: ./ contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/ contrib/piggybank/java/src/test/java/org/apache/pig/pig
Author: gates Date: Tue Oct 14 10:28:16 2008 New Revision: 704589 URL: http://svn.apache.org/viewvc?rev=704589view=rev Log: PIG-476: Added DateExtractor, a piggybank eval func that extracts a date from a string. Added: incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/DateExtractor.java incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/apachelogparser/TestDateExtractor.java Modified: incubator/pig/trunk/CHANGES.txt Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=704589r1=704588r2=704589view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Tue Oct 14 10:28:16 2008 @@ -371,3 +371,6 @@ PIG-488: Added SearchTermExtractor, a piggybank eval func that, for many search engines, recognizes the search term in the URL returns it to the caller (spackest via gates). + + PIG-476: Added DateExtractor, a piggybank eval func that extracts a date + from a string (spackest via gates). Added: incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/DateExtractor.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/DateExtractor.java?rev=704589view=auto == --- incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/DateExtractor.java (added) +++ incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/DateExtractor.java Tue Oct 14 10:28:16 2008 @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the + * NOTICE file distributed with this work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the License); you may not use this file + * except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is + * distributed on an AS IS BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and limitations under the License. + */ + +package org.apache.pig.piggybank.evaluation.util.apachelogparser; + +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Date; + +import org.apache.pig.EvalFunc; +import org.apache.pig.data.DataAtom; +import org.apache.pig.data.Tuple; + +/** + * DateExtractor has three different constructors which each allow for different functionality. The + * incomingDateFormat (MM-dd- by default) is used to match the date string that gets passed in from the + * log. The outgoingDateFormat (dd/MMM/:HH:mm:ss Z by default) is used to format the returned string. + * + * Different constructors exist for each combination; please use the appropriate respective constructor. + * + * Note that any data that exists in the SimpleDateFormat schema can be supported. For example, if you were + * starting with the default incoming format and wanted to extract just the year, you would use the single + * string constructor DateExtractor(). + * + * From pig latin you will need to use aliases to use a non-default format, like + * + * define MyDateExtractor org.apache.pig.piggybank.evaluation.util.apachelogparser.DateExtractor(MM-); + * + * A = FOREACH row GENERATE DateExtractor(dayTime); + */ +public class DateExtractor extends EvalFuncDataAtom { +private static SimpleDateFormat DEFAULT_INCOMING_DATE_FORMAT = new SimpleDateFormat(dd/MMM/:HH:mm:ss Z); +private static SimpleDateFormat DEFAULT_OUTGOING_DATE_FORMAT = new SimpleDateFormat(MM-dd-); + +private SimpleDateFormat incomingDateFormat; +private SimpleDateFormat outgoingDateFormat; + +/** + * forms the formats based on default incomingDateFormat and default outgoingDateFormat + * + * @param outgoingDateString outgoingDateFormat is based on outgoingDateString + */ +public DateExtractor() { +incomingDateFormat = DEFAULT_INCOMING_DATE_FORMAT; +outgoingDateFormat = DEFAULT_OUTGOING_DATE_FORMAT; +} + +/** + * forms the formats based on passed outgoingDateString and the default incomingDateFormat + * + * @param outgoingDateString outgoingDateFormat is based on outgoingDateString + */ +public DateExtractor
svn commit: r704149 - in /incubator/pig/trunk: ./ contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/ contrib/piggybank/java/src/test/java/org/apache/pig/pig
Author: gates Date: Mon Oct 13 09:18:01 2008 New Revision: 704149 URL: http://svn.apache.org/viewvc?rev=704149view=rev Log: PIG-486: Added SearchEngineExtractor, a piggybank eval func that recognizes a set of the most common search engines in a URL and extracts the name of the search engine. Added: incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/ incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/SearchEngineExtractor.java incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/ incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/apachelogparser/ incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/apachelogparser/TestSearchEngineExtractor.java Modified: incubator/pig/trunk/CHANGES.txt Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=704149r1=704148r2=704149view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Mon Oct 13 09:18:01 2008 @@ -361,3 +361,6 @@ PIG-474: Added MyRegexLoader, a subclass of RegExLoader, to piggybank (spackest via gates) +PIG-486: Added SearchEngineExtractor, a piggybank eval func that + recognizes a set of the most common search engines in a URL and extracts + the name of the search engine (spackest via gates). Added: incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/SearchEngineExtractor.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/SearchEngineExtractor.java?rev=704149view=auto == --- incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/SearchEngineExtractor.java (added) +++ incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/SearchEngineExtractor.java Mon Oct 13 09:18:01 2008 @@ -0,0 +1,405 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the + * NOTICE file distributed with this work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the License); you may not use this file + * except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is + * distributed on an AS IS BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and limitations under the License. + */ + +package org.apache.pig.piggybank.evaluation.util.apachelogparser; + +import java.net.URL; +import java.util.HashMap; + +import org.apache.pig.EvalFunc; +import org.apache.pig.data.DataAtom; +import org.apache.pig.data.Tuple; + +/** + * SearchEngineExtractor takes a url string and extracts the search engine. For example, given + * + * http://www.google.com/search?hl=ensafe=activerls=GGLG,GGLG:2005-24,GGLG:enq=purpose+of+lifebtnG=Search + * + * then + * + * Google + * + * would be extracted. + * + * From pig latin, usage looks something like + * + * searchEngine = FOREACH row GENERATE + * org.apache.pig.piggybank.evaluation.util.apachelogparser.SearchEngineExtractor(referer); + * + * Supported search engines include abacho.com, alice.it, alltheweb.com, altavista.com, aolsearch.aol.com, + * as.starware.com, ask.com, blogs.icerocket.com, blogsearch.google.com, blueyonder.co.uk, busca.orange.es, + * buscador.lycos.es, buscador.terra.es, buscar.ozu.es, categorico.it, cuil.com, excite.com, excite.it, + * fastweb.it, feedster.com, godado.com, godado.it, google.ad, google.ae, google.af, google.ag, google.am, + * google.as, google.at, google.az, google.ba, google.be, google.bg, google.bi, google.biz, google.bo, + * google.bs, google.bz, google.ca, google.cc, google.cd, google.cg, google.ch, google.ci, google.cl, + * google.cn, google.co.at , google.co.bi, google.co.bw, google.co.ci, google.co.ck, google.co.cr, + * google.co.gg, google.co.gl, google.co.gy, google.co.hu, google.co.id, google.co.il, google.co.im, + * google.co.in, google.co.it, google.co.je, google.co.jp, google.co.ke, google.co.kr, google.co.ls, + * google.co.ma, google.co.mu, google.co.mw, google.co.nz, google.co.pn, google.co.th, google.co.tt
svn commit: r704151 - in /incubator/pig/trunk: ./ contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/ contrib/piggybank/java/src/test/java/org/apache/pig/pig
Author: gates Date: Mon Oct 13 09:20:49 2008 New Revision: 704151 URL: http://svn.apache.org/viewvc?rev=704151view=rev Log: PIG-487: Added HostExtractor, a piggybank eval func that, given a URL, determines the host. Added: incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/HostExtractor.java incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/apachelogparser/TestHostExtractor.java Modified: incubator/pig/trunk/CHANGES.txt Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=704151r1=704150r2=704151view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Mon Oct 13 09:20:49 2008 @@ -364,3 +364,6 @@ PIG-486: Added SearchEngineExtractor, a piggybank eval func that recognizes a set of the most common search engines in a URL and extracts the name of the search engine (spackest via gates). + +PIG-487: Added HostExtractor, a piggybank eval func that, given a URL, + determines the host (spackest via gates). Added: incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/HostExtractor.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/HostExtractor.java?rev=704151view=auto == --- incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/HostExtractor.java (added) +++ incubator/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/util/apachelogparser/HostExtractor.java Mon Oct 13 09:20:49 2008 @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the + * NOTICE file distributed with this work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the License); you may not use this file + * except in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is + * distributed on an AS IS BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and limitations under the License. + */ + +/* + * HostExtractor takes a url and returns the host. For example, + * + * http://sports.espn.go.com/mlb/recap?gameId=281009122 + * + * leads to + * + * sports.espn.go.com + * + * Pig latin usage looks like + * + * host = FOREACH row GENERATE + * org.apache.pig.piggybank.evaluation.util.apachelogparser.HostExtractor(referer); + */ + +package org.apache.pig.piggybank.evaluation.util.apachelogparser; + + +import java.net.URL; + +import org.apache.pig.EvalFunc; +import org.apache.pig.data.DataAtom; +import org.apache.pig.data.Tuple; + +public class HostExtractor extends EvalFuncDataAtom { +@Override +public void exec(Tuple input, DataAtom output) { +String string = input.getAtomField(0).strval(); + +if (string == null) +return; + +String host = null; +try { +host = new URL(string).getHost().toLowerCase(); +} catch (Exception e) { +} +if (host != null) +output.setValue(host); +} +} Added: incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/apachelogparser/TestHostExtractor.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/apachelogparser/TestHostExtractor.java?rev=704151view=auto == --- incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/apachelogparser/TestHostExtractor.java (added) +++ incubator/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/util/apachelogparser/TestHostExtractor.java Mon Oct 13 09:20:49 2008 @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the + * NOTICE file distributed with this work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the License); you may not use this file + * except in compliance with the License. You may obtain a copy of the License
svn commit: r699534 - in /incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer: Launcher.java LocalLauncher.java MapReduceLauncher.java
Author: gates Date: Fri Sep 26 17:00:03 2008 New Revision: 699534 URL: http://svn.apache.org/viewvc?rev=699534view=rev Log: PIG-457 This just addresses the issue of getting false error messages on success. Launchers changed to send any error messages from hadoop to debug if the job succeeded. Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/Launcher.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalLauncher.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/Launcher.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/Launcher.java?rev=699534r1=699533r2=699534view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/Launcher.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/Launcher.java Fri Sep 26 17:00:03 2008 @@ -85,13 +85,13 @@ return (int)(Math.ceil(prog)) == (int)1; } -protected void getStats(Job job, JobClient jobClient) throws IOException{ +protected void getStats(Job job, JobClient jobClient, boolean errNotDbg) throws IOException{ JobID MRJobID = job.getAssignedJobID(); TaskReport[] mapRep = jobClient.getMapTaskReports(MRJobID); -getErrorMessages(mapRep, map); +getErrorMessages(mapRep, map, errNotDbg); totalHadoopTimeSpent += computeTimeSpent(mapRep); TaskReport[] redRep = jobClient.getReduceTaskReports(MRJobID); -getErrorMessages(redRep, reduce); +getErrorMessages(redRep, reduce, errNotDbg); totalHadoopTimeSpent += computeTimeSpent(mapRep); } @@ -103,13 +103,18 @@ return timeSpent; } -protected void getErrorMessages(TaskReport reports[], String type) +protected void getErrorMessages(TaskReport reports[], String type, boolean errNotDbg) { for (int i = 0; i reports.length; i++) { String msgs[] = reports[i].getDiagnostics(); for (int j = 0; j msgs.length; j++) { -log.error(Error message from task ( + type + ) + -reports[i].getTaskID() + msgs[j]); +if (errNotDbg) { +log.error(Error message from task ( + type + ) + +reports[i].getTaskID() + msgs[j]); +} else { +log.debug(Error message from task ( + type + ) + +reports[i].getTaskID() + msgs[j]); +} } } } Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalLauncher.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalLauncher.java?rev=699534r1=699533r2=699534view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalLauncher.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalLauncher.java Fri Sep 26 17:00:03 2008 @@ -20,6 +20,7 @@ import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil; import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.MROperPlan; import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.MRPrinter; +import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.MRStreamHandler; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan; import org.apache.pig.impl.plan.PlanException; import org.apache.pig.impl.plan.VisitorException; @@ -70,15 +71,16 @@ log.error(Map reduce job failed); for (Job fj : failedJobs) { log.error(fj.getMessage()); -getStats(fj, jobClient); +getStats(fj, jobClient, true); } +jc.stop(); return false; } ListJob succJobs = jc.getSuccessfulJobs(); if(succJobs!=null) for(Job job : succJobs){ -getStats(job,jobClient); +getStats(job,jobClient, false); } jc.stop(); @@ -111,6 +113,11 @@ CombinerOptimizer co = new CombinerOptimizer(plan); co.visit(); } + +// check whether stream operator is present +MRStreamHandler checker = new MRStreamHandler(plan
svn commit: r699170 - in /incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer: JobControlCompiler.java MRCompiler.java MapReduceOper.java
Author: gates Date: Thu Sep 25 21:17:28 2008 New Revision: 699170 URL: http://svn.apache.org/viewvc?rev=699170view=rev Log: PIG-461 Changed additional last limit to use sort comparator when the limit follows an order by. Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceOper.java Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java?rev=699170r1=699169r2=699170view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java Thu Sep 25 21:17:28 2008 @@ -366,9 +366,13 @@ jobConf.setOutputValueClass(NullableTuple.class); } -if(mro.isGlobalSort()){ -jobConf.set(pig.quantilesFile, mro.getQuantFile()); -jobConf.setPartitionerClass(SortPartitioner.class); +if(mro.isGlobalSort() || mro.isLimitAfterSort()){ +// Only set the quantiles file and sort partitioner if we're a +// global sort, not for limit after sort. +if (mro.isGlobalSort()) { +jobConf.set(pig.quantilesFile, mro.getQuantFile()); +jobConf.setPartitionerClass(SortPartitioner.class); +} if(mro.UDFs.size()==1){ String compFuncSpec = mro.UDFs.get(0); Class comparator = PigContext.resolveClassName(compFuncSpec); @@ -469,9 +473,10 @@ // raw comparator. // An operator has an order by if global sort is set or if it's successor has -// global sort set (because in that case it's the sampling job). +// global sort set (because in that case it's the sampling job) or if +// it's a limit after a sort. boolean hasOrderBy = false; -if (mro.isGlobalSort()) { +if (mro.isGlobalSort() || mro.isLimitAfterSort()) { hasOrderBy = true; } else { ListMapReduceOper succs = plan.getSuccessors(mro); Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java?rev=699170r1=699169r2=699170view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java Thu Sep 25 21:17:28 2008 @@ -1302,6 +1302,11 @@ st.setSFile(oldSpec); limitAdjustMROp.reducePlan.addAsLeaf(st); limitAdjustMROp.requestedParallelism = -1; +// If the operator we're following has global sort set, we +// need to indicate that this is a limit after a sort. +// This will assure that we get the right sort comparator +// set. Otherwise our order gets wacked (PIG-461). +if (mr.isGlobalSort()) limitAdjustMROp.setLimitAfterSort(true); ListMapReduceOper successorList = MRPlan.getSuccessors(mr); MapReduceOper successors[] = null; Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceOper.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceOper.java?rev=699170r1=699169r2=699170view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceOper.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceOper.java Thu Sep 25 21:17:28 2008 @@ -77,6 +77,9 @@ //Indicates if this job is an order by job boolean globalSort = false; +// Indicates if this is a limit after a sort +boolean limitAfterSort = false; + // If true, putting an identity combine
svn commit: r697229 [2/2] - in /incubator/pig/branches/types: src/org/apache/pig/ src/org/apache/pig/backend/hadoop/ src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/ src/org/apache/pi
Modified: incubator/pig/branches/types/src/org/apache/pig/impl/io/NullableBooleanWritable.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/io/NullableBooleanWritable.java?rev=697229r1=697228r2=697229view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/io/NullableBooleanWritable.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/io/NullableBooleanWritable.java Fri Sep 19 13:39:31 2008 @@ -3,97 +3,25 @@ */ package org.apache.pig.impl.io; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - import org.apache.hadoop.io.BooleanWritable; /** * */ -public class NullableBooleanWritable extends BooleanWritable { +public class NullableBooleanWritable extends PigNullableWritable { -private boolean isNull = false; - -public static byte NULL = 0x00; -public static byte NOTNULL = 0x01; - - - -/** - * - */ public NullableBooleanWritable() { -super(); +mValue = new BooleanWritable(); } /** * @param value */ public NullableBooleanWritable(boolean value) { -super(value); -} - -/* (non-Javadoc) - * @see org.apache.hadoop.io.IntWritable#compareTo(java.lang.Object) - */ -@Override -public int compareTo(Object o) { -// if both are null they are equal only here! -if(isNull == true ((NullableBooleanWritable)o).isNull()) -return 0; -else if(isNull == true) -return -1; -else if (((NullableBooleanWritable)o).isNull()) -return 1; -else -return super.compareTo(o); +mValue = new BooleanWritable(value); } -/* (non-Javadoc) - * @see org.apache.hadoop.io.IntWritable#readFields(java.io.DataInput) - */ -@Override -public void readFields(DataInput in) throws IOException { -byte nullMarker = in.readByte(); -if(nullMarker == NULL) { -isNull = true; -} -else { -isNull = false; -super.readFields(in); -} - -} - -/* (non-Javadoc) - * @see org.apache.hadoop.io.IntWritable#write(java.io.DataOutput) - */ -@Override -public void write(DataOutput out) throws IOException { -if(isNull()) { -out.writeByte(NULL); -} else { -out.writeByte(NOTNULL); -super.write(out); -} +public Object getValueAsPigType() { +return isNull() ? null : ((BooleanWritable)mValue).get(); } - -/** - * @return the isNull - */ -public boolean isNull() { -return isNull; -} - -/** - * @param isNull the isNull to set - */ -public void setNull(boolean isNull) { -this.isNull = isNull; -} - - - } Modified: incubator/pig/branches/types/src/org/apache/pig/impl/io/NullableBytesWritable.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/io/NullableBytesWritable.java?rev=697229r1=697228r2=697229view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/io/NullableBytesWritable.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/io/NullableBytesWritable.java Fri Sep 19 13:39:31 2008 @@ -17,96 +17,28 @@ */ package org.apache.pig.impl.io; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - import org.apache.hadoop.io.BytesWritable; +import org.apache.pig.data.DataByteArray; + /** * */ -public class NullableBytesWritable extends BytesWritable { +public class NullableBytesWritable extends PigNullableWritable { -private boolean isNull = false; - -public static byte NULL = 0x00; -public static byte NOTNULL = 0x01; - - -/** - * - */ public NullableBytesWritable() { -super(); +mValue = new BytesWritable(); } /** * @param bytes */ public NullableBytesWritable(byte[] bytes) { -super(bytes); -} - -/* (non-Javadoc) - * @see org.apache.hadoop.io.IntWritable#compareTo(java.lang.Object) - */ -@Override -public int compareTo(Object o) { -// if both are null they are equal only here! -if(isNull == true ((NullableBytesWritable)o).isNull()) -return 0; -else if(isNull == true) -return -1; -else if (((NullableBytesWritable)o).isNull()) -return 1; -else -return super.compareTo(o); -} - -/* (non-Javadoc) - * @see org.apache.hadoop.io.IntWritable#readFields(java.io.DataInput) - */ -@Override -public void readFields(DataInput in) throws IOException { -byte
svn commit: r695659 - in /incubator/pig/branches/types/src/org/apache/pig: backend/hadoop/executionengine/mapReduceLayer/ backend/hadoop/executionengine/physicalLayer/ backend/hadoop/executionengine/p
Author: gates Date: Mon Sep 15 15:10:46 2008 New Revision: 695659 URL: http://svn.apache.org/viewvc?rev=695659view=rev Log: Fixing documentation issues. Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/package.html incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPostCombinerPackage.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LODistinct.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOFilter.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOForEach.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOLimit.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOStream.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOUnion.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOVisitor.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java incubator/pig/branches/types/src/org/apache/pig/impl/streaming/DefaultInputHandler.java incubator/pig/branches/types/src/org/apache/pig/impl/streaming/DefaultOutputHandler.java incubator/pig/branches/types/src/org/apache/pig/impl/streaming/ExecutableManager.java incubator/pig/branches/types/src/org/apache/pig/impl/streaming/FileInputHandler.java incubator/pig/branches/types/src/org/apache/pig/impl/streaming/FileOutputHandler.java incubator/pig/branches/types/src/org/apache/pig/impl/streaming/StreamingCommand.java Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java?rev=695659r1=695658r2=695659view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputFormat.java Mon Sep 15 15:10:46 2008 @@ -109,8 +109,7 @@ * We only care about the values, so we are going to skip the keys when * we write. * - * @see org.apache.hadoop.mapred.RecordWriter#write(org.apache.hadoop.io.WritableComparable, - * org.apache.hadoop.io.Writable) + * @see org.apache.hadoop.mapred.RecordWriter#write(K, V) */ public void write(WritableComparable key, Tuple value) throws IOException { Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/package.html URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/package.html?rev=695659r1=695658r2=695659view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/package.html (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/package.html Mon Sep 15 15:10:46 2008 @@ -11,8 +11,8 @@ provided by the [EMAIL PROTECTED] org.apache.pig.impl.plan} package. p As with [EMAIL PROTECTED] org.apache.pig.impl.logicalLayer}, physical operators consist -of [EMAIL PROTECTED] org.apache.pig.impl.physicalLayer.relationalOperators} and [EMAIL PROTECTED] org.apache.pig.impl.physicalLayer.expressionOperators}. In many data +of [EMAIL PROTECTED] org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators} and [EMAIL PROTECTED] org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators}. In many data processing systems relational operators and expression operators are modeled as different entities because they behave differently. Pig blurs, though does not entirely remove, this distinction because of its support for nested Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPostCombinerPackage.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPostCombinerPackage.java?rev=695659r1=695658r2=695659view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPostCombinerPackage.java (original) +++ incubator/pig/branches/types
svn commit: r694112 - /incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC17.gld
Author: gates Date: Wed Sep 10 21:43:39 2008 New Revision: 694112 URL: http://svn.apache.org/viewvc?rev=694112view=rev Log: Fixed expected results for MRCompiler testLimit. Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC17.gld Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC17.gld URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC17.gld?rev=694112r1=694111r2=694112view=diff == --- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC17.gld (original) +++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC17.gld Wed Sep 10 21:43:39 2008 @@ -1,14 +1,14 @@ MapReduce(-1) - -174: | Store(DummyFil:DummyLdr) - -7856319821130535798 | | -| |---New For Each(false)[bag] - -180 +| |---New For Each(true)[bag] - -180 | | | -| | Project[tuple][0] - -179 +| | Project[tuple][1] - -179 | | | |---Limit - -178 | | | |---Package[tuple]{tuple} - -177 -| Local Rearrange[tuple]{tuple} - -176 +| Local Rearrange[tuple]{tuple}(false) - -176 | | | | | Project[tuple][*] - -175 | |
svn commit: r693303 - in /incubator/pig/branches/types: src/org/apache/pig/impl/builtin/ShellBagEvalFunc.java test/org/apache/pig/test/TestBuiltin.java
Author: gates Date: Mon Sep 8 15:30:40 2008 New Revision: 693303 URL: http://svn.apache.org/viewvc?rev=693303view=rev Log: PIG-327 Removed ShellBagEvalFunc, as it's functionality is replaced by streaming. Removed: incubator/pig/branches/types/src/org/apache/pig/impl/builtin/ShellBagEvalFunc.java Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java?rev=693303r1=693302r2=693303view=diff == --- incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java (original) +++ incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java Mon Sep 8 15:30:40 2008 @@ -37,7 +37,6 @@ import org.apache.pig.backend.executionengine.ExecException; import org.apache.pig.builtin.*; import org.apache.pig.data.*; -import org.apache.pig.impl.builtin.ShellBagEvalFunc; import org.apache.pig.impl.io.FileLocalizer; import org.apache.pig.impl.io.BufferedPositionedInputStream; import org.apache.pig.impl.PigContext; @@ -892,69 +891,6 @@ assertTrue(f1.equals(f2)); } -/[EMAIL PROTECTED] -public void testShellFuncSingle() throws Throwable { -//ShellBagEvalFunc func = new ShellBagEvalFunc(tr o 0); -PigServer pig = new PigServer(initString); - -File tempFile = File.createTempFile(tmp, .dat); -PrintWriter writer = new PrintWriter(tempFile); -writer.println(foo); -writer.println(boo); -writer.close(); - -pig.registerFunction(myTr, new FuncSpec(ShellBagEvalFunc.class.getName() + ('tr o 0'))); -pig.registerQuery(a = load 'file: + tempFile + ';); -pig.registerQuery(b = foreach a generate myTr(*);); -IteratorTuple iter = pig.openIterator(b); - -Tuple t; - -assertTrue(iter.hasNext()); -t = iter.next(); -assertEquals({(f00)}, t.get(0).toString()); -assertTrue(iter.hasNext()); -t = iter.next(); -assertEquals({(b00)}, t.get(0).toString()); -assertFalse(iter.hasNext()); -tempFile.delete(); -} - -@Test -public void testShellFuncMultiple() throws Throwable { - -PigServer pig = new PigServer(initString); -final int numTimes = 100; - -File tempFile = File.createTempFile(tmp, .dat); -PrintWriter writer = new PrintWriter(tempFile); -for (int i=0; i numTimes; i++){ -writer.println(i+oo); -} -writer.close(); - -pig.registerFunction(tr1, -new FuncSpec(ShellBagEvalFunc.class.getName() + ('tr o A'))); -pig.registerFunction(tr2, -new FuncSpec(ShellBagEvalFunc.class.getName() + ('tr o B'))); -pig.registerQuery(a = load 'file: + tempFile + ';); -pig.registerQuery(b = foreach a generate tr1(*),tr2(*);); -IteratorTuple iter = pig.openIterator(b); - -for (int i=0; i numTimes; i++){ -Tuple t = iter.next(); -DataBag b = DataType.toBag(t.get(0)); -Tuple t1 = b.iterator().next(); -assertEquals(i+AA, t1.get(0).toString()); -b = DataType.toBag(t.get(1)); -t1 = b.iterator().next(); -assertEquals(i+BB, t1.get(0).toString()); -} - -assertFalse(iter.hasNext()); -tempFile.delete(); -}*/ - private static String getInputType(String typeFor) { return allowedInput.get(typeFor); }
svn commit: r692581 - /incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/ImplicitSplitInserter.java
Author: gates Date: Fri Sep 5 16:36:33 2008 New Revision: 692581 URL: http://svn.apache.org/viewvc?rev=692581view=rev Log: PIG-401 ImplicitSplitInserter class that should have been in the last checkin. Added: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/ImplicitSplitInserter.java Added: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/ImplicitSplitInserter.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/ImplicitSplitInserter.java?rev=692581view=auto == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/ImplicitSplitInserter.java (added) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/ImplicitSplitInserter.java Fri Sep 5 16:36:33 2008 @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.pig.impl.logicalLayer.optimizer; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.pig.data.DataType; +import org.apache.pig.impl.logicalLayer.LOConst; +import org.apache.pig.impl.logicalLayer.LOSplitOutput; +import org.apache.pig.impl.logicalLayer.LogicalOperator; +import org.apache.pig.impl.logicalLayer.LogicalPlan; +import org.apache.pig.impl.logicalLayer.LOSplit; +import org.apache.pig.impl.plan.DepthFirstWalker; +import org.apache.pig.impl.plan.NodeIdGenerator; +import org.apache.pig.impl.plan.OperatorKey; +import org.apache.pig.impl.plan.optimizer.OptimizerException; + +public class ImplicitSplitInserter extends LogicalTransformer { + +public ImplicitSplitInserter(LogicalPlan plan) { +super(plan, new DepthFirstWalkerLogicalOperator, LogicalPlan(plan)); +} + +@Override +public boolean check(ListLogicalOperator nodes) throws OptimizerException { +// Look to see if this is a non-split node with two outputs. If so +// it matches. +LogicalOperator op = nodes.get(0); +ListLogicalOperator succs = mPlan.getSuccessors(op); +if (succs == null || succs.size() 2) return false; +if (op instanceof LOSplit) return false; +return true; +} + +@Override +public void transform(ListLogicalOperator nodes) +throws OptimizerException { +// Insert a split and its corresponding SplitOutput nodes into the plan +// between node 0 and 1 / 2. +String scope = nodes.get(0).getOperatorKey().scope; +NodeIdGenerator idGen = NodeIdGenerator.getGenerator(); +LOSplit splitOp = new LOSplit(mPlan, new OperatorKey(scope, +idGen.getNextNodeId(scope)), new ArrayListLogicalOperator()); +try { +mPlan.add(splitOp); + + +// Find all the successors and disconnect them. Keep our own copy +// of the list, as we're changing the graph by doing these calls +// and that will change the list of predecessors. +ListLogicalOperator succs = +new ArrayListLogicalOperator(mPlan.getSuccessors(nodes.get(0))); +int index = -1; +for (LogicalOperator succ : succs) { +mPlan.disconnect(nodes.get(0), succ); +LogicalPlan condPlan = new LogicalPlan(); +LOConst cnst = new LOConst(mPlan, new OperatorKey(scope, +idGen.getNextNodeId(scope)), new Boolean(true)); +cnst.setType(DataType.BOOLEAN); +condPlan.add(cnst); +LOSplitOutput splitOutput = new LOSplitOutput(mPlan, +new OperatorKey(scope, idGen.getNextNodeId(scope)), ++index, condPlan); +splitOp.addOutput(splitOutput); +mPlan.add(splitOutput); +mPlan.connect(splitOp, splitOutput); +mPlan.connect(splitOutput, succ); +// Patch up the contained plans of succ +fixUpContainedPlans(nodes.get(0), splitOutput, succ, null); +} +mPlan.connect(nodes.get(0), splitOp); +} catch (Exception e
svn commit: r689983 - /incubator/pig/branches/types/src/org/apache/pig/backend/executionengine/PigSlicer.java
Author: gates Date: Thu Aug 28 14:07:09 2008 New Revision: 689983 URL: http://svn.apache.org/viewvc?rev=689983view=rev Log: PIG-403 Fix wrong default in PigSlicer that caused all pig jobs to use a single map. Modified: incubator/pig/branches/types/src/org/apache/pig/backend/executionengine/PigSlicer.java Modified: incubator/pig/branches/types/src/org/apache/pig/backend/executionengine/PigSlicer.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/executionengine/PigSlicer.java?rev=689983r1=689982r2=689983view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/executionengine/PigSlicer.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/executionengine/PigSlicer.java Thu Aug 28 14:07:09 2008 @@ -108,5 +108,5 @@ private FuncSpec funcSpec; -private boolean splittable; +private boolean splittable = true; }
svn commit: r690039 - in /incubator/pig/branches/types: src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/ src/org/apache/pig/backend/hadoop/executionengine/physicalL
Author: gates Date: Thu Aug 28 15:59:06 2008 New Revision: 690039 URL: http://svn.apache.org/viewvc?rev=690039view=rev Log: PIG-359 (yet again). Address an issue Santhosh identified with the earlier patch where POUserFunc and POForEach were assuming that project star was a tuple, when in some cases in inner plans it can be a bag. Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java?rev=690039r1=690038r2=690039view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java Thu Aug 28 15:59:06 2008 @@ -153,7 +153,8 @@ if(temp.returnStatus!=POStatus.STATUS_OK) return temp; -if(op instanceof POProject){ +if(op instanceof POProject +op.getResultType() == DataType.TUPLE){ POProject projOp = (POProject)op; if(projOp.isStar()){ Tuple trslt = (Tuple) temp.result; Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java?rev=690039r1=690038r2=690039view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java Thu Aug 28 15:59:06 2008 @@ -337,7 +337,9 @@ ++i; PhysicalOperator leaf = (PhysicalOperator)p.getLeaves().get(0); planLeaves.add(leaf); -if(leaf instanceof POProject ((POProject)leaf).isStar()) +if(leaf instanceof POProject +leaf.getResultType() == DataType.TUPLE +((POProject)leaf).isStar()) isToBeFlattened.set(i, true); } } Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java?rev=690039r1=690038r2=690039view=diff == --- incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java (original) +++ incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java Thu Aug 28 15:59:06 2008 @@ -376,7 +376,7 @@ + C1 = filter A by $0 -1; + C2 = distinct C1; + C3 = distinct A; -+ generate (int)group, + Identity.class.getName() +(*), COUNT(C2), SUM(C2.$1), + TitleNGrams.class.getName() + (C3), MAX(C3.$1); ++ generate (int)group, + Identity.class.getName() +(*), COUNT(C2), SUM(C2.$1), + TitleNGrams.class.getName() + (C3), MAX(C3.$1), C2; + };; pigServer.registerQuery(query); @@ -389,7 +389,8 @@ assertEquals((Long)5L, (Long)t.get(2)); assertEquals(LOOP_COUNT*2.0, (Double)t.get(3), 0.01); assertEquals(8.0, (Double)t.get(5), 0.01); -assertEquals(6, t.size()); +assertEquals(5L, ((DataBag)t.get(6)).size()); +assertEquals(7, t.size()); ++numIdentity; } assertEquals(LOOP_COUNT, numIdentity);
svn commit: r689631 - in /incubator/pig/branches/types: src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/ src/org/apache/pig/backend/hadoop/executionengine/physicalL
Author: gates Date: Wed Aug 27 14:42:11 2008 New Revision: 689631 URL: http://svn.apache.org/viewvc?rev=689631view=rev Log: PIG-359. Shravan's patch to fix * semantics in generate and when passed as an argument to a UDF. Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java incubator/pig/branches/types/test/org/apache/pig/test/TestPOUserFunc.java incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/ComplexForeach.gld Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java?rev=689631r1=689630r2=689631view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java Wed Aug 27 14:42:11 2008 @@ -152,13 +152,19 @@ } if(temp.returnStatus!=POStatus.STATUS_OK) return temp; - + +if(op instanceof POProject){ +POProject projOp = (POProject)op; +if(projOp.isStar()){ +Tuple trslt = (Tuple) temp.result; +Tuple rslt = (Tuple) res.result; +for(int i=0;itrslt.size();i++) +rslt.append(trslt.get(i)); +continue; +} +} ((Tuple)res.result).append(temp.result); } -Tuple rslt = ((Tuple)res.result); -if(rslt.size()==1 rslt.get(0) instanceof Tuple){ -res.result = rslt.get(0); -} res.returnStatus = temp.returnStatus; return res; } Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java?rev=689631r1=689630r2=689631view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java Wed Aug 27 14:42:11 2008 @@ -17,6 +17,7 @@ import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result; +import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POProject; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan; import org.apache.pig.impl.plan.OperatorKey; @@ -331,8 +332,13 @@ private void getLeaves() { if (inputPlans != null) { +int i=-1; for(PhysicalPlan p : inputPlans) { -planLeaves.add((PhysicalOperator)p.getLeaves().get(0)); +++i; +PhysicalOperator leaf = (PhysicalOperator)p.getLeaves().get(0); +planLeaves.add(leaf); +if(leaf instanceof POProject ((POProject)leaf).isStar()) +isToBeFlattened.set(i, true); } } } Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestPOUserFunc.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestPOUserFunc.java?rev=689631r1=689630r2=689631view=diff == --- incubator/pig/branches/types/test/org/apache/pig/test/TestPOUserFunc.java (original) +++ incubator/pig/branches/types/test/org/apache/pig/test/TestPOUserFunc.java Wed Aug 27 14:42:11 2008 @@ -55,7 +55,13 @@ @Override public Integer exec(Tuple input) throws IOException { - return new Integer(input.size()); + try { +return new Integer(((Tuple)input.get(0
svn commit: r689673 - in /incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer: LOLimit.java PlanSetter.java optimizer/SchemaCalculator.java optimizer/SchemaRemover.java package.html pars
Author: gates Date: Wed Aug 27 17:09:04 2008 New Revision: 689673 URL: http://svn.apache.org/viewvc?rev=689673view=rev Log: PIG-399 Added LOLimit to visitors in the optimizer so that when the plan is rearranged limit is handled correctly. Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOLimit.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/PlanSetter.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/SchemaCalculator.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/SchemaRemover.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/package.html incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOLimit.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOLimit.java?rev=689673r1=689672r2=689673view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOLimit.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOLimit.java Wed Aug 27 17:09:04 2008 @@ -9,7 +9,6 @@ public class LOLimit extends LogicalOperator { private static final long serialVersionUID = 2L; -private LogicalOperator mInput; private long mLimit; /** * @@ -23,15 +22,13 @@ *the input that needs to limit */ -public LOLimit(LogicalPlan plan, OperatorKey k, -long limit, LogicalOperator input) { +public LOLimit(LogicalPlan plan, OperatorKey k, long limit) { super(plan, k); mLimit = limit; -mInput = input; } public LogicalOperator getInput() { -return mInput; +return mPlan.getPredecessors(this).get(0); } public long getLimit() { @@ -45,7 +42,7 @@ public Schema getSchema() throws FrontendException { if (!mIsSchemaComputed) { try { -mSchema = mInput.getSchema(); +mSchema = getInput().getSchema(); mIsSchemaComputed = true; } catch (FrontendException ioe) { mSchema = null; @@ -81,8 +78,9 @@ return DataType.BAG ; } +// Shouldn't this be clone? public LOLimit duplicate() { - return new LOLimit(mPlan, OperatorKey.genOpKey(mKey.scope), mLimit, mInput); + return new LOLimit(mPlan, OperatorKey.genOpKey(mKey.scope), mLimit); } } Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/PlanSetter.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/PlanSetter.java?rev=689673r1=689672r2=689673view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/PlanSetter.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/PlanSetter.java Wed Aug 27 17:09:04 2008 @@ -108,6 +108,10 @@ op.setPlan(mCurrentWalker.getPlan()); } +public void visit(LOLimit op) throws VisitorException { +op.setPlan(mCurrentWalker.getPlan()); +} + public void visit(LOLoad op) throws VisitorException { op.setPlan(mCurrentWalker.getPlan()); } Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/SchemaCalculator.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/SchemaCalculator.java?rev=689673r1=689672r2=689673view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/SchemaCalculator.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/SchemaCalculator.java Wed Aug 27 17:09:04 2008 @@ -94,6 +94,21 @@ /** * + * @param limit + *the logical limit operator that has to be visited + * @throws VisitorException + */ +protected void visit(LOLimit limit) throws VisitorException { +try { +limit.getSchema(); +super.visit(limit); +} catch (FrontendException fe) { +throw new VisitorException(fe); +} +} + +/** + * * @param filter *the logical filter operator that has to be visited * @throws VisitorException Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/SchemaRemover.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/optimizer/SchemaRemover.java?rev=689673r1=689672r2=689673view=diff
svn commit: r689289 - /incubator/pig/branches/types/src/org/apache/pig/PigServer.java
Author: gates Date: Tue Aug 26 17:00:29 2008 New Revision: 689289 URL: http://svn.apache.org/viewvc?rev=689289view=rev Log: PIG-379 Changed describe to not call the optimizer. Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/PigServer.java?rev=689289r1=689288r2=689289view=diff == --- incubator/pig/branches/types/src/org/apache/pig/PigServer.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/PigServer.java Tue Aug 26 17:00:29 2008 @@ -282,7 +282,7 @@ try { LogicalPlan lp = getPlanFromAlias(alias, describe); try { -lp = compileLp(lp, describe); +lp = compileLp(lp, describe, false); } catch (ExecException e) { throw new FrontendException(e.getMessage()); } @@ -529,6 +529,13 @@ private LogicalPlan compileLp( LogicalPlan lp, String operation) throws ExecException, FrontendException { +return compileLp(lp, operation, true); +} + +private LogicalPlan compileLp( +LogicalPlan lp, +String operation, +boolean optimize) throws ExecException, FrontendException { // Look up the logical plan in the aliases map. That plan will be // properly connected to all the others. @@ -585,8 +592,10 @@ } // optimize -LogicalOptimizer optimizer = new LogicalOptimizer(lp); -optimizer.optimize(); +if (optimize) { +LogicalOptimizer optimizer = new LogicalOptimizer(lp); +optimizer.optimize(); +} return lp; }
svn commit: r689292 - /incubator/pig/branches/types/src/org/apache/pig/PigServer.java
Author: gates Date: Tue Aug 26 17:10:08 2008 New Revision: 689292 URL: http://svn.apache.org/viewvc?rev=689292view=rev Log: PIG-372 Fix NPE in dump schema when there is no known schema. Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/PigServer.java?rev=689292r1=689291r2=689292view=diff == --- incubator/pig/branches/types/src/org/apache/pig/PigServer.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/PigServer.java Tue Aug 26 17:10:08 2008 @@ -287,7 +287,8 @@ throw new FrontendException(e.getMessage()); } Schema schema = lp.getLeaves().get(0).getSchema(); -System.out.println(schema.toString()); +if (schema != null) System.out.println(schema.toString()); +else System.out.println(Schema for + alias + unknown.); } catch (FrontendException fe) { throw WrappedIOException.wrap( Unable to describe schema for alias + alias, fe);
svn commit: r689293 - /incubator/pig/branches/types/src/org/apache/pig/PigServer.java
Author: gates Date: Tue Aug 26 17:15:45 2008 New Revision: 689293 URL: http://svn.apache.org/viewvc?rev=689293view=rev Log: PIG-383 Added relation name to describe output. Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/PigServer.java?rev=689293r1=689292r2=689293view=diff == --- incubator/pig/branches/types/src/org/apache/pig/PigServer.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/PigServer.java Tue Aug 26 17:15:45 2008 @@ -287,7 +287,7 @@ throw new FrontendException(e.getMessage()); } Schema schema = lp.getLeaves().get(0).getSchema(); -if (schema != null) System.out.println(schema.toString()); +if (schema != null) System.out.println(alias + : + schema.toString()); else System.out.println(Schema for + alias + unknown.); } catch (FrontendException fe) { throw WrappedIOException.wrap(
svn commit: r684940 - /incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java
Author: gates Date: Mon Aug 11 14:25:37 2008 New Revision: 684940 URL: http://svn.apache.org/viewvc?rev=684940view=rev Log: PIG-337 Daniel's fix for limit k where k total lines returning too few results. Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java?rev=684940r1=684939r2=684940view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java Mon Aug 11 14:25:37 2008 @@ -663,8 +663,8 @@ POProject prj1 = new POProject(new OperatorKey(scope,nig.getNextNodeId(scope))); prj1.setResultType(DataType.TUPLE); prj1.setStar(false); -prj1.setColumn(0); -prj1.setOverloaded(false); +prj1.setColumn(1); +prj1.setOverloaded(true); ep1.add(prj1); eps1.add(ep1); flat1.add(false);
svn commit: r685032 - in /incubator/pig/branches/types: src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/ test/org/apache/pig/test/ test/org/apache/pig/test/data/GoldenFiles/
Author: gates Date: Mon Aug 11 21:35:02 2008 New Revision: 685032 URL: http://svn.apache.org/viewvc?rev=685032view=rev Log: PIG-343 Shravan's patch to fix split. Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/IsNull1.gld incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/Split1.gld Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java?rev=685032r1=685031r2=685032view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java Mon Aug 11 21:35:02 2008 @@ -17,6 +17,7 @@ */ package org.apache.pig.backend.hadoop.executionengine.physicalLayer; +import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -28,6 +29,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.pig.ComparisonFunc; import org.apache.pig.EvalFunc; +import org.apache.pig.FuncSpec; import org.apache.pig.LoadFunc; import org.apache.pig.data.DataType; import org.apache.pig.impl.PigContext; @@ -38,6 +40,9 @@ import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.ExpressionOperator; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.BinaryExpressionOperator; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.UnaryExpressionOperator; +import org.apache.pig.builtin.BinStorage; +import org.apache.pig.impl.io.FileLocalizer; +import org.apache.pig.impl.io.FileSpec; import org.apache.pig.impl.logicalLayer.*; import org.apache.pig.impl.plan.DependencyOrderWalker; import org.apache.pig.impl.plan.DependencyOrderWalkerWOSeenChk; @@ -759,6 +764,14 @@ String scope = split.getOperatorKey().scope; PhysicalOperator physOp = new POSplit(new OperatorKey(scope, nodeGen .getNextNodeId(scope)), split.getRequestedParallelism()); +FileSpec splStrFile; +try { +splStrFile = new FileSpec(FileLocalizer.getTemporaryPath(null, pc).toString(),new FuncSpec(BinStorage.class.getName())); +} catch (IOException e1) { +log.error(Unable to obtain a temporary path because + e1.getMessage()); +throw new VisitorException(e1); +} +((POSplit)physOp).setSplitStore(splStrFile); LogToPhyMap.put(split, physOp); currentPlan.add(physOp); Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java?rev=685032r1=685031r2=685032view=diff == --- incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java (original) +++ incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java Mon Aug 11 21:35:02 2008 @@ -68,10 +68,16 @@ Random r = new Random(); PigContext pc = new PigContext(ExecType.LOCAL, new Properties()); - + private boolean generate = false; + +@Override +protected void setUp() throws Exception { +pc.connect(); +} + private void writeData(File input, int noTuples, int arityOfTuples, char separator) throws IOException { FileOutputStream dat = new FileOutputStream(input); Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/IsNull1.gld URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/IsNull1.gld?rev=685032r1=685031r2=685032view=diff == --- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/IsNull1.gld (original) +++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/IsNull1.gld Mon Aug 11 21:35:02 2008 @@ -1,20 +1,20 @@ Filter[tuple] - Test-Plan-Builder-214 | | -| Not[boolean] - Test-Plan-Builder-217 +| POIsNull[tuple] - Test-Plan-Builder-216 | | -| |---POIsNull[tuple] - Test-Plan-Builder-216 -| | -| |---Project[bytearray][0] - Test-Plan-Builder-215 +| |---Project[bytearray][0] - Test-Plan
svn commit: r680890 - /incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
Author: gates Date: Tue Jul 29 17:43:52 2008 New Revision: 680890 URL: http://svn.apache.org/viewvc?rev=680890view=rev Log: PIG-285 This patch makes it so the the script doesn't fail with error messages about not being able to instantiate the user provided comparator. Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java?rev=680890r1=680889r2=680890view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java Tue Jul 29 17:43:52 2008 @@ -282,7 +282,7 @@ jobConf.setPartitionerClass(SortPartitioner.class); if(mro.UDFs.size()==1){ String compFuncSpec = mro.UDFs.get(0); -Class comparator = Class.forName(compFuncSpec); +Class comparator = PigContext.resolveClassName(compFuncSpec); if(ComparisonFunc.class.isAssignableFrom(comparator)) jobConf.setOutputKeyComparatorClass(comparator); }
svn commit: r677947 - in /incubator/pig/branches/types: src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt test/org/apache/pig/test/TestBuiltin.java
Author: gates Date: Fri Jul 18 08:54:17 2008 New Revision: 677947 URL: http://svn.apache.org/viewvc?rev=677947view=rev Log: PIG-308 Fixed join rewrites in the parser, with Santhosh's help. Also uncommented two tests in TestBuiltin that should have been uncommented a while ago. Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt?rev=677947r1=677946r2=677947view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt Fri Jul 18 08:54:17 2008 @@ -227,12 +227,16 @@ */ - LogicalPlan foreachPlan = new LogicalPlan(); + + //Construct the cogroup operator and add it to the logical plan + LogicalOperator cogroup = parseCogroup(gis, lp); + lp.add(cogroup); + log.debug(Added operator + cogroup.getClass().getName() + to the logical plan); + for (int i = 0; i n; i++) { LogicalPlan projectPlan = new LogicalPlan(); - LogicalOperator projectInput = gis.get(i).op; + LogicalOperator projectInput = cogroup; ExpressionOperator column = new LOProject(projectPlan, new OperatorKey(scope, getNextId()), projectInput, i+1); - ((LOProject)column).setStar(true); flattenList.add(true); flattenedColumns.add(column); (gis.get(i)).isInner = true; @@ -246,26 +250,11 @@ } - //Construct the cogroup operator and add it to the logical plan - - LogicalOperator cogroup = parseCogroup(gis, lp); - lp.add(cogroup); - log.debug(Added operator + cogroup.getClass().getName() + to the logical plan); - - - //Construct the generate operator from the list of projection plans - //Add the generate operator to the foreach logical plan -/* - LogicalOperator generate = new LOGenerate(lp, new OperatorKey(scope, getNextId()), generatePlans, flattenList); - foreachPlan.add(generate); - log.debug(Added operator + generate.getClass().getName() + to the logical plan + lp); -*/ /* * Construct the foreach operator from the foreach logical plan * Add the foreach operator to the top level logical plan */ - LogicalOperator foreach = new LOForEach(lp, new OperatorKey(scope, getNextId()), generatePlans, flattenList); lp.add(foreach); log.debug(Added operator + foreach.getClass().getName() + to the logical plan); Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java?rev=677947r1=677946r2=677947view=diff == --- incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java (original) +++ incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java Fri Jul 18 08:54:17 2008 @@ -873,8 +873,6 @@ assertTrue(f1.equals(f2)); } -// TODO FIX -/* @Test public void testShellFuncSingle() throws Throwable { //ShellBagEvalFunc func = new ShellBagEvalFunc(tr o 0); @@ -935,7 +933,6 @@ assertFalse(iter.hasNext()); tempFile.delete(); } -*/ private static String getInputType(String typeFor) { return allowedInput.get(typeFor);
svn commit: r678038 - in /incubator/pig/branches/types: src/org/apache/pig/ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/logicalLayer/parser/ src/org/apache/pig/tools/pigscript/parser
Author: gates Date: Fri Jul 18 14:07:09 2008 New Revision: 678038 URL: http://svn.apache.org/viewvc?rev=678038view=rev Log: PIG-323 Checking in Santhosh's patch to back out define changes. Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt incubator/pig/branches/types/src/org/apache/pig/tools/pigscript/parser/PigScriptParser.jj incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java incubator/pig/branches/types/test/org/apache/pig/test/TestGrunt.java incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java incubator/pig/branches/types/test/org/apache/pig/test/TestPigScriptParser.java incubator/pig/branches/types/test/org/apache/pig/test/utils/LogicalPlanTester.java Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/PigServer.java?rev=678038r1=678037r2=678038view=diff == --- incubator/pig/branches/types/src/org/apache/pig/PigServer.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/PigServer.java Fri Jul 18 14:07:09 2008 @@ -91,7 +91,6 @@ MapLogicalOperator, LogicalPlan aliases = new HashMapLogicalOperator, LogicalPlan(); MapOperatorKey, LogicalOperator opTable = new HashMapOperatorKey, LogicalOperator(); MapString, LogicalOperator aliasOp = new HashMapString, LogicalOperator(); -MapString, ExpressionOperator defineAliases = new HashMapString, ExpressionOperator(); PigContext pigContext; private String scope = constructScope(); @@ -239,7 +238,7 @@ LogicalPlan lp = null; try { lp = (new LogicalPlanBuilder(pigContext).parse(scope, query, -aliases, opTable, aliasOp, defineAliases)); +aliases, opTable, aliasOp)); } catch (ParseException e) { throw (IOException) new IOException(e.getMessage()).initCause(e); } Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java?rev=678038r1=678037r2=678038view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java Fri Jul 18 14:07:09 2008 @@ -43,12 +43,11 @@ String query, MapLogicalOperator, LogicalPlan aliases, MapOperatorKey, LogicalOperator opTable, - MapString, LogicalOperator aliasOp, - MapString, ExpressionOperator defineAliases) + MapString, LogicalOperator aliasOp) throws IOException, ParseException { ByteArrayInputStream in = new ByteArrayInputStream(query.getBytes()); //QueryParser parser = new QueryParser(in, pigContext, scope, aliases, opTable); -QueryParser parser = new QueryParser(in, pigContext, scope, aliases, opTable, aliasOp, defineAliases); +QueryParser parser = new QueryParser(in, pigContext, scope, aliases, opTable, aliasOp); return parser.Parse(); } Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt?rev=678038r1=678037r2=678038view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt Fri Jul 18 14:07:09 2008 @@ -63,7 +63,6 @@ private NodeIdGenerator nodeIdGen; //a map of alias to logical operator for a quick lookup private MapString, LogicalOperator mapAliasOp; - private MapString, ExpressionOperator defineAliases; private static Log log = LogFactory.getLog(QueryParser.class); private long getNextId() { @@ -75,8 +74,7 @@ String scope, MapLogicalOperator, LogicalPlan aliases
svn commit: r677638 - in /incubator/pig/branches/types: src/org/apache/pig/builtin/Utf8StorageConverter.java src/org/apache/pig/impl/mapReduceLayer/PigMapReduce.java test/org/apache/pig/test/TestConve
Author: gates Date: Thu Jul 17 10:05:06 2008 New Revision: 677638 URL: http://svn.apache.org/viewvc?rev=677638view=rev Log: PIG-312 When casting to an int or long, if the cast fails, try to cast to a double and then to int or long. Modified: incubator/pig/branches/types/src/org/apache/pig/builtin/Utf8StorageConverter.java incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/PigMapReduce.java incubator/pig/branches/types/test/org/apache/pig/test/TestConversions.java Modified: incubator/pig/branches/types/src/org/apache/pig/builtin/Utf8StorageConverter.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/builtin/Utf8StorageConverter.java?rev=677638r1=677637r2=677638view=diff == --- incubator/pig/branches/types/src/org/apache/pig/builtin/Utf8StorageConverter.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/builtin/Utf8StorageConverter.java Thu Jul 17 10:05:06 2008 @@ -50,6 +50,9 @@ protected BagFactory mBagFactory = BagFactory.getInstance(); protected TupleFactory mTupleFactory = TupleFactory.getInstance(); protected final Log mLog = LogFactory.getLog(getClass()); + +private Integer mMaxInt = new Integer(Integer.MAX_VALUE); +private Long mMaxLong = new Long(Long.MAX_VALUE); public Utf8StorageConverter() { } @@ -86,24 +89,54 @@ } public Integer bytesToInteger(byte[] b) throws IOException { +String s = new String(b); try { -return Integer.valueOf(new String(b)); +return Integer.valueOf(s); } catch (NumberFormatException nfe) { -mLog.warn(Unable to interpret value + b + in field being + -converted to int, caught NumberFormatException + -nfe.getMessage() + field discarded); -return null; +// It's possible that this field can be interpreted as a double. +// Unfortunately Java doesn't handle this in Integer.valueOf. So +// we need to try to convert it to a double and if that works then +// go to an int. +try { +Double d = Double.valueOf(s); +// Need to check for an overflow error +if (d.doubleValue() mMaxInt.doubleValue() + 1.0) { +mLog.warn(Value + d + too large for integer); +return null; +} +return new Integer(d.intValue()); +} catch (NumberFormatException nfe2) { +mLog.warn(Unable to interpret value + b + in field being + +converted to int, caught NumberFormatException + +nfe.getMessage() + field discarded); +return null; +} } } public Long bytesToLong(byte[] b) throws IOException { +String s = new String(b); try { -return Long.valueOf(new String(b)); +return Long.valueOf(s); } catch (NumberFormatException nfe) { -mLog.warn(Unable to interpret value + b + in field being + -converted to long, caught NumberFormatException + -nfe.getMessage() + field discarded); -return null; +// It's possible that this field can be interpreted as a double. +// Unfortunately Java doesn't handle this in Long.valueOf. So +// we need to try to convert it to a double and if that works then +// go to an long. +try { +Double d = Double.valueOf(s); +// Need to check for an overflow error +if (d.doubleValue() mMaxLong.doubleValue() + 1.0) { +mLog.warn(Value + d + too large for integer); +return null; +} +return new Long(d.longValue()); +} catch (NumberFormatException nfe2) { +mLog.warn(Unable to interpret value + b + in field being + +converted to long, caught NumberFormatException + +nfe.getMessage() + field discarded); +return null; +} } } Modified: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/PigMapReduce.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/PigMapReduce.java?rev=677638r1=677637r2=677638view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/PigMapReduce.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/PigMapReduce.java Thu Jul 17 10:05:06 2008 @@ -46,7 +46,7 @@ import org.apache.pig.impl.util.ObjectSerializer
svn commit: r675663 - in /incubator/pig/branches/types: src/org/apache/pig/impl/mapReduceLayer/ src/org/apache/pig/impl/physicalLayer/expressionOperators/ test/org/apache/pig/test/data/GoldenFiles/ te
Author: gates Date: Thu Jul 10 11:09:36 2008 New Revision: 675663 URL: http://svn.apache.org/viewvc?rev=675663view=rev Log: PIG-293 Previous patches solved infinite loop. This patch makes order by in map reduce mode actually work. Modified: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POProject.java incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC15.gld incubator/pig/branches/types/test/org/apache/pig/test/utils/GenPhyOp.java Modified: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java?rev=675663r1=675662r2=675663view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java Thu Jul 10 11:09:36 2008 @@ -180,6 +180,7 @@ public MROperPlan compile() throws IOException, PlanException, VisitorException { ListPhysicalOperator leaves = plan.getLeaves(); POStore store = (POStore)leaves.get(0); +System.out.println(store file is + store.getSFile()); FileLocalizer.registerDeleteOnFail(store.getSFile().getFileName(), pigContext); compile(store); @@ -729,20 +730,27 @@ } } -private int[] getSortCols(POSort sort){ +private int[] getSortCols(POSort sort) throws PlanException { ListPhysicalPlan plans = sort.getSortPlans(); if(plans!=null){ int[] ret = new int[plans.size()]; int i=-1; for (PhysicalPlan plan : plans) { +if (((POProject)plan.getLeaves().get(0)).isStar()) return null; ret[++i] = ((POProject)plan.getLeaves().get(0)).getColumn(); } return ret; } -return null; +log.error(No expression plan found in POSort); +throw new PlanException(No Expression Plan found in POSort); } -public MapReduceOper getSortJob(MapReduceOper quantJob, FileSpec lFile, FileSpec quantFile, int rp, int[] fields) throws PlanException{ +public MapReduceOper getSortJob( +MapReduceOper quantJob, +FileSpec lFile, +FileSpec quantFile, +int rp, +int[] fields) throws PlanException{ MapReduceOper mro = startNew(lFile, quantJob); mro.setQuantFile(quantFile.getFileName()); mro.setGlobalSort(true); @@ -750,23 +758,31 @@ ListPhysicalPlan eps1 = new ArrayListPhysicalPlan(); -if(fields==null) { -log.error(No Expression Plan found in POSort); -throw new PlanException(No Expression Plan found in POSort); -} -for (int i : fields) { +if (fields == null) { +// This is project * PhysicalPlan ep = new PhysicalPlan(); POProject prj = new POProject(new OperatorKey(scope,nig.getNextNodeId(scope))); -prj.setColumn(i); +prj.setStar(true); prj.setOverloaded(false); -prj.setResultType(DataType.BYTEARRAY); +prj.setResultType(DataType.TUPLE); ep.add(prj); eps1.add(ep); +} else { +for (int i : fields) { +PhysicalPlan ep = new PhysicalPlan(); +POProject prj = new POProject(new OperatorKey(scope, +nig.getNextNodeId(scope))); +prj.setColumn(i); +prj.setOverloaded(false); +prj.setResultType(DataType.BYTEARRAY); +ep.add(prj); +eps1.add(ep); +} } POLocalRearrange lr = new POLocalRearrange(new OperatorKey(scope,nig.getNextNodeId(scope))); lr.setIndex(0); -lr.setKeyType((fields.length1) ? DataType.TUPLE : DataType.BYTEARRAY); +lr.setKeyType((fields == null || fields.length1) ? DataType.TUPLE : DataType.BYTEARRAY); lr.setPlans(eps1); lr.setResultType(DataType.TUPLE); mro.mapPlan.addAsLeaf(lr); @@ -774,7 +790,7 @@ mro.setMapDone(true); POPackage pkg = new POPackage(new OperatorKey(scope,nig.getNextNodeId(scope))); -pkg.setKeyType((fields.length1) ? DataType.TUPLE : DataType.BYTEARRAY); +pkg.setKeyType((fields == null || fields.length1) ? DataType.TUPLE : DataType.BYTEARRAY); pkg.setNumInps(1); boolean[] inner = {false}; pkg.setInner(inner); @@ -810,19 +826,27 @@ ListPhysicalPlan eps1 = new ArrayListPhysicalPlan(); ListBoolean flat1 = new ArrayListBoolean
svn commit: r675273 - in /incubator/pig/branches/types: src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/mapReduceLayer/ src/org/apache/pig/impl/mapReduceLayer/plans/ src/org/apache/pig/i
Author: gates Date: Wed Jul 9 10:58:42 2008 New Revision: 675273 URL: http://svn.apache.org/viewvc?rev=675273view=rev Log: PIG-285 Shravan's sortfin patch. This addresses some, though not all, order by issues. Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/JobControlCompiler.java incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/plans/UDFFinder.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POUserComparisonFunc.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POUserFunc.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/PhyPlanVisitor.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/relationalOperators/POSort.java incubator/pig/branches/types/test/org/apache/pig/test/TestMRCompiler.java incubator/pig/branches/types/test/org/apache/pig/test/TestPOSort.java incubator/pig/branches/types/test/org/apache/pig/test/TestPOUserFunc.java incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/IsNull1.gld incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/IsNull2.gld incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC15.gld incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC16.gld incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/Sort.gld incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/Split1.gld incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/Split2.gld Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java?rev=675273r1=675272r2=675273view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java Wed Jul 9 10:58:42 2008 @@ -687,7 +687,7 @@ .getNextNodeId(scope)), s.getRequestedParallelism(), null, sortPlans, s.getAscendingCols(), null); } else { -POUserFunc comparator = new POUserComparisonFunc(new OperatorKey( +POUserComparisonFunc comparator = new POUserComparisonFunc(new OperatorKey( scope, nodeGen.getNextNodeId(scope)), s .getRequestedParallelism(), null, s.getUserFunc()); sort = new POSort(new OperatorKey(scope, nodeGen Modified: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/JobControlCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/JobControlCompiler.java?rev=675273r1=675272r2=675273view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/JobControlCompiler.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/JobControlCompiler.java Wed Jul 9 10:58:42 2008 @@ -32,6 +32,7 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.jobcontrol.Job; import org.apache.hadoop.mapred.jobcontrol.JobControl; +import org.apache.pig.ComparisonFunc; import org.apache.pig.data.DataType; import org.apache.pig.data.IndexedTuple; import org.apache.pig.data.TupleFactory; @@ -265,6 +266,12 @@ if(mro.isGlobalSort()){ jobConf.set(pig.quantilesFile, mro.getQuantFile()); jobConf.setPartitionerClass(SortPartitioner.class); +if(mro.UDFs.size()==1){ +String compFuncSpec = mro.UDFs.get(0); +Class comparator = Class.forName(compFuncSpec); +if(ComparisonFunc.class.isAssignableFrom(comparator)) +jobConf.setOutputKeyComparatorClass(comparator); +} } return jobConf; Modified: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java?rev=675273r1=675272r2=675273view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java (original) +++ incubator/pig/branches/types/src/org/apache/pig
svn commit: r674908 - /incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java
Author: gates Date: Tue Jul 8 11:26:18 2008 New Revision: 674908 URL: http://svn.apache.org/viewvc?rev=674908view=rev Log: PIG-287 Modified test to confirm bug is fixed. Thanks Santhosh. Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java?rev=674908r1=674907r2=674908view=diff == --- incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java (original) +++ incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java Tue Jul 8 11:26:18 2008 @@ -358,7 +358,7 @@ + C1 = filter A by $0 -1; + C2 = distinct C1; + C3 = distinct A; -+ generate group, + Identity.class.getName() +(*), COUNT(C2), SUM(C2.$1), + TitleNGrams.class.getName() + (C3), MAX(C3.$1); ++ generate (int)group, + Identity.class.getName() +(*), COUNT(C2), SUM(C2.$1), + TitleNGrams.class.getName() + (C3), MAX(C3.$1); + };; pig.registerQuery(query); @@ -367,6 +367,7 @@ int numIdentity = 0; while(iter.hasNext()){ Tuple t = iter.next(); +assertEquals((Integer)numIdentity, (Integer)t.get(0)); assertEquals((Long)5L, (Long)t.get(3)); assertEquals(LOOP_COUNT*2.0, (Double)t.get(4), 0.01); assertEquals(8.0, (Double)t.get(6), 0.01); @@ -376,5 +377,4 @@ assertEquals(LOOP_COUNT, numIdentity); } - }
svn commit: r673198 - in /incubator/pig/branches/types: src/org/apache/pig/ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/logicalLayer/parser/ src/org/apache/pig/impl/logicalLayer/vali
Author: gates Date: Tue Jul 1 13:33:41 2008 New Revision: 673198 URL: http://svn.apache.org/viewvc?rev=673198view=rev Log: Fixed bugs in equal and notequal physical operators that were returning wrong results for numeric types. Added PlanSetter visitor that walks logical plan and sets all the mPlan values to the final plan the operator is in. Added: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/PlanSetter.java Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOCogroup.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOCross.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LODistinct.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOFilter.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOProject.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSort.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOUnion.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalOperator.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/EqualToExpr.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/NotEqualToExpr.java incubator/pig/branches/types/src/org/apache/pig/impl/plan/OperatorPlan.java incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpNumeric.java incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidator.java incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidatorNoSchema.java incubator/pig/branches/types/test/org/apache/pig/test/utils/dotGraph/LogicalPlanLoader.java Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/PigServer.java?rev=673198r1=673197r2=673198view=diff == --- incubator/pig/branches/types/src/org/apache/pig/PigServer.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/PigServer.java Tue Jul 1 13:33:41 2008 @@ -50,6 +50,7 @@ import org.apache.pig.impl.logicalLayer.LogicalPlan; import org.apache.pig.impl.logicalLayer.LogicalPlanBuilder; import org.apache.pig.impl.logicalLayer.LOPrinter; +import org.apache.pig.impl.logicalLayer.PlanSetter; import org.apache.pig.impl.logicalLayer.optimizer.LogicalOptimizer; import org.apache.pig.impl.logicalLayer.parser.ParseException; import org.apache.pig.impl.logicalLayer.parser.QueryParser; @@ -481,6 +482,10 @@ throw new FrontendException(Cannot operate on null logical plan); } +// Set the logical plan values correctly in all the operators +PlanSetter ps = new PlanSetter(lp); +ps.visit(); + // run through validator CompilationMessageCollector collector = new CompilationMessageCollector() ; FrontendException caught = null; Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOCogroup.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOCogroup.java?rev=673198r1=673197r2=673198view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOCogroup.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOCogroup.java Tue Jul 1 13:33:41 2008 @@ -43,7 +43,6 @@ * operator. Each generate operator in turn contains a list of expressions * for the columns that are projected */ -private ArrayListLogicalOperator mInputs; private boolean[] mIsInner; private static Log log = LogFactory.getLog(LOCogroup.class); private MultiMapLogicalOperator, LogicalPlan mGroupByPlans; @@ -54,22 +53,23 @@ *LogicalPlan this operator is a part of. * @param k *OperatorKey for this operator - * @param inputs - *List of input operators * @param groupByPlans *the group by columns + * @param isInner + *indicates whether the cogroup is inner for each relation */ -public LOCogroup(LogicalPlan plan, OperatorKey k, -ArrayListLogicalOperator inputs, -MultiMapLogicalOperator, LogicalPlan groupByPlans, boolean[] isInner) { +public LOCogroup( +LogicalPlan plan, +OperatorKey k, +MultiMapLogicalOperator, LogicalPlan
svn commit: r672801 [2/2] - in /incubator/pig/branches/types: src/org/apache/pig/ src/org/apache/pig/backend/hadoop/executionengine/ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/logic
Modified: incubator/pig/branches/types/test/org/apache/pig/test/utils/LogicalPlanTester.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/utils/LogicalPlanTester.java?rev=672801r1=672800r2=672801view=diff == --- incubator/pig/branches/types/test/org/apache/pig/test/utils/LogicalPlanTester.java (original) +++ incubator/pig/branches/types/test/org/apache/pig/test/utils/LogicalPlanTester.java Mon Jun 30 08:56:41 2008 @@ -19,12 +19,14 @@ package org.apache.pig.test.utils; import org.apache.pig.impl.logicalLayer.*; +import org.apache.pig.impl.logicalLayer.optimizer.LogicalOptimizer; import org.apache.pig.impl.logicalLayer.validators.TypeCheckingValidator; import org.apache.pig.impl.PigContext; import org.apache.pig.impl.plan.PlanValidationException; import org.apache.pig.impl.plan.CompilationMessageCollector; import org.apache.pig.impl.plan.OperatorKey; import org.apache.pig.impl.plan.NodeIdGenerator; +import org.apache.pig.impl.plan.optimizer.OptimizerException; import org.apache.pig.ExecType; import static org.apache.pig.test.utils.TypeCheckingTestUtil.* ; import org.apache.pig.test.utils.dotGraph.LogicalPlanLoader; @@ -87,7 +89,14 @@ TypeCheckingValidator typeValidator = new TypeCheckingValidator() ; typeValidator.validate(plan, collector) ; printMessageCollector(collector) ; -System.out.println(Actual plan:) ; +System.out.println(Actual plan after type check:) ; +printTypeGraph(plan) ; +} + +public void optimizePlan(LogicalPlan plan) throws OptimizerException { +LogicalOptimizer optimizer = new LogicalOptimizer(plan); +optimizer.optimize(); +System.out.println(Actual plan after after optimization:) ; printTypeGraph(plan) ; } @@ -98,11 +107,30 @@ * @param file * @throws PlanValidationException */ -public void typeCheckAgainstDotFile(LogicalPlan plan, String file) -throws PlanValidationException { +public void typeCheckAgainstDotFile( +LogicalPlan plan, +String file) throws PlanValidationException, OptimizerException { +typeCheckAgainstDotFile(plan, file, false); +} + +/*** + * Run type checking and compare the result with plan structure + * stored in Dot file + * @param plan + * @param file + * @param optimize if true, the plan will be run through the optimizer + * @throws PlanValidationException + */ +public void typeCheckAgainstDotFile( +LogicalPlan plan, +String file, +boolean optimize) throws PlanValidationException, + OptimizerException { // validate the given plan typeCheckPlan(plan); +if (optimize) optimizePlan(plan); + // load the expected plan from file LogicalPlanLoader planLoader = new LogicalPlanLoader() ; LogicalPlan expectedPlan = planLoader.loadFromFile(file, LogicalPlan.class) ; @@ -120,8 +148,15 @@ System.out.println(Checking DONE!) ; } -public void typeCheckUsingDotFile(String file) -throws PlanValidationException { +public void typeCheckUsingDotFile( +String file) throws PlanValidationException, OptimizerException { +typeCheckUsingDotFile(file, false); +} + +public void typeCheckUsingDotFile( +String file, +boolean optimize) throws PlanValidationException, + OptimizerException { DotGraphReader reader = new DotGraphReader() ; DotGraph graph = reader.loadFromFile(file) ; if (!graph.attributes.containsKey(pigScript)) { @@ -138,7 +173,7 @@ plan = buildPlan(query + ;) ; } } -typeCheckAgainstDotFile(plan, file) ; +typeCheckAgainstDotFile(plan, file, optimize) ; }
svn commit: r659695 - in /incubator/pig/branches/types: src/org/apache/pig/FilterFunc.java src/org/apache/pig/builtin/IsEmpty.java src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt test/org/
Author: gates Date: Fri May 23 15:31:28 2008 New Revision: 659695 URL: http://svn.apache.org/viewvc?rev=659695view=rev Log: PIG-159 Santhosh's fix to bug that prevented instantiation of UDFs. Modified: incubator/pig/branches/types/src/org/apache/pig/FilterFunc.java incubator/pig/branches/types/src/org/apache/pig/builtin/IsEmpty.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java Modified: incubator/pig/branches/types/src/org/apache/pig/FilterFunc.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/FilterFunc.java?rev=659695r1=659694r2=659695view=diff == --- incubator/pig/branches/types/src/org/apache/pig/FilterFunc.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/FilterFunc.java Fri May 23 15:31:28 2008 @@ -22,19 +22,7 @@ import org.apache.pig.data.Tuple; -public abstract class FilterFunc { - -/** - * This callback method must be implemented by all subclasses. This - * is the method that will be invoked on every Tuple of a given dataset. - * Since the dataset may be divided up in a variety of ways the programmer - * should not make assumptions about state that is maintained between - * invocations of this method. - * - * @param input the Tuple to be processed. - * @throws IOException - */ -abstract public boolean exec(Tuple input) throws IOException; +public abstract class FilterFunc extends EvalFuncBoolean { /** * Placeholder for cleanup to be performed at the end. User defined functions can override. Modified: incubator/pig/branches/types/src/org/apache/pig/builtin/IsEmpty.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/builtin/IsEmpty.java?rev=659695r1=659694r2=659695view=diff == --- incubator/pig/branches/types/src/org/apache/pig/builtin/IsEmpty.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/builtin/IsEmpty.java Fri May 23 15:31:28 2008 @@ -30,7 +30,7 @@ public class IsEmpty extends FilterFunc { @Override -public boolean exec(Tuple input) throws IOException { +public Boolean exec(Tuple input) throws IOException { try { Object values = input.get(0); if (values instanceof DataBag) Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt?rev=659695r1=659694r2=659695view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt Fri May 23 15:31:28 2008 @@ -48,6 +48,7 @@ import org.apache.pig.data.Tuple; import org.apache.pig.data.BagFactory; import org.apache.pig.data.DataBag; +import org.apache.pig.EvalFunc; public class QueryParser { @@ -2048,13 +2049,11 @@ if(null == userFunc) { //TODO //Commented out the code for instaniateFunc as it's failing - /* try{ - LOUserFunc ef = (LOUserFunc) pigContext.instantiateFuncFromAlias(funcName); + EvalFunc ef = (EvalFunc) pigContext.instantiateFuncFromAlias(funcName); }catch (Exception e){ throw new ParseException(e.getMessage()); } - */ } log.trace(Exiting EvalFunction); Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java?rev=659695r1=659694r2=659695view=diff == --- incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java (original) +++ incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java Fri May 23 15:31:28 2008 @@ -35,7 +35,7 @@ import org.apache.pig.LoadFunc; //TODO //Not able to include PigServer.java -//import org.apache.pig.PigServer; +import org.apache.pig.PigServer; import org.apache.pig.builtin.PigStorage; import org.apache.pig.data.DataBag; import org.apache.pig.data.Tuple; @@ -72,7 +72,7 @@ buildPlan(query); } -/* TODO FIX +// TODO FIX Query3 and Query4 @Test public void testQuery3
svn commit: r659721 - in /incubator/pig/branches/types: build.xml src/org/apache/pig/PigServer.java
Author: gates Date: Fri May 23 16:58:27 2008 New Revision: 659721 URL: http://svn.apache.org/viewvc?rev=659721view=rev Log: Fixes in PigServer to deal with change of alias from map string-logicalplan to logicaloperator-logicalplan. Added TestMapReduce to build.xml to begin end to end testing. It currently fails, but I'm leaving it in so we can test with it. Modified: incubator/pig/branches/types/build.xml incubator/pig/branches/types/src/org/apache/pig/PigServer.java Modified: incubator/pig/branches/types/build.xml URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/build.xml?rev=659721r1=659720r2=659721view=diff == --- incubator/pig/branches/types/build.xml (original) +++ incubator/pig/branches/types/build.xml Fri May 23 16:58:27 2008 @@ -222,47 +222,6 @@ batchtest fork=yes todir=${test.log.dir} unless=testcase fileset dir=test -!-- -include name=**/TestBuiltin.java / -include name=**/TestOperatorPlan.java / - include name=**/TestPhyOp.java / - include name=**/TestConstExpr.java / - include name=**/TestProject.java / - include name=**/TestFilter.java / - include name=**/TestAdd.java / - include name=**/TestSubtract.java / - include name=**/TestMultiply.java / - include name=**/TestDivide.java / - include name=**/TestMod.java / - include name=**/TestGreaterThan.java / - include name=**/TestGTOrEqual.java / - include name=**/TestLessThan.java / - include name=**/TestLTOrEqual.java / - include name=**/TestEqualTo.java / - include name=**/TestNotEqualTo.java / - include name=**/TestPOGenerate.java / - include name=**/TestPOSort.java / - include name=**/TestPOUserFunc.java / - include name=**/TestPODistinct.java / - include name=**/TestLoad.java / - include name=**/TestStore.java / - include name=**/TestPackage.java / - include name=**/TestLocalRearrange.java / - include name=**/TestForEach.java / - include name=**/TestUnion.java / - include name=**/TestMRCompiler.java / - include name=**/TestJobSubmission.java / - include name=**/TestInputOutputFileValidator.java / - include name=**/TestTypeCheckingValidator.java / - include name=**/TestSchema.java / - include name=**/TestLogicalPlanBuilder.java / -include name=**/TestLocalJobSubmission.java / - include name=**/TestPOMapLookUp.java / - include name=**/TestPOBinCond.java / - include name=**/TestPONegative.java / - include name=**/TestGrunt.java / - include name=**/TestPOCast.java / --- include name=**/*Test*.java / !-- Excluced because they are end-to-end, don't work yet. -- exclude name=**/TestAlgebraicEval.java / @@ -272,7 +231,7 @@ exclude name=**/TestFilterOpNumeric.java / exclude name=**/TestFilterOpString.java / exclude name=**/TestInfixArithmetic.java / -exclude name=**/TestMapReduce.java / +!-- exclude name=**/TestMapReduce.java / -- exclude name=**/TestPigFile.java / exclude name=**/TestPigSplit.java / exclude name=**/TestStoreOld.java / Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/PigServer.java?rev=659721r1=659720r2=659721view=diff == --- incubator/pig/branches/types/src/org/apache/pig/PigServer.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/PigServer.java Fri May 23 16:58:27 2008 @@ -231,17 +231,13 @@ } public void dumpSchema(String alias) throws IOException{ -LogicalPlan lp = aliases.get(alias); -if (lp == null) -throw new IOException(Invalid alias - + alias); - try { +LogicalPlan lp = getPlanFromAlias(alias, describe); Schema schema = lp.getLeaves().get(0).getSchema(); System.out.println
svn commit: r659161 [4/4] - in /incubator/pig/branches/types: ./ src/org/apache/pig/data/ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/logicalLayer/schema/ src/org/apache/pig/impl/log
Added: incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidatorNoSchema.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidatorNoSchema.java?rev=659161view=auto == --- incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidatorNoSchema.java (added) +++ incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidatorNoSchema.java Thu May 22 10:12:56 2008 @@ -0,0 +1,1002 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.pig.test; + +import junit.framework.TestCase; +import org.junit.Test; +import org.apache.pig.impl.logicalLayer.*; +import org.apache.pig.impl.logicalLayer.validators.TypeCheckingValidator; +import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.apache.pig.impl.io.FileSpec; +import org.apache.pig.impl.plan.CompilationMessageCollector; +import org.apache.pig.impl.plan.PlanValidationException; +import org.apache.pig.impl.plan.MultiMap; +import org.apache.pig.builtin.PigStorage; +import org.apache.pig.data.DataType; +import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema ; +import static org.apache.pig.test.utils.TypeCheckingTestUtil.* ; +import org.apache.pig.test.utils.TypeCheckingTestUtil; + +import java.util.List; +import java.util.ArrayList; + +public class TestTypeCheckingValidatorNoSchema extends TestCase { + + +@Test +public void testUnion1() throws Throwable { + +printCurrentMethodName(); +LogicalPlan plan = new LogicalPlan() ; + +LOLoad load1 = genDummyLOLoad(plan) ; +LOLoad load2 = genDummyLOLoad(plan) ; + +// set schemas +load1.setEnforcedSchema(null) ; +load2.setEnforcedSchema(null) ; + +// create union operator +ArrayListLogicalOperator inputList = new ArrayListLogicalOperator() ; +inputList.add(load1) ; +inputList.add(load2) ; +LOUnion union = new LOUnion(plan, genNewOperatorKey(), inputList) ; + +// wiring +plan.add(load1) ; +plan.add(load2) ; +plan.add(union) ; + +plan.connect(load1, union); +plan.connect(load2, union); + +// validate +CompilationMessageCollector collector = new CompilationMessageCollector() ; +TypeCheckingValidator typeValidator = new TypeCheckingValidator() ; +typeValidator.validate(plan, collector) ; +printMessageCollector(collector) ; +printTypeGraph(plan) ; + +// check end result schema +Schema outputSchema = union.getSchema() ; +assertEquals(outputSchema, null); + +} + + +@Test +public void testUnion2() throws Throwable { + +printCurrentMethodName(); +LogicalPlan plan = new LogicalPlan() ; + +LOLoad load1 = genDummyLOLoad(plan) ; +LOLoad load2 = genDummyLOLoad(plan) ; + +String[] aliases = new String[]{ a, b, c } ; +byte[] types = new byte[] { DataType.INTEGER, DataType.LONG, DataType.BYTEARRAY } ; +Schema schema1 = genFlatSchema(aliases, types) ; + +// set schemas +load1.setEnforcedSchema(schema1) ; +load2.setEnforcedSchema(null) ; + +// create union operator +ArrayListLogicalOperator inputList = new ArrayListLogicalOperator() ; +inputList.add(load1) ; +inputList.add(load2) ; +LOUnion union = new LOUnion(plan, genNewOperatorKey(), inputList) ; + +// wiring +plan.add(load1) ; +plan.add(load2) ; +plan.add(union) ; + +plan.connect(load1, union); +plan.connect(load2, union); + +// validate +CompilationMessageCollector collector = new CompilationMessageCollector() ; +TypeCheckingValidator typeValidator = new TypeCheckingValidator() ; +typeValidator.validate(plan, collector) ; +printMessageCollector(collector) ; +printTypeGraph(plan) ; + +// check end result schema +Schema outputSchema = union.getSchema() ; +assertEquals(outputSchema, null); + +} + + +// Positive expression cond
svn commit: r659161 [1/4] - in /incubator/pig/branches/types: ./ src/org/apache/pig/data/ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/logicalLayer/schema/ src/org/apache/pig/impl/log
Author: gates Date: Thu May 22 10:12:56 2008 New Revision: 659161 URL: http://svn.apache.org/viewvc?rev=659161view=rev Log: PIG-143 Pi's latest type checker checkin. Added: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/SchemaMergeException.java incubator/pig/branches/types/test/org/apache/pig/test/TestTypeChecking.java incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidatorNoSchema.java incubator/pig/branches/types/test/org/apache/pig/test/utils/TypeCheckingTestUtil.java Modified: incubator/pig/branches/types/build.xml incubator/pig/branches/types/src/org/apache/pig/data/DataType.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOCogroup.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOCross.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOForEach.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOLoad.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOProject.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSort.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSplit.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSplitOutput.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOUnion.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalOperator.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlan.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java incubator/pig/branches/types/test/org/apache/pig/test/TestSchema.java incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidator.java Modified: incubator/pig/branches/types/build.xml URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/build.xml?rev=659161r1=659160r2=659161view=diff == --- incubator/pig/branches/types/build.xml (original) +++ incubator/pig/branches/types/build.xml Thu May 22 10:12:56 2008 @@ -277,6 +277,8 @@ exclude name=**/TestPigSplit.java / exclude name=**/TestStoreOld.java / !-- Excluced because we don't want to run them -- +exclude name=**/TypeCheckingTestUtil.java / +exclude name=**/TypeGraphPrinter.java / exclude name=**/TestHelper.java / exclude name=**/TestLargeFile.java / exclude name=**/TestOrderBy.java / Modified: incubator/pig/branches/types/src/org/apache/pig/data/DataType.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/data/DataType.java?rev=659161r1=659160r2=659161view=diff == --- incubator/pig/branches/types/src/org/apache/pig/data/DataType.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/data/DataType.java Thu May 22 10:12:56 2008 @@ -424,10 +424,13 @@ * float, or boolean. */ public static boolean isAtomic(byte dataType) { -return ((dataType == BYTEARRAY) || (dataType == CHARARRAY) || -(dataType == INTEGER) || (dataType == LONG) || -(dataType == FLOAT) || (dataType == DOUBLE) || -(dataType == FLOAT)); +return ((dataType == BYTEARRAY) || +(dataType == CHARARRAY) || +(dataType == INTEGER) || +(dataType == LONG) || +(dataType == FLOAT) || +(dataType == DOUBLE) || +(dataType == BOOLEAN)); } /** @@ -822,4 +825,45 @@ default :return true ; } } + +/*** + * Merge types if possible + * @param type1 + * @param type2 + * @return the merged type, or DataType.ERROR if not successful + */ +public static byte mergeType(byte type1, byte type2) { +// Only legal types can be merged +if ( (!DataType.isUsableType(type1)) || + (!DataType.isUsableType(type2)) ) { +return DataType.ERROR ; +} + +// Same type is OK +if (type1==type2) { +return type1 ; +} + +// Both are number so we return the bigger type +if ( (DataType.isNumberType(type1)) + (DataType.isNumberType(type2)) ) { +return type1type2 ? type1:type2 ; +} + +// One is bytearray and the other is (number or chararray) +if ( (type1 == DataType.BYTEARRAY) +( (type2 == DataType.CHARARRAY) || (DataType.isNumberType(type2
svn commit: r659206 - in /incubator/pig/branches/types: src/org/apache/pig/ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/logicalLayer/parser/ test/org/apache/pig/test/
Author: gates Date: Thu May 22 12:04:25 2008 New Revision: 659206 URL: http://svn.apache.org/viewvc?rev=659206view=rev Log: PIG-159 Santhosh's work to chain together logical plans. Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java incubator/pig/branches/types/src/org/apache/pig/StandAloneParser.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java incubator/pig/branches/types/test/org/apache/pig/test/TestTypeChecking.java Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/PigServer.java?rev=659206r1=659205r2=659206view=diff == --- incubator/pig/branches/types/src/org/apache/pig/PigServer.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/PigServer.java Thu May 22 12:04:25 2008 @@ -84,7 +84,7 @@ } -MapString, LogicalPlan aliases = new HashMapString, LogicalPlan(); +MapLogicalOperator, LogicalPlan aliases = new HashMapLogicalOperator, LogicalPlan(); MapOperatorKey, LogicalOperator opTable = new HashMapOperatorKey, LogicalOperator(); MapString, LogicalOperator aliasOp = new HashMapString, LogicalOperator(); MapString, ExpressionOperator defineAliases = new HashMapString, ExpressionOperator(); @@ -252,7 +252,7 @@ * result */ public IteratorTuple openIterator(String id) throws IOException { -if (!aliases.containsKey(id)) +if (!aliases.containsKey(aliasOp.get(id))) throw new IOException(Invalid alias: + id); try { @@ -471,7 +471,14 @@ } public MapString, LogicalPlan getAliases() { -return this.aliases; +MapString, LogicalPlan aliasPlans = new HashMapString, LogicalPlan(); +for(LogicalOperator op: this.aliases.keySet()) { +String alias = op.getAlias(); +if(null != alias) { +aliasPlans.put(alias, this.aliases.get(op)); +} +} +return aliasPlans; } public void shutdown() { Modified: incubator/pig/branches/types/src/org/apache/pig/StandAloneParser.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/StandAloneParser.java?rev=659206r1=659205r2=659206view=diff == --- incubator/pig/branches/types/src/org/apache/pig/StandAloneParser.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/StandAloneParser.java Thu May 22 12:04:25 2008 @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.InputStreamReader; import java.util.Iterator; +import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -28,6 +29,7 @@ import org.apache.pig.backend.executionengine.ExecException; import org.apache.pig.impl.logicalLayer.FrontendException; import org.apache.pig.impl.logicalLayer.LogicalPlan; +import org.apache.pig.impl.logicalLayer.LogicalOperator; public class StandAloneParser { @@ -66,9 +68,10 @@ try{ pig.registerQuery(query); System.out.print(Current aliases: ); -for (IteratorString it = pig.getAliases().keySet().iterator(); it.hasNext(); ) { +MapString, LogicalPlan aliasPlan = pig.getAliases(); +for (IteratorString it = aliasPlan.keySet().iterator(); it.hasNext(); ) { String alias = it.next(); -LogicalPlan lp = pig.getAliases().get(alias); +LogicalPlan lp = aliasPlan.get(alias); System.out.print(alias + - + lp.getLeaves().get(0).getSchema()); if (it.hasNext()) System.out.print(, \n); else System.out.print(\n); Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java?rev=659206r1=659205r2=659206view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java Thu May 22 12:04:25 2008 @@ -40,7 +40,7 @@ public LogicalPlan parse(String scope, String query, - MapString, LogicalPlan aliases, + MapLogicalOperator, LogicalPlan
svn commit: r659250 - in /incubator/pig/branches/types/src/org/apache/pig: PigServer.java backend/executionengine/ExecutionEngine.java backend/hadoop/executionengine/HExecutionEngine.java backend/loca
Author: gates Date: Thu May 22 13:53:05 2008 New Revision: 659250 URL: http://svn.apache.org/viewvc?rev=659250view=rev Log: Changed ExecutionEngine.compile implementations and PigServer to work with the new way logical plans are stitched together. Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java incubator/pig/branches/types/src/org/apache/pig/backend/executionengine/ExecutionEngine.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/LocalExecutionEngine.java Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/PigServer.java?rev=659250r1=659249r2=659250view=diff == --- incubator/pig/branches/types/src/org/apache/pig/PigServer.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/PigServer.java Thu May 22 13:53:05 2008 @@ -54,8 +54,10 @@ import org.apache.pig.impl.logicalLayer.parser.ParseException; import org.apache.pig.impl.logicalLayer.parser.QueryParser; import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.apache.pig.impl.logicalLayer.validators.LogicalPlanValidationExecutor; import org.apache.pig.impl.physicalLayer.POPrinter; import org.apache.pig.impl.physicalLayer.plans.PhysicalPlan; +import org.apache.pig.impl.plan.CompilationMessageCollector; import org.apache.pig.impl.plan.VisitorException; import org.apache.pig.impl.util.WrappedIOException; @@ -356,8 +358,8 @@ public void explain(String alias, PrintStream stream) throws IOException { try { +LogicalPlan lp = compileLp(alias, explain); stream.println(Logical Plan:); -LogicalPlan lp = compileLp(); LOPrinter lv = new LOPrinter(stream, lp); lv.visit(); @@ -493,18 +495,53 @@ private ExecJob execute(String jobName) throws ExecException { ExecJob job = null; -LogicalPlan lp = compileLp(); +LogicalPlan lp = compileLp(jobName, execute); PhysicalPlan pp = compilePp(lp); // execute using appropriate engine return pigContext.getExecutionEngine().execute(pp, jobName); } -// TODO FIX -private LogicalPlan compileLp() { -LogicalPlan lp = null; -// TODO, stitch together logical plans +private LogicalPlan compileLp(String alias, String op) throws ExecException { +// Look up the logical plan in the aliases map. That plan will be +// properly connected to all the others. +LogicalPlan lp = aliases.get(alias); +if (lp == null) { +throw new ExecException(No alias + alias + to + op); +} + +// run through validator +LogicalPlanValidationExecutor validator = +new LogicalPlanValidationExecutor(lp, pigContext); +CompilationMessageCollector collector = new CompilationMessageCollector() ; +validator.validate(lp, collector); +// Check to see if we had any problems. +StringBuilder sb = new StringBuilder(); +for (CompilationMessageCollector.Message msg : collector) { +switch (msg.getMessageType()) { +case Info: +log.info(msg.getMessage()); +break; + +case Warning: +log.warn(msg.getMessage()); +break; + +case Unknown: +case Error: +log.error(msg.getMessage()); +sb.append(msg.getMessage()); +break; + +default: +throw new AssertionError(Unknown message type + +msg.getMessageType()); -// TODO run through validator +} +} + +if (sb.length() 0) { +throw new ExecException(sb.toString()); +} // TODO optimize Modified: incubator/pig/branches/types/src/org/apache/pig/backend/executionengine/ExecutionEngine.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/executionengine/ExecutionEngine.java?rev=659250r1=659249r2=659250view=diff == --- incubator/pig/branches/types/src/org/apache/pig/backend/executionengine/ExecutionEngine.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/backend/executionengine/ExecutionEngine.java Thu May 22 13:53:05 2008 @@ -93,8 +93,10 @@ public PhysicalPlan compile(LogicalPlan plan, Properties properties) throws ExecException; +/* public PhysicalPlan compile(LogicalPlan[] plans, Properties properties) throws ExecException
svn commit: r658728 [2/2] - in /incubator/pig/branches/types: src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/logicalLayer/validators/ src/org/apache/pig/impl/physicalLayer/plans/ src/or
Added: incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java?rev=658728view=auto == --- incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java (added) +++ incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java Wed May 21 08:40:49 2008 @@ -0,0 +1,434 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.pig.test; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.pig.ExecType; +import org.apache.pig.backend.executionengine.ExecException; +import org.apache.pig.builtin.PigStorage; +import org.apache.pig.impl.PigContext; +import org.apache.pig.impl.logicalLayer.ExpressionOperator; +import org.apache.pig.impl.logicalLayer.LogToPhyTranslationVisitor; +import org.apache.pig.impl.logicalLayer.LogicalOperator; +import org.apache.pig.impl.logicalLayer.LogicalPlan; +import org.apache.pig.impl.logicalLayer.LogicalPlanBuilder; +import org.apache.pig.impl.logicalLayer.OperatorKey; +import org.apache.pig.impl.physicalLayer.plans.PhysicalPlan; +import org.apache.pig.impl.plan.VisitorException; +import org.junit.Test; + +/** + * All new tests should be included at the end of the existing test cases. This is to ensure that + * nodeIdGenerator produces the same sequence as in the golden files for the previously existing test cases + * + * To generate golden files, use the following code : + * ByteArrayOutputStream baos = new ByteArrayOutputStream(); + * pp.explain(baos); + * FileOutputStream fos = new FileOutputStream(test/org/apache/pig/test/data/GoldenFiles/Union.gld); + * fos.write(baos.toByteArray()); + * + */ +public class TestLogToPhyCompiler extends junit.framework.TestCase { + +private final Log log = LogFactory.getLog(getClass()); + +File A; +final int MAX_RANGE = 10; + +Random r = new Random(); +PigContext pc = new PigContext(ExecType.LOCAL); + + +private void writeData(File input, int noTuples, int arityOfTuples, char separator) throws IOException { + FileOutputStream dat = new FileOutputStream(input); + +for(int i = 0; i noTuples; i++) { + +for(int j = 0; j arityOfTuples; j++) { + int temp = r.nextInt(MAX_RANGE); + if(j == arityOfTuples - 1) { + dat.write((temp + \n).getBytes()); + } else { + dat.write((temp + + separator).getBytes()); + } +} +} + +dat.close(); +} + +@Test +public void testComplexForeach() throws VisitorException, IOException { +/*String query = foreach (load 'a') { + +B = FILTER $0 BY (($1 == $2) AND ('a' 'b')); + +generate B; + +};;*/ + String query = foreach (load 'a') { + +B = FILTER $0 BY ($1 == $2); + +generate B; + +};; +LogicalPlan plan = buildPlan(query); + PhysicalPlan pp = buildPhysicalPlan(plan); + int MAX_SIZE = 10; + ByteArrayOutputStream baos = new ByteArrayOutputStream(); +pp.explain(baos); +String compiledPlan = baos.toString(); + FileInputStream fis = new FileInputStream(test/org/apache/pig/test/data/GoldenFiles/ComplexForeach.gld); +byte[] b = new byte[MAX_SIZE]; +int len = fis.read(b); +String goldenPlan = new String(b, 0, len); + +System.out.println(); +System.out.println(compiledPlan); +System.out.println(-); +//System.out.println(compiledPlan.compareTo(goldenPlan)==0); +assertEquals(true, compiledPlan.compareTo(goldenPlan) ==
svn commit: r657836 - in /incubator/pig/branches/types: src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt test/org/apache/pig/test/TestLogicalPlanBuilder.java
Author: gates Date: Mon May 19 07:30:30 2008 New Revision: 657836 URL: http://svn.apache.org/viewvc?rev=657836view=rev Log: PIG-159 Santhosh's addition of explicit casts to the grammar. Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt?rev=657836r1=657835r2=657836view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt Mon May 19 07:30:30 2008 @@ -463,6 +463,8 @@ splitOp = new LOSplit(lp, input.getOperatorKey(), new ArrayListLogicalOperator(), new HashMapString,LogicalPlan()); lp.add(splitOp); log.debug(Adding operator + splitOp.getClass().getName() + to the logical plan); +lp.connect(input, splitOp); + log.debug(Connected alias: + input.getAlias() + operator + input.getClass().getName() + to operator + splitOp.getClass().getName()); } alias = IDENTIFIER IF cond = PCond(input.getSchema(), null, condPlan = new LogicalPlan(), input) { @@ -881,8 +883,6 @@ } } - - LogicalOperator CogroupClause(LogicalPlan lp) : { CogroupInput gi; @@ -1488,9 +1488,11 @@ } { ( - lhs = UnaryExpr(over,specs,lp,input) +// lhs = UnaryExpr(over,specs,lp,input) + lhs = CastExpr(over,specs,lp,input) ( - ( t = STAR | t = / | t = %) rhs = UnaryExpr(over,specs,lp,input) +// ( t = STAR | t = / | t = %) rhs = UnaryExpr(over,specs,lp,input) + ( t = STAR | t = / | t = %) rhs = CastExpr(over,specs,lp,input) { assertAtomic(lhs,true); assertAtomic(rhs,true); @@ -1517,6 +1519,32 @@ } } +ExpressionOperator CastExpr(Schema over, MapString, LogicalOperator specs,LogicalPlan lp,LogicalOperator input) : +{ +byte type = DataType.BYTEARRAY; +ExpressionOperator cast; +ExpressionOperator exprOp; +boolean castRequired = false; +log.trace(Entering Cast); +} +{ +[LOOKAHEAD(2)( type = Type() {castRequired = true;})] exprOp = UnaryExpr(over, specs, lp, input) +{ +if(castRequired) { +cast = new LOCast(lp, new OperatorKey(scope, getNextId()), exprOp, type); +lp.add(cast); + log.debug(Added operator + cast.getClass().getName() + + cast + to logical plan + lp); +lp.connect(exprOp, cast); + log.debug(Connected operator + exprOp.getClass().getName() + + exprOp + to + cast + logical plan + lp); +log.trace(Exiting Cast); +return cast; +} else { +log.trace(Exiting Cast); +return exprOp; +} +} +} + ExpressionOperator UnaryExpr(Schema over, MapString, LogicalOperator specs, LogicalPlan lp, LogicalOperator input) : { ExpressionOperator expr; @@ -1539,7 +1567,8 @@ log.trace(Entering NegativeExpr); } { - - c1=UnaryExpr(over,specs,lp,input) +// - c1=UnaryExpr(over,specs,lp,input) + - c1=CastExpr(over,specs,lp,input) { ExpressionOperator eOp = new LONegative(lp, new OperatorKey(scope, getNextId()), c1); lp.add(eOp); @@ -1550,7 +1579,6 @@ } } - ExpressionOperator BaseEvalSpec(Schema over, MapString, LogicalOperator specs, LogicalPlan lp, LogicalOperator input) : { Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java?rev=657836r1=657835r2=657836view=diff == --- incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java (original) +++ incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java Mon May 19 07:30:30 2008 @@ -843,6 +843,28 @@ } } +@Test +public void testQuery71() { +buildPlan(split (load 'a') into x if $0 '7', y if $0 '7';); +buildPlan(b = foreach x generate $0;); +buildPlan(c = foreach y generate $1;); +} + +@Test +public void testQuery72() { +buildPlan(split (load 'a') into x if $0 '7', y if $0 '7';); +buildPlan(b = foreach x generate
svn commit: r657848 - in /incubator/pig/branches/types: ./ src/org/apache/pig/ src/org/apache/pig/builtin/ src/org/apache/pig/data/ src/org/apache/pig/impl/logicalLayer/
Author: gates Date: Mon May 19 08:15:19 2008 New Revision: 657848 URL: http://svn.apache.org/viewvc?rev=657848view=rev Log: Fixed a few TODO FIXs, added DOUBLE to isAtomic in DataType. Modified: incubator/pig/branches/types/build.xml incubator/pig/branches/types/src/org/apache/pig/PigServer.java incubator/pig/branches/types/src/org/apache/pig/builtin/ARITY.java incubator/pig/branches/types/src/org/apache/pig/builtin/AVG.java incubator/pig/branches/types/src/org/apache/pig/builtin/COUNT.java incubator/pig/branches/types/src/org/apache/pig/builtin/MAX.java incubator/pig/branches/types/src/org/apache/pig/builtin/MIN.java incubator/pig/branches/types/src/org/apache/pig/builtin/SUM.java incubator/pig/branches/types/src/org/apache/pig/data/DataType.java incubator/pig/branches/types/src/org/apache/pig/data/DefaultAbstractBag.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOProject.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOVisitor.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java Modified: incubator/pig/branches/types/build.xml URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/build.xml?rev=657848r1=657847r2=657848view=diff == --- incubator/pig/branches/types/build.xml (original) +++ incubator/pig/branches/types/build.xml Mon May 19 08:15:19 2008 @@ -134,6 +134,7 @@ target name=compile-sources javac encoding=${build.encoding} srcdir=${sources} includes=**/plan/*.java, **/plan/optimizer/*.java, **/data/*.java, **/pig/builtin/*.java, +**/impl/io/*.java, **/impl/mapReduceLayer/*.java, **/test/utils/*.java, **/test/TestOperatorPlan.java, **/test/TestBuiltin.java, **/test/TestConstExpr.java, **/test/TestFilter.java, **/test/TestPhyOp.java, **/test/TestAdd.java, **/test/TestSubtract.java, **/test/TestMultiply.java, Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/PigServer.java?rev=657848r1=657847r2=657848view=diff == --- incubator/pig/branches/types/src/org/apache/pig/PigServer.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/PigServer.java Mon May 19 08:15:19 2008 @@ -257,7 +257,6 @@ LogicalPlan readFrom = (LogicalPlan) aliases.get(id); -// TODO FIX Make this work try { ExecPhysicalPlan pp = pigContext.getExecutionEngine().compile(readFrom, null); Modified: incubator/pig/branches/types/src/org/apache/pig/builtin/ARITY.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/builtin/ARITY.java?rev=657848r1=657847r2=657848view=diff == --- incubator/pig/branches/types/src/org/apache/pig/builtin/ARITY.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/builtin/ARITY.java Mon May 19 08:15:19 2008 @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.pig.EvalFunc; +import org.apache.pig.data.DataType; import org.apache.pig.data.Tuple; import org.apache.pig.impl.logicalLayer.schema.Schema; @@ -32,8 +33,6 @@ @Override public Schema outputSchema(Schema input) { -// TODO FIX -// return new AtomSchema(arity); -return null; +return new Schema(new Schema.FieldSchema(null, DataType.INTEGER)); } } Modified: incubator/pig/branches/types/src/org/apache/pig/builtin/AVG.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/builtin/AVG.java?rev=657848r1=657847r2=657848view=diff == --- incubator/pig/branches/types/src/org/apache/pig/builtin/AVG.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/builtin/AVG.java Mon May 19 08:15:19 2008 @@ -162,9 +162,7 @@ @Override public Schema outputSchema(Schema input) { -// TODO FIX -// return new AtomSchema(average); -return null; +return new Schema(new Schema.FieldSchema(null, DataType.DOUBLE)); } } Modified: incubator/pig/branches/types/src/org/apache/pig/builtin/COUNT.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/builtin/COUNT.java?rev=657848r1=657847r2=657848view=diff == --- incubator/pig/branches/types/src/org/apache/pig/builtin/COUNT.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/builtin
svn commit: r657939 - in /incubator/pig/branches/types: ./ src/org/apache/pig/ src/org/apache/pig/backend/hadoop/executionengine/ src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/ src/o
Author: gates Date: Mon May 19 12:45:33 2008 New Revision: 657939 URL: http://svn.apache.org/viewvc?rev=657939view=rev Log: Make it all compile. With this patch we can now return the build.xml compile-sources rule to its previous state of building all java files below src. There are a number of places where code is commented out (all marked TODO FIX). These will be addressed over the next few days as we tie together loose ends. All non-map reduce unit tests also now run, and pass. Removed: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/POMapreduce.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/SplitSpec.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/MapReduceLauncher.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigCombine.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigInputFormat.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigMapReduce.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigOutputFormat.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigSplit.java incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/POCogroup.java incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/POEval.java incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/POLoad.java incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/POSort.java incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/POSplit.java incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/POStore.java incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/POUnion.java incubator/pig/branches/types/src/org/apache/pig/impl/builtin/ADD.java incubator/pig/branches/types/src/org/apache/pig/impl/builtin/DIVIDE.java incubator/pig/branches/types/src/org/apache/pig/impl/builtin/MULTIPLY.java incubator/pig/branches/types/src/org/apache/pig/impl/builtin/SUBTRACT.java incubator/pig/branches/types/src/org/apache/pig/impl/eval/cond/AndCond.java incubator/pig/branches/types/src/org/apache/pig/impl/eval/cond/CompCond.java incubator/pig/branches/types/src/org/apache/pig/impl/eval/cond/Cond.java incubator/pig/branches/types/src/org/apache/pig/impl/eval/cond/FalseCond.java incubator/pig/branches/types/src/org/apache/pig/impl/eval/cond/FuncCond.java incubator/pig/branches/types/src/org/apache/pig/impl/eval/cond/NotCond.java incubator/pig/branches/types/src/org/apache/pig/impl/eval/cond/OrCond.java incubator/pig/branches/types/src/org/apache/pig/impl/eval/cond/RegexpCond.java incubator/pig/branches/types/src/org/apache/pig/impl/eval/cond/TrueCond.java incubator/pig/branches/types/src/org/apache/pig/impl/eval/window/TimeWindowSpec.java incubator/pig/branches/types/src/org/apache/pig/impl/eval/window/TupleWindowSpec.java incubator/pig/branches/types/src/org/apache/pig/impl/eval/window/WindowSpec.java Modified: incubator/pig/branches/types/build.xml incubator/pig/branches/types/src/org/apache/pig/Main.java incubator/pig/branches/types/src/org/apache/pig/PigServer.java incubator/pig/branches/types/src/org/apache/pig/StandAloneParser.java incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/MapRedPhysicalPlan.java incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/LocalExecutionEngine.java incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/LocalPhysicalPlan.java incubator/pig/branches/types/src/org/apache/pig/builtin/TOKENIZE.java incubator/pig/branches/types/src/org/apache/pig/impl/builtin/ShellBagEvalFunc.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalPlan.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POPrinter.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POVisitor.java incubator/pig/branches/types/test/org/apache/pig/test/NonDefaultBagFactory.java incubator/pig/branches/types/test/org/apache/pig/test/TestDataBag.java incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java incubator/pig/branches/types/test/org/apache/pig/test/TestPigFile.java Modified: incubator/pig/branches/types/build.xml URL: http://svn.apache.org/viewvc/incubator/pig/branches
svn commit: r657224 - in /incubator/pig/branches/types: src/org/apache/pig/ src/org/apache/pig/impl/mapReduceLayer/ src/org/apache/pig/impl/physicalLayer/ src/org/apache/pig/impl/physicalLayer/topLeve
Author: gates Date: Fri May 16 14:38:40 2008 New Revision: 657224 URL: http://svn.apache.org/viewvc?rev=657224view=rev Log: PIG-162 Shravan's addition of PigProgressable and distinct for MR. Added: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/ProgressableReporter.java incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC16.gld Removed: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/RunnableReporter.java Modified: incubator/pig/branches/types/src/org/apache/pig/ComparisonFunc.java incubator/pig/branches/types/src/org/apache/pig/EvalFunc.java incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/PigMapBase.java incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/PigMapReduce.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/PigProgressable.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POGenerate.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POPackage.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POUnion.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/PhysicalOperator.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POUserComparisonFunc.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POUserFunc.java incubator/pig/branches/types/test/org/apache/pig/test/TestMRCompiler.java Modified: incubator/pig/branches/types/src/org/apache/pig/ComparisonFunc.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/ComparisonFunc.java?rev=657224r1=657223r2=657224view=diff == --- incubator/pig/branches/types/src/org/apache/pig/ComparisonFunc.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/ComparisonFunc.java Fri May 16 14:38:40 2008 @@ -23,9 +23,14 @@ import org.apache.hadoop.io.WritableComparator; import org.apache.pig.data.Tuple; import org.apache.pig.data.TupleFactory; +import org.apache.pig.impl.physicalLayer.PigProgressable; public abstract class ComparisonFunc extends WritableComparator { +// If the comparison is a time consuming process +// this reporter must be used to report progress +protected PigProgressable reporter; + public ComparisonFunc() { super(TupleFactory.getInstance().tupleClass()); } @@ -48,4 +53,8 @@ * @see java.util.Comparator */ abstract public int compare(Tuple t1, Tuple t2); + +public void setReporter(PigProgressable reporter) { +this.reporter = reporter; +} } Modified: incubator/pig/branches/types/src/org/apache/pig/EvalFunc.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/EvalFunc.java?rev=657224r1=657223r2=657224view=diff == --- incubator/pig/branches/types/src/org/apache/pig/EvalFunc.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/EvalFunc.java Fri May 16 14:38:40 2008 @@ -26,6 +26,7 @@ // TODO FIX // import org.apache.pig.impl.PigContext; import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.apache.pig.impl.physicalLayer.PigProgressable; /** @@ -44,6 +45,9 @@ * */ public abstract class EvalFuncT { +// UDFs must use this to report progress +// if the exec is taking more that 300 ms +protected PigProgressable reporter; protected Type returnType; @@ -107,16 +111,7 @@ // report that progress is being made (otherwise hadoop times out after 600 seconds working on one outer tuple) protected void progress() { -//This part appears to be unused and is causing problems due to changing hadoop signature -/* -if (PigMapReduce.reporter != null) { -try { -PigMapReduce.reporter.progress(); -} catch (IOException ignored) { -} -} -*/ - +if(reporter!=null) reporter.progress(); } /** @@ -158,4 +153,14 @@ public boolean isAsynchronous(){ return false; } + + +public PigProgressable getReporter() { +return reporter; +} + + +public void setReporter(PigProgressable reporter) { +this.reporter = reporter; +} } Modified: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java?rev=657224r1=657223r2=657224view=diff
svn commit: r656811 - in /incubator/pig/branches/types: ./ src/org/apache/pig/impl/physicalLayer/plans/ src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ test/org/apache/pig
Author: gates Date: Thu May 15 12:49:36 2008 New Revision: 656811 URL: http://svn.apache.org/viewvc?rev=656811view=rev Log: PIG-161 Checking in POMap parts of Shubham's CastAndMapLookupPatch. See PIG-161 for comments on cast part. Added: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POMapLookUp.java incubator/pig/branches/types/test/org/apache/pig/test/TestPOMapLookUp.java Modified: incubator/pig/branches/types/build.xml incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java Modified: incubator/pig/branches/types/build.xml URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/build.xml?rev=656811r1=656810r2=656811view=diff == --- incubator/pig/branches/types/build.xml (original) +++ incubator/pig/branches/types/build.xml Thu May 15 12:49:36 2008 @@ -146,7 +146,7 @@ **/test/TestPODistinct.java, **/test/TestPOSort.java, **/test/TestSchema.java, **/test/TestLogicalPlanBuilder.java,**/test/TestUnion.java, **/test/TestMRCompiler.java, **/test/FakeFSInputStream.java, **/test/Util.java, **/test/TestJobSubmission.java, - **/test/TestLocalJobSubmission.java, + **/test/TestLocalJobSubmission.java, **/test/TestPOMapLookUp.java, **/logicalLayer/*.java, **/logicalLayer/parser/NodeIdGenerator.java, **/logicalLayer/schema/*.java, **/physicalLayer/topLevelOperators/*.java, **/physicalLayer/topLevelOperators/**/*.java, **/physicalLayer/plans/*.java, @@ -276,6 +276,7 @@ include name=**/TestSchema.java / include name=**/TestLogicalPlanBuilder.java / include name=**/TestLocalJobSubmission.java / + include name=**/TestPOMapLookUp.java / !-- include name=**/*Test*.java / Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java?rev=656811r1=656810r2=656811view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java Thu May 15 12:49:36 2008 @@ -22,6 +22,8 @@ import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.ConstantExpression; import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.ExpressionOperator; import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POBinCond; +import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POCast; +import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POMapLookUp; import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POProject; //import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.EqualToExpr; //import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.GTOrEqualToExpr; @@ -118,4 +120,14 @@ } + public void visitMapLookUp(POMapLookUp mapLookUp) { + // TODO Auto-generated method stub + + } + + public void visitCast(POCast cast) { + // TODO Auto-generated method stub + + } + } Added: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POMapLookUp.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POMapLookUp.java?rev=656811view=auto == --- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POMapLookUp.java (added) +++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POMapLookUp.java Thu May 15 12:49:36 2008 @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http
svn commit: r656913 [3/3] - in /incubator/pig/branches/types: ./ src/org/apache/pig/backend/hadoop/executionengine/ src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/ src/org/apache/pig/
Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestPOUserFunc.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestPOUserFunc.java?rev=656913r1=656912r2=656913view=diff == --- incubator/pig/branches/types/test/org/apache/pig/test/TestPOUserFunc.java (original) +++ incubator/pig/branches/types/test/org/apache/pig/test/TestPOUserFunc.java Thu May 15 18:09:30 2008 @@ -40,293 +40,293 @@ import org.apache.pig.impl.physicalLayer.POStatus; import org.apache.pig.impl.physicalLayer.Result; import org.apache.pig.impl.physicalLayer.topLevelOperators.PORead; -import org.apache.pig.impl.physicalLayer.topLevelOperators.POUserComparisonFunc; -import org.apache.pig.impl.physicalLayer.topLevelOperators.POUserFunc; import org.apache.pig.impl.physicalLayer.topLevelOperators.PhysicalOperator; +import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POUserComparisonFunc; +import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POUserFunc; import org.apache.pig.test.utils.GenRandomData; import org.junit.Test; public class TestPOUserFunc extends TestCase { -Random r = new Random(); -int MAX_TUPLES = 10; + Random r = new Random(); + int MAX_TUPLES = 10; -public static class ARITY extends EvalFuncInteger { + public static class ARITY extends EvalFuncInteger { -@Override -public Integer exec(Tuple input) throws IOException { -return new Integer(input.size()); -} - -@Override -public Schema outputSchema(Schema input) { -// TODO FIX -// return new AtomSchema(arity); -return null; -} -} - -public static class WeirdComparator extends ComparisonFunc { - -@Override -public int compare(Tuple t1, Tuple t2) { -// TODO Auto-generated method stub -Object o1 = null; -Object o2 = null; -try { -o1 = t1.get(2); -o2 = t2.get(2); -} catch (ExecException e) { -// TODO Auto-generated catch block -e.printStackTrace(); -} -int i1 = (Integer) o1 - 2; -int i2 = (Integer) o2 - 2; - -return (int) (i1 * i1 - i2 * i2); -} - -} - -/** - * Generates the average of the values of the first field of a tuple. This - * class is Algebraic in implemenation, so if possible the execution will be - * split into a local and global application - */ -public static class AVG extends EvalFuncDouble implements Algebraic { - -private static TupleFactory mTupleFactory = TupleFactory.getInstance(); - -@Override -public Double exec(Tuple input) throws IOException { -double sum = 0; -double count = 0; - -try { -sum = sum(input); -count = count(input); -} catch (ExecException e) { -e.printStackTrace(); -} - -double avg = 0; -if (count 0) -avg = sum / count; - -return new Double(avg); -} - -public String getInitial() { -return Initial.class.getName(); -} - -public String getIntermed() { -return Intermed.class.getName(); -} - -public String getFinal() { -return Final.class.getName(); -} - -static public class Initial extends EvalFuncTuple { -@Override -public Tuple exec(Tuple input) throws IOException { -try { -Tuple t = mTupleFactory.newTuple(2); -t.set(0, new Double(sum(input))); -t.set(1, new Long(count(input))); -return t; -} catch (ExecException t) { -throw new RuntimeException(t.getMessage() + : + input); -} -} -} - -static public class Intermed extends EvalFuncTuple { -@Override -public Tuple exec(Tuple input) throws IOException { -DataBag b = null; -Tuple t = null; -try { -b = (DataBag) input.get(0); -t = combine(b); -} catch (ExecException e) { -// TODO Auto-generated catch block -e.printStackTrace(); -} -return t; -} -} - -static public class Final extends EvalFuncDouble { -@Override -public Double exec(Tuple input) throws IOException { -double sum = 0; -double count = 0; -try { -DataBag b = (DataBag) input.get(0); -
svn commit: r655940 - in /incubator/pig/trunk: CHANGES.txt src/org/apache/pig/Main.java src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java src/org/apache/pig/impl/util/PropertiesU
Author: gates Date: Tue May 13 09:35:44 2008 New Revision: 655940 URL: http://svn.apache.org/viewvc?rev=655940view=rev Log: PIG-236: Fix properties so that values specified via the command line (-D) are not ignored. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/Main.java incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java incubator/pig/trunk/src/org/apache/pig/impl/util/PropertiesUtil.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=655940r1=655939r2=655940view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Tue May 13 09:35:44 2008 @@ -293,3 +293,5 @@ PIG-232: let valid cache specifications through (acmurthy via olgan) PIG-237: validation of the output directory (pi_song via olgan) + +PIG-236: Fix properties so that values specified via the command line (-D) are not ignored (pkamath via gates). Modified: incubator/pig/trunk/src/org/apache/pig/Main.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/Main.java?rev=655940r1=655939r2=655940view=diff == --- incubator/pig/trunk/src/org/apache/pig/Main.java (original) +++ incubator/pig/trunk/src/org/apache/pig/Main.java Tue May 13 09:35:44 2008 @@ -206,8 +206,6 @@ } } } -// set the cluster -properties.setProperty(HExecutionEngine.JOB_TRACKER_LOCATION, cluster); // configure logging configureLog4J(properties); // create the context with the parameter Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java?rev=655940r1=655939r2=655940view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java Tue May 13 09:35:44 2008 @@ -111,8 +111,9 @@ setSSHFactory(); String hodServer = properties.getProperty(HOD_SERVER); -String cluster = properties.getProperty(JOB_TRACKER_LOCATION); -String nameNode = properties.getProperty( FILE_SYSTEM_LOCATION); +String cluster = null; +String nameNode = null; +Configuration configuration = null; if (hodServer != null hodServer.length() 0) { String hdfsAndMapred[] = doHod(hodServer, properties); @@ -120,6 +121,24 @@ properties.setProperty(JOB_TRACKER_LOCATION, hdfsAndMapred[1]); } else { + +// We need to build a configuration object first in the manner described below +// and then get back a properties object to inspect the JOB_TRACKER_LOCATION +// and FILE_SYSTEM_LOCATION. The reason to do this is if we looked only at +// the existing properties object, we may not get the right settings. So we want +// to read the configurations in the order specified below and only then look +// for JOB_TRACKER_LOCATION and FILE_SYSTEM_LOCATION. + +// Hadoop by default specifies two resources, loaded in-order from the classpath: +// 1. hadoop-default.xml : Read-only defaults for hadoop. +// 2. hadoop-site.xml: Site-specific configuration for a given hadoop installation. +// Now add the settings from properties object to override any existing properties +// All of the above is accomplished in the method call below +configuration = ConfigurationUtil.toConfiguration(properties); +properties = ConfigurationUtil.toProperties(configuration); +cluster = properties.getProperty(JOB_TRACKER_LOCATION); +nameNode = properties.getProperty(FILE_SYSTEM_LOCATION); + if (cluster != null cluster.length() 0) { if(!cluster.contains(:) !cluster.equalsIgnoreCase(LOCAL)) { cluster = cluster + :50020; @@ -137,8 +156,13 @@ log.info(Connecting to hadoop file system at: + (nameNode==null? LOCAL: nameNode) ) ; ds = new HDataStorage(properties); + +// The above HDataStorage constructor sets DEFAULT_REPLICATION_FACTOR_KEY in properties. +// So we need to reconstruct the configuration object for the non HOD case +// In the HOD case, this is the first time the configuration object will be created
svn commit: r655950 - in /incubator/pig/branches/types: src/org/apache/pig/impl/physicalLayer/plans/ src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ src/org/apache/pig/imp
Author: gates Date: Tue May 13 10:11:43 2008 New Revision: 655950 URL: http://svn.apache.org/viewvc?rev=655950view=rev Log: Shubham's bincond and unary operators patch. Added: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POBinCond.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/unaryExprOps/ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/unaryExprOps/PONegative.java incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/unaryExprOps/UnaryExpressionOperator.java incubator/pig/branches/types/test/org/apache/pig/test/TestPOBinCond.java incubator/pig/branches/types/test/org/apache/pig/test/TestPONegative.java Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java?rev=655950r1=655949r2=655950view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java Tue May 13 10:11:43 2008 @@ -21,6 +21,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.ConstantExpression; import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.ExpressionOperator; +import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POBinCond; import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POProject; //import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.EqualToExpr; //import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.GTOrEqualToExpr; @@ -35,6 +36,7 @@ import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.LTOrEqualToExpr; import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.LessThanExpr; import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.NotEqualToExpr; +import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.unaryExprOps.PONegative; import org.apache.pig.impl.plan.PlanWalker; /** @@ -105,4 +107,14 @@ //do nothing } + public void visitBinCond(POBinCond binCond) { + // do nothing + + } + + public void visitNegative(PONegative negative) { + //do nothing + + } + } Added: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POBinCond.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POBinCond.java?rev=655950view=auto == --- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POBinCond.java (added) +++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POBinCond.java Tue May 13 10:11:43 2008 @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators; + +import java.util.Map; + +import org.apache.pig.backend.executionengine.ExecException; +import org.apache.pig.data.DataBag; +import org.apache.pig.data.DataByteArray; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.logicalLayer.OperatorKey; +import org.apache.pig.impl.physicalLayer.Result
svn commit: r654629 [2/4] - in /incubator/pig/branches/types: ./ src/org/apache/pig/impl/eval/ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/logicalLayer/parser/ src/org/apache/pig/imp
Added: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LORegexp.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LORegexp.java?rev=654629view=auto == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LORegexp.java (added) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LORegexp.java Thu May 8 14:25:22 2008 @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.pig.impl.logicalLayer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.pig.data.DataType; +import org.apache.pig.impl.plan.VisitorException; +import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.apache.pig.impl.plan.PlanVisitor; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +public class LORegexp extends ExpressionOperator { +private static final long serialVersionUID = 2L; + +/** + * The expression and the column to be projected. + */ +private ExpressionOperator mOperand; +private String mRegexp; +private static Log log = LogFactory.getLog(LORegexp.class); + +/** + * + * @param plan + *Logical plan this operator is a part of. + * @param k + *Operator key to assign to this node. + * @param exp + *the expression which might contain the column to project + * @param projection + *the list of columns to project + */ +public LORegexp(LogicalPlan plan, OperatorKey key, +ExpressionOperator operand, String regexp) { +super(plan, key); +mOperand = operand; +mRegexp = regexp; +} + +public ExpressionOperator getOperand() { +return mOperand; +} + +public String getRegexp() { +return mRegexp; +} + +@Override +public String name() { +return Project + mKey.scope + - + mKey.id; +} + +@Override +public boolean supportsMultipleInputs() { +return false; +} + +@Override +public Schema getSchema() { +return mSchema; +} + +@Override +public void visit(LOVisitor v) throws VisitorException { +v.visit(this); +} + +} Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSort.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSort.java?rev=654629r1=654628r2=654629view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSort.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOSort.java Thu May 8 14:25:22 2008 @@ -26,19 +26,26 @@ import org.apache.pig.impl.logicalLayer.schema.Schema; import org.apache.pig.impl.plan.VisitorException; import org.apache.pig.impl.plan.PlanVisitor; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; public class LOSort extends LogicalOperator { private static final long serialVersionUID = 2L; -private ListInteger mSortCols; +private LogicalOperator mInput; private ListBoolean mAscCols; -private LOUserFunc mSortFunc; +private String mSortFunc; +private boolean mIsStar = false; +private ListLogicalPlan mSortColPlans; + private static Log log = LogFactory.getLog(LOSort.class); /** * @param plan *LogicalPlan this operator is a part of. * @param key *OperatorKey for this operator + * @param input + *Input to sort * @param sortCols *Array of column numbers that will be used for sorting data. * @param ascCols @@ -47,33 +54,40 @@ *this array is null, then all columns will be sorted ascending. * @param sorFunc *the user defined sorting function - * @param rp - *Requested level of parallelism to be used in the sort.
svn commit: r654629 [4/4] - in /incubator/pig/branches/types: ./ src/org/apache/pig/impl/eval/ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/logicalLayer/parser/ src/org/apache/pig/imp
Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java?rev=654629r1=654628r2=654629view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java Thu May 8 14:25:22 2008 @@ -349,7 +349,7 @@ private void insertLeftCastForBinaryOp(BinaryExpressionOperator binOp, byte toType ) { OperatorKey newKey = genNewOperatorKey(binOp) ; -LOCast cast = new LOCast(mPlan, newKey, 1, binOp.getLhsOperand(), toType) ; +LOCast cast = new LOCast(mPlan, newKey, binOp.getLhsOperand(), toType) ; mPlan.add(cast) ; mPlan.disconnect(binOp.getLhsOperand(), binOp) ; try { @@ -367,7 +367,7 @@ private void insertRightCastForBinaryOp(BinaryExpressionOperator binOp, byte toType ) { OperatorKey newKey = genNewOperatorKey(binOp) ; -LOCast cast = new LOCast(mPlan, newKey, 1, binOp.getRhsOperand(), toType) ; +LOCast cast = new LOCast(mPlan, newKey, binOp.getRhsOperand(), toType) ; mPlan.add(cast) ; mPlan.disconnect(binOp.getRhsOperand(), binOp) ; try { @@ -425,7 +425,7 @@ // All uniOps at the moment only work with Expression input ExpressionOperator input = (ExpressionOperator) list.get(0) ; OperatorKey newKey = genNewOperatorKey(uniOp) ; -LOCast cast = new LOCast(mPlan, newKey, 1, input, toType) ; +LOCast cast = new LOCast(mPlan, newKey, input, toType) ; mPlan.disconnect(input, uniOp) ; try { @@ -518,7 +518,7 @@ private void insertLeftCastForBinCond(LOBinCond binCond, byte toType) { OperatorKey newKey = genNewOperatorKey(binCond) ; -LOCast cast = new LOCast(mPlan, newKey, 1, binCond.getLhsOp(), toType) ; +LOCast cast = new LOCast(mPlan, newKey, binCond.getLhsOp(), toType) ; mPlan.add(cast) ; mPlan.disconnect(binCond.getLhsOp(), binCond) ; try { @@ -535,7 +535,7 @@ private void insertRightCastForBinCond(LOBinCond binCond, byte toType) { OperatorKey newKey = genNewOperatorKey(binCond) ; -LOCast cast = new LOCast(mPlan, newKey, 1, binCond.getRhsOp(), toType) ; +LOCast cast = new LOCast(mPlan, newKey, binCond.getRhsOp(), toType) ; mPlan.add(cast) ; mPlan.disconnect(binCond.getRhsOp(), binCond) ; try { @@ -831,6 +831,8 @@ */ // TODO: NOT DONE YET + //COmmenting out this method as its not using the new APIs + /* protected void visit(LOGenerate g) throws VisitorException { // Visit each of generates projection elements. IteratorExpressionOperator i = g.getProjections().iterator(); @@ -838,6 +840,7 @@ i.next().visit(this); } } + */ /*** * This does:- Modified: incubator/pig/branches/types/src/org/apache/pig/impl/plan/OperatorPlan.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/plan/OperatorPlan.java?rev=654629r1=654628r2=654629view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/plan/OperatorPlan.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/plan/OperatorPlan.java Thu May 8 14:25:22 2008 @@ -26,6 +26,9 @@ import java.util.Map; import org.apache.pig.impl.logicalLayer.OperatorKey; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + //import org.apache.commons.collections.map.MultiValueMap; @@ -40,7 +43,7 @@ private ListE mRoots; private ListE mLeaves; - +private Log log = LogFactory.getLog(OperatorPlan.class); public OperatorPlan() { mRoots = new ArrayListE(); @@ -100,6 +103,25 @@ } /** + * Get the map of operator key and associated operators + * @return map of operator key and operators. + */ +/* +public MapOperatorKey, E getKeys() { +return mKeys; +} +*/ +/** + * Get the map of operators and associated operator keys + * @return map of operator and operator keys. + */ +/* +public MapE, OperatorKey getOps() { +return mOps; +} +*/ + +/** * Insert an operator into the plan. This only inserts it as a node in * the graph, it does not connect it to any other operators. That should * be done as a separate step using connect. @@ -232,6 +254,8 @@ private
svn commit: r654172 - in /incubator/pig/trunk: ./ src/org/apache/pig/backend/hadoop/executionengine/ src/org/apache/pig/backend/local/executionengine/ src/org/apache/pig/impl/builtin/ src/org/apache/p
Author: gates Date: Wed May 7 10:12:11 2008 New Revision: 654172 URL: http://svn.apache.org/viewvc?rev=654172view=rev Log: PIG-202: Fix Order by so that user provided comparator func is used for quantile determination. Added: incubator/pig/trunk/src/org/apache/pig/backend/local/executionengine/InstantiateFuncCallerPOVisitor.java Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java incubator/pig/trunk/src/org/apache/pig/backend/local/executionengine/LocalExecutionEngine.java incubator/pig/trunk/src/org/apache/pig/impl/builtin/FindQuantiles.java incubator/pig/trunk/src/org/apache/pig/impl/eval/BinCondSpec.java incubator/pig/trunk/src/org/apache/pig/impl/eval/CompositeEvalSpec.java incubator/pig/trunk/src/org/apache/pig/impl/eval/EvalSpec.java incubator/pig/trunk/src/org/apache/pig/impl/eval/FilterSpec.java incubator/pig/trunk/src/org/apache/pig/impl/eval/FuncEvalSpec.java incubator/pig/trunk/src/org/apache/pig/impl/eval/GenerateSpec.java incubator/pig/trunk/src/org/apache/pig/impl/eval/SortDistinctSpec.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=654172r1=654171r2=654172view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Wed May 7 10:12:11 2008 @@ -276,3 +276,5 @@ PIG-229: Proper error handling in case of deserializer failure PIG-219: Change unit tests to run both local and map reduce modes (kali via gates). + + PIG-202: Fix Order by so that user provided comparator func is used for quantile determination (kali via gates). Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java?rev=654172r1=654171r2=654172view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java Wed May 7 10:12:11 2008 @@ -353,7 +353,9 @@ //sort the first column of the cogroup output and feed it to the quantiles function EvalSpec sortedSampleSpec = new ProjectSpec(1); -sortedSampleSpec = sortedSampleSpec.addSpec(new SortDistinctSpec(false, new StarSpec())); +EvalSpec starSpec = new StarSpec(); +starSpec.setComparatorName(loSort.getSortSpec().getComparatorName()); +sortedSampleSpec = sortedSampleSpec.addSpec(new SortDistinctSpec(false, starSpec)); argsList.add(sortedSampleSpec); EvalSpec args = new GenerateSpec(argsList); Added: incubator/pig/trunk/src/org/apache/pig/backend/local/executionengine/InstantiateFuncCallerPOVisitor.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/local/executionengine/InstantiateFuncCallerPOVisitor.java?rev=654172view=auto == --- incubator/pig/trunk/src/org/apache/pig/backend/local/executionengine/InstantiateFuncCallerPOVisitor.java (added) +++ incubator/pig/trunk/src/org/apache/pig/backend/local/executionengine/InstantiateFuncCallerPOVisitor.java Wed May 7 10:12:11 2008 @@ -0,0 +1,62 @@ +/** + * + */ +package org.apache.pig.backend.local.executionengine; + +import java.io.IOException; +import java.util.Map; + +import org.apache.pig.backend.executionengine.ExecPhysicalOperator; +import org.apache.pig.backend.hadoop.executionengine.POMapreduce; +import org.apache.pig.impl.FunctionInstantiator; +import org.apache.pig.impl.eval.EvalSpec; +import org.apache.pig.impl.logicalLayer.OperatorKey; +import org.apache.pig.impl.physicalLayer.POVisitor; + +/** + * POVisitor calling instantiateFunc on all EvalSpec members of visited nodes. + */ +public class InstantiateFuncCallerPOVisitor extends POVisitor { + +private FunctionInstantiator instantiator; + +protected InstantiateFuncCallerPOVisitor(FunctionInstantiator instantiator, +MapOperatorKey, ExecPhysicalOperator opTable) { +super(opTable); +this.instantiator = instantiator; +} + +private void callInstantiateFunc(EvalSpec spec) { +try { +spec.instantiateFunc(instantiator); +} catch (IOException e) { +throw new RuntimeException(e); +} +} + +@Override +public void visitCogroup(POCogroup g) { +super.visitCogroup(g); +for (EvalSpec es : g.specs) +callInstantiateFunc(es); +} + +@Override +public void visitMapreduce(POMapreduce mr
svn commit: r654319 - /incubator/pig/trunk/test/org/apache/pig/test/TestBinaryStorage.java
Author: gates Date: Wed May 7 16:18:06 2008 New Revision: 654319 URL: http://svn.apache.org/viewvc?rev=654319view=rev Log: Changed TestBinaryStorage to create it's test file in the local directory instead of /pig. This fixes the test in local mode on linux machines. Modified: incubator/pig/trunk/test/org/apache/pig/test/TestBinaryStorage.java Modified: incubator/pig/trunk/test/org/apache/pig/test/TestBinaryStorage.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestBinaryStorage.java?rev=654319r1=654318r2=654319view=diff == --- incubator/pig/trunk/test/org/apache/pig/test/TestBinaryStorage.java (original) +++ incubator/pig/trunk/test/org/apache/pig/test/TestBinaryStorage.java Wed May 7 16:18:06 2008 @@ -83,7 +83,7 @@ pigServer.registerQuery(OP = stream IP through CMD;); // Save the output using BinaryStorage -String output = /pig/out; +String output = ./pig.BinaryStorage.out; pigServer.store(OP, output, BinaryStorage()); // Get output data
svn commit: r654320 - in /incubator/pig/branches/types: build.xml src/org/apache/pig/data/DataType.java src/org/apache/pig/impl/logicalLayer/schema/Schema.java test/org/apache/pig/test/TestSchema.java
Author: gates Date: Wed May 7 16:18:45 2008 New Revision: 654320 URL: http://svn.apache.org/viewvc?rev=654320view=rev Log: Added equals and merge to schema class. Work done by Pi. Added: incubator/pig/branches/types/test/org/apache/pig/test/TestSchema.java Modified: incubator/pig/branches/types/build.xml incubator/pig/branches/types/src/org/apache/pig/data/DataType.java incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java Modified: incubator/pig/branches/types/build.xml URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/build.xml?rev=654320r1=654319r2=654320view=diff == --- incubator/pig/branches/types/build.xml (original) +++ incubator/pig/branches/types/build.xml Wed May 7 16:18:45 2008 @@ -144,6 +144,7 @@ **/test/FakeFSOutputStream.java, **/test/TestPackage.java, **/test/TestForEach.java, **/test/TestLocalRearrange.java, **/test/TestPOUserFunc.java, **/test/TestPODistinct.java, **/test/TestPOSort.java, + **/test/TestSchema.java, **/test/FakeFSInputStream.java, **/test/Util.java, **/logicalLayer/*.java, **/logicalLayer/parser/NodeIdGenerator.java, **/logicalLayer/schema/*.java, **/physicalLayer/topLevelOperators/*.java, @@ -263,6 +264,7 @@ include name=**/TestForEach.java / include name=**/TestInputOutputFileValidator.java / include name=**/TestTypeCheckingValidator.java / + include name=**/TestSchema.java / !-- include name=**/*Test*.java / exclude name=**/TestLargeFile.java / Modified: incubator/pig/branches/types/src/org/apache/pig/data/DataType.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/data/DataType.java?rev=654320r1=654319r2=654320view=diff == --- incubator/pig/branches/types/src/org/apache/pig/data/DataType.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/data/DataType.java Wed May 7 16:18:45 2008 @@ -572,4 +572,5 @@ default :return true ; } } + } Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java?rev=654320r1=654319r2=654320view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java Wed May 7 16:18:45 2008 @@ -78,6 +78,44 @@ return true; } + +/*** + * Compare two field schema for equality + * @param fschema + * @param fother + * @param relaxInner If true, we don't check inner tuple schemas + * @param relaxAlias If true, we don't check aliases + * @return + */ +public static boolean equals(FieldSchema fschema, + FieldSchema fother, + boolean relaxInner, + boolean relaxAlias) { +if (fschema == null) { +return false ; +} + +if (fother == null) { +return false ; +} + +if (fschema.type != fother.type) { +return false ; +} + +if ( (!relaxAlias) (fschema.alias != fother.alias) ) { +return false ; +} + +if ( (!relaxInner) (fschema.type == DataType.TUPLE) ) { + // compare recursively using schema + if (!Schema.equals(fschema.schema, fother.schema, false, relaxAlias)) { + return false ; + } +} + +return true ; +} } private ListFieldSchema mFields; @@ -181,6 +219,196 @@ } return true; } + +/** + * Recursively compare two schemas for equality + * @param schema + * @param other + * @param relaxInner if true, inner schemas will not be checked + * @return + */ +public static boolean equals(Schema schema, + Schema other, + boolean relaxInner, + boolean relaxAlias) { +if (schema == null) { +return false ; +} + +if (other == null) { +return false
svn commit: r652896 [2/2] - in /incubator/pig/branches/types: ./ src/org/apache/pig/ src/org/apache/pig/data/ src/org/apache/pig/impl/ src/org/apache/pig/impl/io/ src/org/apache/pig/impl/logicalLayer/
Added: incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidator.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidator.java?rev=652896view=auto == --- incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidator.java (added) +++ incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidator.java Fri May 2 13:29:54 2008 @@ -0,0 +1,427 @@ +package org.apache.pig.test; + +import java.util.Iterator; + +import junit.framework.TestCase; + +import org.apache.pig.impl.logicalLayer.validators.*; +import org.apache.pig.impl.logicalLayer.* ; +import org.apache.pig.impl.plan.PlanValidationException; +import org.apache.pig.impl.plan.CompilationMessageCollector; +import org.apache.pig.impl.plan.CompilationMessageCollector.Message; +import org.apache.pig.data.*; +import org.apache.pig.impl.logicalLayer.parser.NodeIdGenerator; +import org.junit.Test; + +public class TestTypeCheckingValidator extends TestCase { + + +@Test +public void testExpressionTypeChecking1() throws Throwable { +LogicalPlan plan = new LogicalPlan() ; +LOConst constant1 = new LOConst(plan, genNewOperatorKeyId(), 1, 10) ; +constant1.setType(DataType.INTEGER) ; +LOConst constant2 = new LOConst(plan, genNewOperatorKeyId(), 1, 20D) ; +constant2.setType(DataType.DOUBLE) ; +LOConst constant3 = new LOConst(plan, genNewOperatorKeyId(), 1, 123) ; +constant3.setType(DataType.CHARARRAY) ; + +LOAdd add1 = new LOAdd(plan, genNewOperatorKeyId(),1 , constant1, constant2) ; +LOCast cast1 = new LOCast(plan, genNewOperatorKeyId(), 1, constant3, DataType.BYTEARRAY) ; +LOMultiply mul1 = new LOMultiply(plan, genNewOperatorKeyId(), 1, add1, cast1) ; + +plan.add(constant1) ; +plan.add(constant2) ; +plan.add(constant3) ; +plan.add(cast1) ; +plan.add(add1) ; +plan.add(mul1) ; + +plan.connect(constant1, add1) ; +plan.connect(constant2, add1) ; +plan.connect(add1, mul1) ; +plan.connect(constant3, cast1) ; +plan.connect(cast1, mul1) ; + +CompilationMessageCollector collector = new CompilationMessageCollector() ; +TypeCheckingValidator typeValidator = new TypeCheckingValidator() ; +typeValidator.validate(plan, collector) ; +printMessageCollector(collector) ; +printTypeGraph(plan) ; + +if (collector.hasError()) { +throw new Exception(Error during type checking) ; +} + +// Induction check +assertEquals(DataType.DOUBLE, add1.getType()) ; +assertEquals(DataType.DOUBLE, mul1.getType()) ; + +// Cast insertion check +assertEquals(DataType.DOUBLE, add1.getLhsOperand().getType()) ; +assertEquals(DataType.DOUBLE, mul1.getRhsOperand().getType()) ; + +} + +@Test +public void testExpressionTypeChecking2() throws Throwable { +LogicalPlan plan = new LogicalPlan() ; +LOConst constant1 = new LOConst(plan, genNewOperatorKeyId(), 1, 10) ; +constant1.setType(DataType.INTEGER) ; +LOConst constant2 = new LOConst(plan, genNewOperatorKeyId(), 1, 20D) ; +constant2.setType(DataType.BYTEARRAY) ; +LOConst constant3 = new LOConst(plan, genNewOperatorKeyId(), 1, 123L) ; +constant3.setType(DataType.LONG) ; +LOConst constant4 = new LOConst(plan, genNewOperatorKeyId(), 1, true) ; +constant4.setType(DataType.BOOLEAN) ; + +LOSubtract sub1 = new LOSubtract(plan, genNewOperatorKeyId(), 1, constant1, constant2) ; +LOGreaterThan gt1 = new LOGreaterThan(plan, genNewOperatorKeyId(), 1, sub1, constant3) ; +LOAnd and1 = new LOAnd(plan, genNewOperatorKeyId(), 1, gt1, constant4) ; +LONot not1 = new LONot(plan, genNewOperatorKeyId(), 1, and1) ; + +plan.add(constant1) ; +plan.add(constant2) ; +plan.add(constant3) ; +plan.add(constant4) ; + +plan.add(sub1) ; +plan.add(gt1) ; +plan.add(and1) ; +plan.add(not1) ; + + +plan.connect(constant1, sub1) ; +plan.connect(constant2, sub1) ; +plan.connect(sub1, gt1) ; +plan.connect(constant3, gt1) ; +plan.connect(gt1, and1) ; +plan.connect(constant4, and1) ; +plan.connect(and1, not1) ; + +CompilationMessageCollector collector = new CompilationMessageCollector() ; +TypeCheckingValidator typeValidator = new TypeCheckingValidator() ; +typeValidator.validate(plan, collector) ; +printMessageCollector(collector) ; +
svn commit: r652906 - in /incubator/pig/trunk: ./ conf/ src/org/apache/pig/ src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/ src/org/apache/pig/impl/util/
Author: gates Date: Fri May 2 13:57:56 2008 New Revision: 652906 URL: http://svn.apache.org/viewvc?rev=652906view=rev Log: PIG-176: Change bag spilling so that bags below a certain threshold are not spilled, thus avoiding proliferation of small files. Added: incubator/pig/trunk/src/org/apache/pig/impl/util/ConfigurationValidator.java Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/conf/pig.properties incubator/pig/trunk/src/org/apache/pig/PigServer.java incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/MapReduceLauncher.java incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigMapReduce.java incubator/pig/trunk/src/org/apache/pig/impl/util/PropertiesUtil.java incubator/pig/trunk/src/org/apache/pig/impl/util/SpillableMemoryManager.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=652906r1=652905r2=652906view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Fri May 2 13:57:56 2008 @@ -264,3 +264,6 @@ PIG-224: fix to error handling code to produce correct error code + PIG-176: Change bag spilling so that bags below a certain threshold are + not spilled, thus avoiding proliferation of small files (pi_song via + gates). Modified: incubator/pig/trunk/conf/pig.properties URL: http://svn.apache.org/viewvc/incubator/pig/trunk/conf/pig.properties?rev=652906r1=652905r2=652906view=diff == --- incubator/pig/trunk/conf/pig.properties (original) +++ incubator/pig/trunk/conf/pig.properties Fri May 2 13:57:56 2008 @@ -32,3 +32,11 @@ #hod.config.dir #hod.param + +#Do not spill temp files smaller than this size (bytes) +#pig.spill.size.threshold=10 + +#EXPERIMENT: Activate garbage collection when spilling a file bigger than this size (bytes) +#This should help reduce the number of files being spilled. +#pig.spill.gc.activation.size=1 + Modified: incubator/pig/trunk/src/org/apache/pig/PigServer.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/PigServer.java?rev=652906r1=652905r2=652906view=diff == --- incubator/pig/trunk/src/org/apache/pig/PigServer.java (original) +++ incubator/pig/trunk/src/org/apache/pig/PigServer.java Fri May 2 13:57:56 2008 @@ -133,10 +133,6 @@ this(execType, PropertiesUtil.loadPropertiesFromFile()); } -public PigServer() throws ExecException { -this(ExecType.MAPREDUCE, new Properties()); -} - public PigServer(ExecType execType, Properties properties) throws ExecException { this.pigContext = new PigContext(execType, properties); if (this.pigContext.getProperties().getProperty(PigContext.JOB_NAME) == null) { Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/MapReduceLauncher.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/MapReduceLauncher.java?rev=652906r1=652905r2=652906view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/MapReduceLauncher.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/MapReduceLauncher.java Fri May 2 13:57:56 2008 @@ -60,6 +60,7 @@ import org.apache.pig.impl.util.JarManager; import org.apache.pig.impl.util.ObjectSerializer; import org.apache.pig.impl.util.WrappedIOException; +import org.apache.pig.impl.util.ConfigurationValidator; /** @@ -133,6 +134,7 @@ JobConf conf = new JobConf(config); setJobProperties(conf, pom); Properties properties = pom.pigContext.getProperties(); +ConfigurationValidator.validatePigProperties(properties) ; String jobName = properties.getProperty(PigContext.JOB_NAME); conf.setJobName(jobName); boolean success = false; @@ -160,6 +162,11 @@ String user = System.getProperty(user.name); conf.setUser(user != null ? user : Pigster); +conf.set(pig.spill.size.threshold, + properties.getProperty(pig.spill.size.threshold)) ; +conf.set(pig.spill.gc.activation.size, +properties.getProperty(pig.spill.gc.activation.size)) ; + if (pom.reduceParallelism != -1) { conf.setNumReduceTasks(pom.reduceParallelism); } Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigMapReduce.java URL: http
svn commit: r652935 - in /incubator/pig/trunk: ./ conf/ src/org/apache/pig/ src/org/apache/pig/backend/hadoop/executionengine/ src/org/apache/pig/impl/ src/org/apache/pig/impl/util/
Author: gates Date: Fri May 2 15:41:58 2008 New Revision: 652935 URL: http://svn.apache.org/viewvc?rev=652935view=rev Log: PIG-215: Cleanup a few dangling ends left by PIG-111. Added: incubator/pig/trunk/conf/log4j.properties Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/conf/pig.properties incubator/pig/trunk/src/org/apache/pig/Main.java incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java incubator/pig/trunk/src/org/apache/pig/impl/PigContext.java incubator/pig/trunk/src/org/apache/pig/impl/util/PropertiesUtil.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=652935r1=652934r2=652935view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Fri May 2 15:41:58 2008 @@ -267,6 +267,9 @@ PIG-176: Change bag spilling so that bags below a certain threshold are not spilled, thus avoiding proliferation of small files (pi_song via gates). - + PIG-227: making load/store function optional in stream input/output spec (acmurthy via olgan) + +PIG-215: Cleanup a few dangling ends left by PIG-111 (pi_song via gates). + Added: incubator/pig/trunk/conf/log4j.properties URL: http://svn.apache.org/viewvc/incubator/pig/trunk/conf/log4j.properties?rev=652935view=auto == --- incubator/pig/trunk/conf/log4j.properties (added) +++ incubator/pig/trunk/conf/log4j.properties Fri May 2 15:41:58 2008 @@ -0,0 +1,8 @@ +# * Set root logger level to DEBUG and its only appender to A. +log4j.rootLogger=info, A + +# * A is set to be a ConsoleAppender. +log4j.appender.A=org.apache.log4j.ConsoleAppender +# * A uses PatternLayout. +log4j.appender.A.layout=org.apache.log4j.PatternLayout +log4j.appender.A.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n Modified: incubator/pig/trunk/conf/pig.properties URL: http://svn.apache.org/viewvc/incubator/pig/trunk/conf/pig.properties?rev=652935r1=652934r2=652935view=diff == --- incubator/pig/trunk/conf/pig.properties (original) +++ incubator/pig/trunk/conf/pig.properties Fri May 2 15:41:58 2008 @@ -23,7 +23,7 @@ verbose=false #exectype local|mapreduce, mapreduce is default -exectype=local +#exectype=mapreduce # hod realted properties #ssh.gateway #hod.expect.root Modified: incubator/pig/trunk/src/org/apache/pig/Main.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/Main.java?rev=652935r1=652934r2=652935view=diff == --- incubator/pig/trunk/src/org/apache/pig/Main.java (original) +++ incubator/pig/trunk/src/org/apache/pig/Main.java Fri May 2 15:41:58 2008 @@ -99,7 +99,7 @@ ExecMode mode = ExecMode.UNKNOWN; String file = null; -ExecType execType = ExecType.LOCAL; +ExecType execType = ExecType.MAPREDUCE ; String execTypeString = properties.getProperty(exectype); if(execTypeString!=null execTypeString.length()0){ execType = PigServer.parseExecType(execTypeString); Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java?rev=652935r1=652934r2=652935view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java Fri May 2 15:41:58 2008 @@ -135,7 +135,7 @@ } } -log.info(Connecting to hadoop file system at: + properties.get(FILE_SYSTEM_LOCATION)); +log.info(Connecting to hadoop file system at: + (nameNode==null? LOCAL: nameNode) ) ; ds = new HDataStorage(properties); Configuration configuration = ConfigurationUtil.toConfiguration(properties); Modified: incubator/pig/trunk/src/org/apache/pig/impl/PigContext.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/impl/PigContext.java?rev=652935r1=652934r2=652935view=diff == --- incubator/pig/trunk/src/org/apache/pig/impl/PigContext.java (original) +++ incubator/pig/trunk/src/org/apache/pig/impl/PigContext.java Fri May 2 15:41:58 2008 @@ -128,9 +128,7 @@ public PigContext(ExecType execType, Properties properties){ this.execType = execType; -this.properties = properties
svn commit: r650606 - in /incubator/pig/trunk: CHANGES.txt src/org/apache/pig/impl/streaming/ExecutableManager.java test/org/apache/pig/test/TestStreaming.java
Author: gates Date: Tue Apr 22 11:41:49 2008 New Revision: 650606 URL: http://svn.apache.org/viewvc?rev=650606view=rev Log: PIG-216 Fix streaming to work with commands that use unix pipes. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/impl/streaming/ExecutableManager.java incubator/pig/trunk/test/org/apache/pig/test/TestStreaming.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=650606r1=650605r2=650606view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Tue Apr 22 11:41:49 2008 @@ -243,3 +243,6 @@ PIG-213: Remove non-static references to logger from data bags and tuples, as it causes significant overhead (vgeschel via gates). + + PIG-216 Fix streaming to work with commands that use unix pipes (acmurthy + via gates). Modified: incubator/pig/trunk/src/org/apache/pig/impl/streaming/ExecutableManager.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/impl/streaming/ExecutableManager.java?rev=650606r1=650605r2=650606view=diff == --- incubator/pig/trunk/src/org/apache/pig/impl/streaming/ExecutableManager.java (original) +++ incubator/pig/trunk/src/org/apache/pig/impl/streaming/ExecutableManager.java Tue Apr 22 11:41:49 2008 @@ -236,18 +236,8 @@ cmdArgs.add(-c); StringBuffer sb = new StringBuffer(); sb.append(exec ); - for (int i=0; i argv.length; ++i) { - // Single-quote each component, however ensure that already - // quoted args are handled right - sb.append('\''); - - String arg = argv[i]; - if (arg.charAt(0) == '\'' arg.charAt(arg.length()-1) == '\'') { - arg = arg.substring(1, arg.length()-1); - } - sb.append(arg); - -sb.append('\''); + for (String arg : argv) { +sb.append(arg); sb.append( ); } cmdArgs.add(sb.toString()); Modified: incubator/pig/trunk/test/org/apache/pig/test/TestStreaming.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestStreaming.java?rev=650606r1=650605r2=650606view=diff == --- incubator/pig/trunk/test/org/apache/pig/test/TestStreaming.java (original) +++ incubator/pig/trunk/test/org/apache/pig/test/TestStreaming.java Tue Apr 22 11:41:49 2008 @@ -432,4 +432,42 @@ // Cleanup pigServer.deleteFile(output); } + +@Test +public void testLocalSimpleMapSideStreamingWithUnixPipes() +throws Exception { +testSimpleMapSideStreamingWithUnixPipes(ExecType.LOCAL); +} + +@Test +public void testMRSimpleMapSideStreamingWithUnixPipes() throws Exception { +testSimpleMapSideStreamingWithUnixPipes(ExecType.MAPREDUCE); +} + +private void testSimpleMapSideStreamingWithUnixPipes(ExecType execType) +throws Exception { +PigServer pigServer = createPigServer(execType); +File input = Util.createInputFile(tmp, , + new String[] {A,1, B,2, C,3, D,2, +A,5, B,5, C,8, A,8, +D,8, A,9}); + +// Expected results +String[] expectedFirstFields = +new String[] {A, B, C, D, A, B, C, A, D, A}; +int[] expectedSecondFields = new int[] {1, 2, 3, 2, 5, 5, 8, 8, 8, 9}; +Tuple[] expectedResults = +setupExpectedResults(expectedFirstFields, expectedSecondFields); + +// Pig query to run +pigServer.registerQuery(define CMD ` + simpleEchoStreamingCommand + + | + simpleEchoStreamingCommand + `;); +pigServer.registerQuery(IP = load 'file: + input + ' using + +PigStorage.class.getName() + (',');); +pigServer.registerQuery(OP = stream IP through CMD;); + +// Run the query and check the results +Util.checkQueryOutputs(pigServer.openIterator(OP), expectedResults); +} + }
svn commit: r650688 - in /incubator/pig/trunk: CHANGES.txt src/org/apache/pig/pen/DisplayExamples.java src/org/apache/pig/pen/ExGen.java test/org/apache/pig/test/TestExGenCogroup.java test/org/apache/
Author: gates Date: Tue Apr 22 15:58:28 2008 New Revision: 650688 URL: http://svn.apache.org/viewvc?rev=650688view=rev Log: PIG-207 Fix illustrate command to work in mapreduce mode. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/pen/DisplayExamples.java incubator/pig/trunk/src/org/apache/pig/pen/ExGen.java incubator/pig/trunk/test/org/apache/pig/test/TestExGenCogroup.java incubator/pig/trunk/test/org/apache/pig/test/TestExGenEval.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=650688r1=650687r2=650688view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Tue Apr 22 15:58:28 2008 @@ -246,3 +246,7 @@ PIG-216 Fix streaming to work with commands that use unix pipes (acmurthy via gates). + + PIG-207 Fix illustrate command to work in mapreduce mode (shubhamc via + gates). + Modified: incubator/pig/trunk/src/org/apache/pig/pen/DisplayExamples.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/pen/DisplayExamples.java?rev=650688r1=650687r2=650688view=diff == --- incubator/pig/trunk/src/org/apache/pig/pen/DisplayExamples.java (original) +++ incubator/pig/trunk/src/org/apache/pig/pen/DisplayExamples.java Tue Apr 22 15:58:28 2008 @@ -38,7 +38,7 @@ public class DisplayExamples { public static StringBuffer Result = new StringBuffer(); - public static final int MAX_DATAATOM_LENGTH = 10; + public static final int MAX_DATAATOM_LENGTH = 15; public static String PrintTabular(LogicalPlan lp, MapLogicalOperator, DataBag exampleData) { StringBuffer output = new StringBuffer(); @@ -96,13 +96,13 @@ } //Display the schema first - output.append(AddSpaces(total + 3*cols + 9, false) + \n); + output.append(AddSpaces(total + 3*(cols +1) + aliasLength + 1, false) + \n); output.append(| + op.getAlias() + AddSpaces(4, true) + | ); for(int i = 0; i cols; ++i) { String field = fields.get(i).toString(); output.append(field + AddSpaces(maxColSizes[i] - field.length(), true) + | ); } - output.append(\n + AddSpaces(total + 3*cols + 9, false) + \n); + output.append(\n + AddSpaces(total + 3*(cols +1) + aliasLength + 1, false) + \n); //now start displaying the data for(int i = 0; i rows; ++i) { output.append(| + AddSpaces(aliasLength, true) + | ); @@ -113,7 +113,7 @@ output.append(\n); } //now display the finish line - output.append(AddSpaces(total + 3*cols + 9, false) + \n); + output.append(AddSpaces(total + 3*(cols +1) + aliasLength + 1, false) + \n); } static String[][] MakeArray(LogicalOperator op, DataBag bag) { @@ -144,7 +144,7 @@ static String ShortenField(DataAtom da) { int length = da.toString().length(); if(length MAX_DATAATOM_LENGTH) { - return new String(da.toString().substring(0, 3) + ... + da.toString().substring(length - 4, length - 1)); + return new String(da.toString().substring(0, 4) + ... + da.toString().substring(length - 4, length - 1)); } return da.toString(); } @@ -159,7 +159,7 @@ while(it.hasNext()) { Tuple t = it.next(); if(!it.hasNext()) { - str.append(, ... + ShortenField(t)); + str.append(, ..., + ShortenField(t)); } } } else { @@ -182,7 +182,7 @@ if(noFields 3) { Datum d = t.getField(0); - str.append(ShortenField(d) + , ...); + str.append(ShortenField(d) + , ..., ); d = t.getField(noFields - 1); str.append(ShortenField(d)); Modified: incubator/pig/trunk/src/org/apache/pig/pen/ExGen.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/pen/ExGen.java?rev=650688r1=650687r2=650688view=diff == --- incubator/pig/trunk/src/org/apache/pig/pen/ExGen.java (original) +++ incubator/pig/trunk/src/org/apache/pig/pen/ExGen.java Tue Apr 22 15:58:28 2008 @@ -24,6 +24,7 @@ import java.util.LinkedList; import java.util.Map; +import org.apache.pig.PigServer.ExecType; import org.apache.pig.backend.executionengine.ExecException; import org.apache.pig.backend.executionengine.ExecPhysicalOperator; import org.apache.pig.backend.executionengine.ExecPhysicalPlan; @@ -45,14 +46,16
svn commit: r649710 - in /incubator/pig/trunk: ./ src/org/apache/pig/data/
Author: gates Date: Fri Apr 18 15:11:59 2008 New Revision: 649710 URL: http://svn.apache.org/viewvc?rev=649710view=rev Log: PIG-213: Remove non-static references to logger from data bags and tuples, as it causes significant overhead. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/data/DataBag.java incubator/pig/trunk/src/org/apache/pig/data/DefaultDataBag.java incubator/pig/trunk/src/org/apache/pig/data/DistinctDataBag.java incubator/pig/trunk/src/org/apache/pig/data/SortedDataBag.java incubator/pig/trunk/src/org/apache/pig/data/TimestampedTuple.java incubator/pig/trunk/src/org/apache/pig/data/Tuple.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=649710r1=649709r2=649710view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Fri Apr 18 15:11:59 2008 @@ -240,3 +240,6 @@ PIG-114: store one alias/logicalPlan twice leads to instantiation of StoreFunc as LoadFunc (pi_song via gates). + + PIG-213: Remove non-static references to logger from data bags and tuples, + as it causes significant overhead (vgeschel via gates). Modified: incubator/pig/trunk/src/org/apache/pig/data/DataBag.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/data/DataBag.java?rev=649710r1=649709r2=649710view=diff == --- incubator/pig/trunk/src/org/apache/pig/data/DataBag.java (original) +++ incubator/pig/trunk/src/org/apache/pig/data/DataBag.java Fri Apr 18 15:11:59 2008 @@ -69,7 +69,7 @@ */ public abstract class DataBag extends Datum implements Spillable, IterableTuple { - private final Log log = LogFactory.getLog(getClass()); + private static final Log log = LogFactory.getLog(DataBag.class); // Container that holds the tuples. Actual object instantiated by // subclasses. Modified: incubator/pig/trunk/src/org/apache/pig/data/DefaultDataBag.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/data/DefaultDataBag.java?rev=649710r1=649709r2=649710view=diff == --- incubator/pig/trunk/src/org/apache/pig/data/DefaultDataBag.java (original) +++ incubator/pig/trunk/src/org/apache/pig/data/DefaultDataBag.java Fri Apr 18 15:11:59 2008 @@ -38,7 +38,7 @@ */ public class DefaultDataBag extends DataBag { -private final Log log = LogFactory.getLog(getClass()); +private static final Log log = LogFactory.getLog(DefaultDataBag.class); public DefaultDataBag() { mContents = new ArrayListTuple(); Modified: incubator/pig/trunk/src/org/apache/pig/data/DistinctDataBag.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/data/DistinctDataBag.java?rev=649710r1=649709r2=649710view=diff == --- incubator/pig/trunk/src/org/apache/pig/data/DistinctDataBag.java (original) +++ incubator/pig/trunk/src/org/apache/pig/data/DistinctDataBag.java Fri Apr 18 15:11:59 2008 @@ -51,7 +51,7 @@ */ public class DistinctDataBag extends DataBag { -private final Log log = LogFactory.getLog(getClass()); +private static final Log log = LogFactory.getLog(DistinctDataBag.class); public DistinctDataBag() { mContents = new HashSetTuple(); Modified: incubator/pig/trunk/src/org/apache/pig/data/SortedDataBag.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/data/SortedDataBag.java?rev=649710r1=649709r2=649710view=diff == --- incubator/pig/trunk/src/org/apache/pig/data/SortedDataBag.java (original) +++ incubator/pig/trunk/src/org/apache/pig/data/SortedDataBag.java Fri Apr 18 15:11:59 2008 @@ -51,7 +51,7 @@ */ public class SortedDataBag extends DataBag { -private final Log log = LogFactory.getLog(getClass()); +private static final Log log = LogFactory.getLog(SortedDataBag.class); private ComparatorTuple mComp; private boolean mReadStarted = false; Modified: incubator/pig/trunk/src/org/apache/pig/data/TimestampedTuple.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/data/TimestampedTuple.java?rev=649710r1=649709r2=649710view=diff == --- incubator/pig/trunk/src/org/apache/pig/data/TimestampedTuple.java (original) +++ incubator/pig/trunk/src/org/apache/pig/data/TimestampedTuple.java Fri Apr 18 15:11:59 2008 @@ -26,7 +26,7 @@ public class TimestampedTuple extends Tuple { -private final Log log = LogFactory.getLog(getClass()); +private static final Log log = LogFactory.getLog
svn commit: r649715 [2/2] - in /incubator/pig/branches/types: ./ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/physicalLayer/plans/ src/org/apache/pig/impl/physicalLayer/topLevelOperat
Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/NotEqualToExpr.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/NotEqualToExpr.java?rev=649715r1=649714r2=649715view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/NotEqualToExpr.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/NotEqualToExpr.java Fri Apr 18 15:32:08 2008 @@ -26,11 +26,11 @@ import org.apache.pig.data.DataType; import org.apache.pig.data.Tuple; import org.apache.pig.impl.logicalLayer.OperatorKey; -import org.apache.pig.impl.logicalLayer.parser.ParseException; import org.apache.pig.backend.executionengine.ExecException; import org.apache.pig.impl.physicalLayer.POStatus; import org.apache.pig.impl.physicalLayer.Result; import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor; +import org.apache.pig.impl.plan.VisitorException; public class NotEqualToExpr extends ComparisonOperator { @@ -45,7 +45,7 @@ } @Override -public void visit(ExprPlanVisitor v) throws ParseException { +public void visit(ExprPlanVisitor v) throws VisitorException { v.visitNotEqualTo(this); } Added: incubator/pig/branches/types/src/org/apache/pig/impl/plan/DependencyOrderWalker.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/plan/DependencyOrderWalker.java?rev=649715view=auto == --- incubator/pig/branches/types/src/org/apache/pig/impl/plan/DependencyOrderWalker.java (added) +++ incubator/pig/branches/types/src/org/apache/pig/impl/plan/DependencyOrderWalker.java Fri Apr 18 15:32:08 2008 @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.pig.impl.plan; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + + +/** + * DependencyOrderWalker traverses the graph in such a way that no node is visited + * before all the nodes it depends on have been visited. Beyond this, it does not + * guarantee any particular order. So, you have a graph with node 1 2 3 4, and + * edges 1-3, 2-3, and 3-4, this walker guarnatees that 1 and 2 will be visited + * before 3 and 3 before 4, but it does not guarantee whether 1 or 2 will be + * visited first. + */ +public class DependencyOrderWalker O extends Operator, P extends OperatorPlanO +extends PlanWalkerO, P { + +/** + * @param plan Plan for this walker to traverse. + */ +public DependencyOrderWalker(P plan) { +super(plan); +} + +/** + * Begin traversing the graph. + * @param visitor Visitor this walker is being used by. + * @throws VisitorException if an error is encountered while walking. + */ +public void walk(PlanVisitorO, P visitor) throws VisitorException { +// This is highly inefficient, but our graphs are small so it should be okay. +// The algorithm works by starting at any node in the graph, finding it's +// predecessors and calling itself for each of those predecessors. When it +// finds a node that has no unfinished predecessors it puts that node in the +// list. It then unwinds itself putting each of the other nodes in the list. +// It keeps track of what nodes it's seen as it goes so it doesn't put any +// nodes in the graph twice. + +ListO fifo = new ArrayListO(); +SetO seen = new HashSetO(); +ListO leaves = mPlan.getLeaves(); +if (leaves == null) return; +for (O op : leaves) { +doAllPredecessors(op, seen, fifo); +} + +for (O op: fifo) { +op.visit(visitor); +} +} + +public
svn commit: r649154 - in /incubator/pig/trunk: ./ src/org/apache/pig/ src/org/apache/pig/backend/hadoop/executionengine/ src/org/apache/pig/backend/local/executionengine/ src/org/apache/pig/builtin/ t
Author: gates Date: Thu Apr 17 08:47:08 2008 New Revision: 649154 URL: http://svn.apache.org/viewvc?rev=649154view=rev Log: PIG-114: store one alias/logicalPlan twice leads to instantiation of StoreFunc as LoadFunc. Added: incubator/pig/trunk/src/org/apache/pig/ReversibleLoadStoreFunc.java incubator/pig/trunk/test/org/apache/pig/test/TestReversibleLoadStore.java Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java incubator/pig/trunk/src/org/apache/pig/backend/local/executionengine/LocalExecutionEngine.java incubator/pig/trunk/src/org/apache/pig/builtin/BinStorage.java incubator/pig/trunk/src/org/apache/pig/builtin/PigStorage.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=649154r1=649153r2=649154view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Thu Apr 17 08:47:08 2008 @@ -237,3 +237,6 @@ PIG-183: Catch when a UDF has been compiled with the wrong version of java and give a RuntimeException (pi_song via gates). + + PIG-114: store one alias/logicalPlan twice leads to instantiation of + StoreFunc as LoadFunc (pi_song via gates). Added: incubator/pig/trunk/src/org/apache/pig/ReversibleLoadStoreFunc.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/ReversibleLoadStoreFunc.java?rev=649154view=auto == --- incubator/pig/trunk/src/org/apache/pig/ReversibleLoadStoreFunc.java (added) +++ incubator/pig/trunk/src/org/apache/pig/ReversibleLoadStoreFunc.java Thu Apr 17 08:47:08 2008 @@ -0,0 +1,14 @@ +package org.apache.pig; + +/** + * This interface is used to implement classes that can perform both + * Load and Store functionalities in a symmetric fashion (thus reversible). + * + * The symmetry property of implementations is used in the optimization + * engine therefore violation of this property while implementing this + * interface is likely to result in unexpected output from executions. + * + */ +public interface ReversibleLoadStoreFunc extends LoadFunc, StoreFunc { + +} Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java?rev=649154r1=649153r2=649154view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java Thu Apr 17 08:47:08 2008 @@ -22,6 +22,7 @@ import java.util.Map; import java.util.Iterator; +import org.apache.pig.ReversibleLoadStoreFunc; import org.apache.pig.builtin.BinStorage; import org.apache.pig.impl.PigContext; import org.apache.pig.impl.FunctionInstantiator; @@ -83,16 +84,18 @@ MapRedResult materializedResult = materializedResults.get(logicalKey); if (materializedResult != null) { -POMapreduce pom = new POMapreduce(logicalKey.getScope(), - nodeIdGenerator.getNextNodeId(logicalKey.getScope()), - execEngine.getPhysicalOpTable(), - logicalKey, - pigContext); +if (PigContext.instantiateFuncFromSpec(materializedResult.outFileSpec.getFuncSpec()) +instanceof ReversibleLoadStoreFunc) { +POMapreduce pom = new POMapreduce(logicalKey.getScope(), +nodeIdGenerator.getNextNodeId(logicalKey.getScope()), +execEngine.getPhysicalOpTable(), logicalKey, +pigContext); -pom.addInputFile(materializedResult.outFileSpec); -pom.mapParallelism = Math.max(pom.mapParallelism, materializedResult.parallelismRequest); + pom.addInputFile(materializedResult.outFileSpec); + pom.mapParallelism = Math.max(pom.mapParallelism, materializedResult.parallelismRequest); -return pom.getOperatorKey(); + return pom.getOperatorKey(); + } } // first, compile inputs into MapReduce operators Modified: incubator/pig/trunk/src/org/apache/pig/backend/local/executionengine/LocalExecutionEngine.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/local/executionengine/LocalExecutionEngine.java?rev=649154r1=649153r2=649154view=diff
svn commit: r648768 - in /incubator/pig/trunk: CHANGES.txt build.xml docs/ docs/overview.html
Author: gates Date: Wed Apr 16 10:46:15 2008 New Revision: 648768 URL: http://svn.apache.org/viewvc?rev=648768view=rev Log: PIG-149, PIG-150: Fix doc target so that ant can generate docs. Added: incubator/pig/trunk/docs/ incubator/pig/trunk/docs/overview.html Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/build.xml Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=648768r1=648767r2=648768view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Wed Apr 16 10:46:15 2008 @@ -231,3 +231,6 @@ PIG-188: Fix mismatches between pig slicer changes and new streaming feature (acmurthy via gates). + + PIG-149, PIG-150: Fix doc target so that ant can generate docs (xuzh via + gates). Modified: incubator/pig/trunk/build.xml URL: http://svn.apache.org/viewvc/incubator/pig/trunk/build.xml?rev=648768r1=648767r2=648768view=diff == --- incubator/pig/trunk/build.xml (original) +++ incubator/pig/trunk/build.xml Wed Apr 16 10:46:15 2008 @@ -16,6 +16,7 @@ property name=src.dir value=${basedir}/src/ / property name=src.lib.dir value=${basedir}/lib-src/ / property name=src.gen.dir value=${basedir}/src-gen/ / +property name=src.docs.dir value=${basedir}/docs/ / !-- javacc properties -- property name=src.gen.query.parser.dir value=${src.gen.dir}/org/apache/pig/impl/logicalLayer/parser / @@ -64,6 +65,9 @@ property name=test.log.dir value=${basedir}/test/logs/ property name=junit.hadoop.conf value=${user.home}/pigtest/conf// property name=test.output value=no/ + + !-- javadoc properties -- + property name=javadoc.link.java value=http://java.sun.com/j2se/1.5.0/docs/api/; / !-- == -- !-- Stuff needed by all targets-- @@ -166,14 +170,15 @@ !-- == -- !-- Documentation -- !-- == -- -target name=javadoc depends=init description=Create documentation +target name=javadoc depends=jar description=Create documentation mkdir dir=${build.javadoc} / -javadoc overview=${src.dir}/overview.html packagenames=org.apache.pig.* destdir=${build.javadoc} author=true version=true use=true windowtitle=${Name} ${version} API doctitle=${Name} ${version} API bottom=Copyright amp;copy; ${year} The Apache Software Foundation +javadoc overview=${src.docs.dir}/overview.html packagenames=org.apache.pig.* destdir=${build.javadoc} author=true version=true use=true windowtitle=${Name} ${version} API doctitle=${Name} ${version} API bottom=Copyright amp;copy; ${year} The Apache Software Foundation packageset dir=${src.dir} / link href=${javadoc.link.java} / classpath path refid=classpath / pathelement path=${java.class.path} / + pathelement path=${output.jarfile} / /classpath group title=pig packages=org.apache.* / /javadoc Added: incubator/pig/trunk/docs/overview.html URL: http://svn.apache.org/viewvc/incubator/pig/trunk/docs/overview.html?rev=648768view=auto == --- incubator/pig/trunk/docs/overview.html (added) +++ incubator/pig/trunk/docs/overview.html Wed Apr 16 10:46:15 2008 @@ -0,0 +1,13 @@ +HTML +BODY +Provides the classes necessary to create programs in the high-level Pig Latin +language and the infrastructure for evaluating these programs. +p +Pig is a platform for analyzing large data sets and consists of 2 layers: +Pig's infrastructure layer consists of a compiler that produces sequences of +Map-Reduce programs, for which large-scale parallel implementations already +exist (e.g., the Hadoop project). Pig's language layer currently consists of +a textual language called Pig Latin for expressing data analysis programs. +/BODY +/HTML +
svn commit: r648789 - in /incubator/pig/trunk: CHANGES.txt src/org/apache/pig/impl/PigContext.java
Author: gates Date: Wed Apr 16 11:32:30 2008 New Revision: 648789 URL: http://svn.apache.org/viewvc?rev=648789view=rev Log: PIG-183: Catch when a UDF has been compiled with the wrong version of java and give a RuntimeException. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/impl/PigContext.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=648789r1=648788r2=648789view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Wed Apr 16 11:32:30 2008 @@ -234,3 +234,6 @@ PIG-149, PIG-150: Fix doc target so that ant can generate docs (xuzh via gates). + + PIG-183: Catch when a UDF has been compiled with the wrong version of + java and give a RuntimeException (pi_song via gates). Modified: incubator/pig/trunk/src/org/apache/pig/impl/PigContext.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/impl/PigContext.java?rev=648789r1=648788r2=648789view=diff == --- incubator/pig/trunk/src/org/apache/pig/impl/PigContext.java (original) +++ incubator/pig/trunk/src/org/apache/pig/impl/PigContext.java Wed Apr 16 11:32:30 2008 @@ -444,11 +444,16 @@ public static Class resolveClassName(String name) throws IOException{ for(String prefix: packageImportList) { Class c; -try { -c = Class.forName(prefix+name,true, LogicalPlanBuilder.classloader); -return c; -} catch (ClassNotFoundException e) { -} catch (LinkageError e) {} +try { +c = Class.forName(prefix+name,true, LogicalPlanBuilder.classloader); +return c; +} +catch (ClassNotFoundException e) { +// do nothing +} +catch (UnsupportedClassVersionError e) { +throw new RuntimeException(e) ; +} } // create ClassNotFoundException exception and attach to IOException
svn commit: r647997 - in /incubator/pig/trunk: ./ src/org/apache/pig/ src/org/apache/pig/backend/executionengine/ src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/ src/org/apache/pig/ba
Author: gates Date: Mon Apr 14 14:04:05 2008 New Revision: 647997 URL: http://svn.apache.org/viewvc?rev=647997view=rev Log: PIG-188: Fix mismatches between pig slicer changes and new streaming feature. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/Slice.java incubator/pig/trunk/src/org/apache/pig/backend/executionengine/PigSlice.java incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/SliceWrapper.java incubator/pig/trunk/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java incubator/pig/trunk/test/org/apache/pig/test/RangeSlicer.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=647997r1=647996r2=647997view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Mon Apr 14 14:04:05 2008 @@ -228,3 +228,6 @@ 1k caused pig to freeze. (kali via gates) PIG-204: Repair broken input splits (acmurthy via gates). + + PIG-188: Fix mismatches between pig slicer changes and new streaming + feature (acmurthy via gates). Modified: incubator/pig/trunk/src/org/apache/pig/Slice.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/Slice.java?rev=647997r1=647996r2=647997view=diff == --- incubator/pig/trunk/src/org/apache/pig/Slice.java (original) +++ incubator/pig/trunk/src/org/apache/pig/Slice.java Mon Apr 14 14:04:05 2008 @@ -41,6 +41,11 @@ void init(DataStorage store) throws IOException; /** + * Returns the offset from which data in this Slice will be processed. + */ +long getStart(); + +/** * Returns the length in bytes of all of the data that will be processed by * this Slice. * p Modified: incubator/pig/trunk/src/org/apache/pig/backend/executionengine/PigSlice.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/executionengine/PigSlice.java?rev=647997r1=647996r2=647997view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/executionengine/PigSlice.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/executionengine/PigSlice.java Mon Apr 14 14:04:05 2008 @@ -48,6 +48,10 @@ return new String[] { file }; } +public long getStart() { +return start; +} + public long getLength() { return length; } Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/SliceWrapper.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/SliceWrapper.java?rev=647997r1=647996r2=647997view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/SliceWrapper.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/SliceWrapper.java Mon Apr 14 14:04:05 2008 @@ -92,6 +92,15 @@ DataStorage store = new HDataStorage(ConfigurationUtil.toProperties(job)); store.setActiveContainer(store.asContainer(/user/ + job.getUser())); wrapped.init(store); + +// Mimic org.apache.hadoop.mapred.FileSplit if feasible... +String[] locations = wrapped.getLocations(); +if (locations.length 0) { +job.set(map.input.file, locations[0]); +job.setLong(map.input.start, wrapped.getStart()); +job.setLong(map.input.length, wrapped.getLength()); +} + return new RecordReaderText, Tuple() { public void close() throws IOException { Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java?rev=647997r1=647996r2=647997view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java Mon Apr 14 14:04:05 2008 @@ -187,12 +187,13 @@ } processError(\nCommand: + sb.toString()); processError(\nStart time: + new Date(System.currentTimeMillis())); -processError(\nInput-split file: + job.get(map.input.file)); -processError(\nInput-split start-offset: + -job.getLong(map.input.start, -1)); -processError(\nInput-split length: + -job.getLong
svn commit: r647208 - in /incubator/pig/trunk: CHANGES.txt src/org/apache/pig/impl/logicalLayer/LOSplitOutput.java test/org/apache/pig/test/TestPigSplit.java
Author: gates Date: Fri Apr 11 09:14:15 2008 New Revision: 647208 URL: http://svn.apache.org/viewvc?rev=647208view=rev Log: PIG-178: Use of schema on secondary output of SPLIT throws IndexOutOfBoundsException. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOSplitOutput.java incubator/pig/trunk/test/org/apache/pig/test/TestPigSplit.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=647208r1=647207r2=647208view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Fri Apr 11 09:14:15 2008 @@ -218,3 +218,6 @@ PIG-111: Reworked configuration to be setable via properties. (Contributions from joa23, pi_song, and oae via gates). + + PIG-178: Use of schema on secondary output of SPLIT throws + IndexOutOfBoundsException (kali via gates). Modified: incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOSplitOutput.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOSplitOutput.java?rev=647208r1=647207r2=647208view=diff == --- incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOSplitOutput.java (original) +++ incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOSplitOutput.java Fri Apr 11 09:14:15 2008 @@ -43,7 +43,7 @@ @Override public TupleSchema outputSchema() { -return opTable.get(inputs.get(index)).outputSchema(); +return opTable.get(getInputs().get(0)).outputSchema(); } @Override Modified: incubator/pig/trunk/test/org/apache/pig/test/TestPigSplit.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestPigSplit.java?rev=647208r1=647207r2=647208view=diff == --- incubator/pig/trunk/test/org/apache/pig/test/TestPigSplit.java (original) +++ incubator/pig/trunk/test/org/apache/pig/test/TestPigSplit.java Fri Apr 11 09:14:15 2008 @@ -64,4 +64,36 @@ f.delete(); } +@Test +public void testSchemaWithSplit() throws Exception { +File f = File.createTempFile(tmp, ); + +PrintWriter pw = new PrintWriter(f); +pw.println(2); +pw.println(12); +pw.println(42); +pw.close(); +pig.registerQuery(a = load 'file: + f + ' as (value);); +pig.registerQuery(split a into b if value 20, c if value 10;); +pig.registerQuery(b1 = order b by value;); +pig.registerQuery(c1 = order c by value;); + +// order in lexicographic, so 12 comes before 2 +IteratorTuple iter = pig.openIterator(b1); +assertTrue(b1 has an element, iter.hasNext()); +assertEquals(first item in b1, iter.next().getAtomField(0).longVal(), 12); +assertTrue(b1 has an element, iter.hasNext()); +assertEquals(second item in b1, iter.next().getAtomField(0).longVal(), 2); +assertFalse(b1 is over, iter.hasNext()); + +iter = pig.openIterator(c1); +assertTrue(c1 has an element, iter.hasNext()); +assertEquals(first item in b1, iter.next().getAtomField(0).longVal(), 12); +assertTrue(c1 has an element, iter.hasNext()); +assertEquals(second item in b1, iter.next().getAtomField(0).longVal(), 42); +assertFalse(c1 is over, iter.hasNext()); + +f.delete(); +} + }
svn commit: r647253 [2/2] - in /incubator/pig/trunk: ./ src/org/apache/pig/ src/org/apache/pig/backend/local/executionengine/ src/org/apache/pig/data/ src/org/apache/pig/impl/eval/ src/org/apache/pig/
Added: incubator/pig/trunk/src/org/apache/pig/pen/ExGen.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/pen/ExGen.java?rev=647253view=auto == --- incubator/pig/trunk/src/org/apache/pig/pen/ExGen.java (added) +++ incubator/pig/trunk/src/org/apache/pig/pen/ExGen.java Fri Apr 11 11:24:23 2008 @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.pig.pen; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; + +import org.apache.pig.backend.executionengine.ExecException; +import org.apache.pig.backend.executionengine.ExecPhysicalOperator; +import org.apache.pig.backend.executionengine.ExecPhysicalPlan; +import org.apache.pig.backend.local.executionengine.LocalExecutionEngine; +import org.apache.pig.data.DataBag; +import org.apache.pig.data.ExampleTuple; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.PigContext; +import org.apache.pig.impl.logicalLayer.LOLoad; +import org.apache.pig.impl.logicalLayer.LogicalOperator; +import org.apache.pig.impl.logicalLayer.LogicalPlan; +import org.apache.pig.impl.logicalLayer.OperatorKey; +import org.apache.pig.impl.util.IdentityHashSet; +import org.apache.pig.impl.util.LineageTracer; + +public class ExGen { + + final static int SAMPLE_SIZE = 1;// size of sample used in initial downstream pass + +static MapLOLoad, DataBag GlobalBaseData = new HashMapLOLoad, DataBag(); + +public static MapLogicalOperator, DataBag GenerateExamples(LogicalPlan plan, PigContext pigContext) throws IOException { + + long time = System.currentTimeMillis(); + String Result; + + //compile the logical plan to get the physical plan once and for all + ExecPhysicalPlan PhyPlan = null; + try { + PhyPlan = pigContext.getExecutionEngine().compile(plan, null); + } catch (ExecException e1) { + // TODO Auto-generated catch block + e1.printStackTrace(); + } + MapOperatorKey, OperatorKey logicalToPhysicalKeys = ((LocalExecutionEngine)pigContext.getExecutionEngine()).getLogicalToPhysicalMap(); + MapOperatorKey, ExecPhysicalOperator physicalOpTable = PhyPlan.getOpTable(); + +// Acquire initial base data by sampling from input relations (this is idempotent) + FetchBaseData.ReadBaseData(plan.getRootOperator(), GlobalBaseData, SAMPLE_SIZE, pigContext); + +/// PASS 1: push data sample through query plan + +// Push base data through query plan +MapLogicalOperator, DataBag derivedData; +MapLogicalOperator, CollectionIdentityHashSetTuple OperatorToEqClasses = new HashMapLogicalOperator, CollectionIdentityHashSetTuple(); +CollectionIdentityHashSetTuple equivalenceClasses = new LinkedListIdentityHashSetTuple(); +derivedData = DerivedData.CreateDerivedData(plan.getRootOperator(), GlobalBaseData, logicalToPhysicalKeys, physicalOpTable); +//derivedData = DerivedData.CreateDerivedData(plan.getRootOperator(), GlobalBaseData, logicalToPhysicalKeys, physicalOpTable); + +/// PASS 2: augment data sample to reduce sparsity + +// Use constraint back-prop to synthesize additional base data, in order to reduce sparsity +// (and keep track of which tuples are synthetic) +IdentityHashSetTuple syntheticTuples = new IdentityHashSetTuple(); +MapLOLoad, DataBag modifiedBaseData = AugmentData.AugmentBaseData(plan.getRootOperator(), GlobalBaseData, syntheticTuples, derivedData, pigContext); + +{ + LineageTracer lineage = new LineageTracer(); + derivedData = DerivedData.CreateDerivedData(plan.getRootOperator(), modifiedBaseData, lineage, equivalenceClasses, OperatorToEqClasses, logicalToPhysicalKeys, physicalOpTable); + modifiedBaseData = ShapeLineage.PruneBaseData(modifiedBaseData,
svn commit: r647335 - in /incubator/pig/trunk: CHANGES.txt src/org/apache/pig/Main.java
Author: gates Date: Fri Apr 11 15:22:09 2008 New Revision: 647335 URL: http://svn.apache.org/viewvc?rev=647335view=rev Log: PIG-203: Fix bug in parameter substitution code where any pig script over 1k caused pig to freeze. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/Main.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=647335r1=647334r2=647335view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Fri Apr 11 15:22:09 2008 @@ -223,3 +223,6 @@ IndexOutOfBoundsException (kali via gates). PIG-59: Add a new ILLUSTRATE command (shubhamc via gates). + + PIG-203: Fix bug in parameter substitution code where any pig script over + 1k caused pig to freeze. (kali via gates) Modified: incubator/pig/trunk/src/org/apache/pig/Main.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/Main.java?rev=647335r1=647334r2=647335view=diff == --- incubator/pig/trunk/src/org/apache/pig/Main.java (original) +++ incubator/pig/trunk/src/org/apache/pig/Main.java Fri Apr 11 15:22:09 2008 @@ -380,11 +380,10 @@ return new BufferedReader(new FileReader (scriptFile)); } else { -PipedWriter pw = new PipedWriter(); -PipedReader pr = new PipedReader(pw); -psp.genSubstitutedFile (origPigScript, pw, params.size() 0 ? params.toArray(type1) : null, +StringWriter writer = new StringWriter(); +psp.genSubstitutedFile (origPigScript, writer, params.size() 0 ? params.toArray(type1) : null, paramFiles.size() 0 ? paramFiles.toArray(type2) : null); -return new BufferedReader(pr); +return new BufferedReader(new StringReader(writer.toString())); } }
svn commit: r647356 - in /incubator/pig/trunk: ./ src/org/apache/pig/backend/hadoop/executionengine/ src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/ src/org/apache/pig/impl/eval/
Author: gates Date: Fri Apr 11 17:33:09 2008 New Revision: 647356 URL: http://svn.apache.org/viewvc?rev=647356view=rev Log: PIG-204: Repair broken input splits. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigInputFormat.java incubator/pig/trunk/src/org/apache/pig/impl/eval/EvalSpec.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=647356r1=647355r2=647356view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Fri Apr 11 17:33:09 2008 @@ -226,3 +226,5 @@ PIG-203: Fix bug in parameter substitution code where any pig script over 1k caused pig to freeze. (kali via gates) + + PIG-204: Repair broken input splits (acmurthy via gates). Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java?rev=647356r1=647355r2=647356view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java Fri Apr 11 17:33:09 2008 @@ -233,21 +233,28 @@ private void connectInputs(OperatorKey[] compiledInputs, POMapreduce pom, MapOperatorKey, ExecPhysicalOperator physicalOpTable) throws IOException { -// connect inputs (by merging operators, if possible; else connect via temp files) +// connect inputs (by merging operators, if possible; +// else connect via temp files) for (int i = 0; i compiledInputs.length; i++) { -if (okayToMergeWithBinaryOp((POMapreduce)physicalOpTable.get(compiledInputs[i]))) { +POMapreduce input = +(POMapreduce)physicalOpTable.get(compiledInputs[i]); + +if (okayToMergeWithBinaryOp(input)) { // can merge input i with this operator - pom.addInputFile(((POMapreduce)physicalOpTable.get(compiledInputs[i])).getFileSpec(0), - ((POMapreduce)physicalOpTable.get(compiledInputs[i])).getEvalSpec(0)); - pom.addInputOperators(((PhysicalOperator)physicalOpTable.get(compiledInputs[i])).inputs); +pom.addInputFile(input.getFileSpec(0), input.getEvalSpec(0)); +pom.addInputOperators(input.inputs); } else { // chain together via a temp file String tempFile = getTempFile(pigContext); -FileSpec fileSpec = new FileSpec( tempFile, BinStorage.class.getName()); - ((POMapreduce)physicalOpTable.get(compiledInputs[i])).outputFileSpec = fileSpec; +FileSpec fileSpec = new FileSpec(tempFile, + BinStorage.class.getName()); +input.outputFileSpec = fileSpec; pom.addInputFile(fileSpec); pom.addInputOperator(compiledInputs[i]); } + +// propagate input properties +pom.properties.putAll(input.properties); } } Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigInputFormat.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigInputFormat.java?rev=647356r1=647355r2=647356view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigInputFormat.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigInputFormat.java Fri Apr 11 17:33:09 2008 @@ -47,7 +47,7 @@ @SuppressWarnings(unchecked) public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { -boolean isSplittable = job.getBoolean(pig.input.splittable, false); +boolean isSplittable = job.getBoolean(pig.input.splittable, true); ArrayListFileSpec inputs = (ArrayListFileSpec) ObjectSerializer .deserialize(job.get(pig.inputs)); ArrayListEvalSpec mapFuncs = (ArrayListEvalSpec) ObjectSerializer Modified: incubator/pig/trunk/src/org/apache/pig/impl/eval/EvalSpec.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/impl/eval/EvalSpec.java?rev
svn commit: r646993 - in /incubator/pig/trunk/conf: ./ pig.properties
Author: gates Date: Thu Apr 10 15:26:24 2008 New Revision: 646993 URL: http://svn.apache.org/viewvc?rev=646993view=rev Log: PIG-111 Template for properties file that should have been checked in with revision 646988. Added: incubator/pig/trunk/conf/ incubator/pig/trunk/conf/pig.properties Added: incubator/pig/trunk/conf/pig.properties URL: http://svn.apache.org/viewvc/incubator/pig/trunk/conf/pig.properties?rev=646993view=auto == --- incubator/pig/trunk/conf/pig.properties (added) +++ incubator/pig/trunk/conf/pig.properties Thu Apr 10 15:26:24 2008 @@ -0,0 +1,34 @@ +# Pig configuration file. All values can be overwritten by command line arguments. +# see bin/pig -help + +# log4jconf log4j configuration file +log4jconf=./conf/log4j.properties + +# brief logging (no timestamps) +brief=false + +# clustername, name of the hadoop jobtracker. If no port is defined port 50020 will be used. +#cluster + +#debug level, INFO is default +debug=INFO + +# a file that contains pig script +#file= + +# load jarfile, colon separated +#jar= + +#verbose print all log messages to screen (default to print only INFO and above to screen) +verbose=false + +#exectype local|mapreduce, mapreduce is default +exectype=local +# hod realted properties +#ssh.gateway +#hod.expect.root +#hod.expect.uselatest +#hod.command +#hod.config.dir +#hod.param +
svn commit: r646546 - /incubator/pig/trunk/src/org/apache/pig/tools/parameters/ParameterSubstitutionPreprocessor.java
Author: gates Date: Wed Apr 9 14:23:38 2008 New Revision: 646546 URL: http://svn.apache.org/viewvc?rev=646546view=rev Log: Removed unnecessary import statement that caused builds to not work on 1.5 JVM. Modified: incubator/pig/trunk/src/org/apache/pig/tools/parameters/ParameterSubstitutionPreprocessor.java Modified: incubator/pig/trunk/src/org/apache/pig/tools/parameters/ParameterSubstitutionPreprocessor.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/tools/parameters/ParameterSubstitutionPreprocessor.java?rev=646546r1=646545r2=646546view=diff == --- incubator/pig/trunk/src/org/apache/pig/tools/parameters/ParameterSubstitutionPreprocessor.java (original) +++ incubator/pig/trunk/src/org/apache/pig/tools/parameters/ParameterSubstitutionPreprocessor.java Wed Apr 9 14:23:38 2008 @@ -28,7 +28,6 @@ import org.apache.log4j.Logger; import java.util.Hashtable; -import java.io.Console; import java.io.FileInputStream; import java.io.FileReader; import java.io.IOException;
svn commit: r646189 - in /incubator/pig/trunk: CHANGES.txt src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java test/org/apache/pig/test/TestMapReduceResultRecycling.java
Author: gates Date: Tue Apr 8 21:38:56 2008 New Revision: 646189 URL: http://svn.apache.org/viewvc?rev=646189view=rev Log: PIG-153: Incorrect result caused by dump in between statements. Added: incubator/pig/trunk/test/org/apache/pig/test/TestMapReduceResultRecycling.java Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=646189r1=646188r2=646189view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Tue Apr 8 21:38:56 2008 @@ -207,3 +207,6 @@ PIG-182: streaming bug fix (arunc via olgan) PIG-184: streaming bug fixes + + PIG-153: Incorrect result caused by dump in between statements (pi_song + via gates). Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java?rev=646189r1=646188r2=646189view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java Tue Apr 8 21:38:56 2008 @@ -280,7 +280,7 @@ pom.close(); -this.materializedResults.put(pom.sourceLogicalKey, +this.materializedResults.put(plan.getRoot(), new MapRedResult(pom.outputFileSpec, pom.reduceParallelism)); } Added: incubator/pig/trunk/test/org/apache/pig/test/TestMapReduceResultRecycling.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestMapReduceResultRecycling.java?rev=646189view=auto == --- incubator/pig/trunk/test/org/apache/pig/test/TestMapReduceResultRecycling.java (added) +++ incubator/pig/trunk/test/org/apache/pig/test/TestMapReduceResultRecycling.java Tue Apr 8 21:38:56 2008 @@ -0,0 +1,67 @@ +package org.apache.pig.test; + +import static org.apache.pig.PigServer.ExecType ; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.PrintStream; +import java.util.Iterator; + +import org.apache.pig.PigServer; +import org.apache.pig.data.Tuple; +import org.junit.Test; + +import junit.framework.TestCase; + +public class TestMapReduceResultRecycling extends TestCase { +MiniCluster cluster = MiniCluster.buildCluster(); + +@Test +public void testPlanRecycling() throws Throwable { +PigServer pig = new PigServer(ExecType.MAPREDUCE); +File tmpFile = this.createTempFile(); +{ +String query = a = load 'file: + tmpFile + '; ; +System.out.println(query); +pig.registerQuery(query); +pig.explain(a, System.out) ; +IteratorTuple it = pig.openIterator(a); +assertTrue(it.next().getAtomField(0).strval().equals(a1)) ; +assertTrue(it.next().getAtomField(0).strval().equals(b1)) ; +assertTrue(it.next().getAtomField(0).strval().equals(c1)) ; +assertFalse(it.hasNext()) ; +} + +{ +String query = b = filter a by $0 eq 'a1'; ; +System.out.println(query); +pig.registerQuery(query); +pig.explain(b, System.out) ; +IteratorTuple it = pig.openIterator(b); +assertTrue(it.next().getAtomField(0).strval().equals(a1)) ; +assertFalse(it.hasNext()) ; +} + +{ +String query = c = filter a by $0 eq 'b1'; ; +System.out.println(query); +pig.registerQuery(query); +pig.explain(c, System.out) ; +IteratorTuple it = pig.openIterator(c); +assertTrue(it.next().getAtomField(0).strval().equals(b1)) ; +assertFalse(it.hasNext()) ; +} + +} + +private File createTempFile() throws Throwable { +File tmpFile = File.createTempFile(pi_test1, txt); +tmpFile.deleteOnExit() ; +PrintStream ps = new PrintStream(new FileOutputStream(tmpFile)); +ps.println(a1\t1\t1000) ; +ps.println(b1\t2\t1000) ; +ps.println(c1\t3\t1000) ; +ps.close(); +return tmpFile ; +} +}
svn commit: r642504 - in /incubator/pig/trunk: CHANGES.txt src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigCombine.java
Author: gates Date: Fri Mar 28 21:37:30 2008 New Revision: 642504 URL: http://svn.apache.org/viewvc?rev=642504view=rev Log: Fixed PigCombine to not do initialization on every call to reduce, but instead only do it once in the call to configure. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigCombine.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=642504r1=642503r2=642504view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Fri Mar 28 21:37:30 2008 @@ -184,5 +184,9 @@ PIG-94: changes for M2 of streaming: input/ouptut/ ship/cache error handling + + PIG-108: Fixed PigCombine to not do initialization on every call to + reduce, but instead only do it once in the call to configure. (joa23 via + gates). PIG-172: dealing with NULL error messages in exceptions Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigCombine.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigCombine.java?rev=642504r1=642503r2=642504view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigCombine.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigCombine.java Fri Mar 28 21:37:30 2008 @@ -17,7 +17,6 @@ */ package org.apache.pig.backend.hadoop.executionengine.mapreduceExec; -import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; @@ -48,23 +47,18 @@ private JobConf job; private CombineDataOutputCollector finalout; private DataCollector evalPipe; -private OutputCollector oc; private int index; private int inputCount; private DataBag bags[]; +private PigContext pigContext; +private EvalSpec esp; public void reduce(WritableComparable key, Iterator values, OutputCollector output, Reporter reporter) throws IOException { try { -PigContext pigContext = (PigContext) ObjectSerializer.deserialize(job.get(pig.pigContext)); if (evalPipe == null) { -inputCount = ((ArrayListFileSpec)ObjectSerializer.deserialize(job.get(pig.inputs))).size(); -oc = output; -finalout = new CombineDataOutputCollector(oc); -String evalSpec = job.get(pig.combineFunc, ); -EvalSpec esp = (EvalSpec)ObjectSerializer.deserialize(evalSpec); -if(esp != null) esp.instantiateFunc(pigContext); +finalout = new CombineDataOutputCollector(output); evalPipe = esp.setupPipe(null, finalout); //throw new RuntimeException(combine spec: + evalSpec + combine pipe: + esp.toString()); @@ -112,6 +106,17 @@ */ public void configure(JobConf job) { this.job = job; +try { +this.pigContext = (PigContext) ObjectSerializer.deserialize(job.get(pig.pigContext)); +this.inputCount = ((ArrayListFileSpec)ObjectSerializer.deserialize(job.get(pig.inputs))).size(); +String evalSpec = job.get(pig.combineFunc, ); +this.esp = (EvalSpec)ObjectSerializer.deserialize(evalSpec); +if(esp != null) { + esp.instantiateFunc(pigContext); +} +} catch (IOException e) { +throw new RuntimeException(unable to deserialize data, e); +} } /**
svn commit: r642060 - in /incubator/pig/branches/types: ./ src/org/apache/pig/ src/org/apache/pig/builtin/ test/org/apache/pig/test/
Author: gates Date: Thu Mar 27 19:02:43 2008 New Revision: 642060 URL: http://svn.apache.org/viewvc?rev=642060view=rev Log: Took Shravan's changes to PigStorage and BinStorage so that they'd compile. Added changes to other builtin functions so whole directory now compiles. Changed ant test to only run the tests we expect to pass at this point. Modified: incubator/pig/branches/types/build.xml incubator/pig/branches/types/src/org/apache/pig/EvalFunc.java incubator/pig/branches/types/src/org/apache/pig/builtin/ARITY.java incubator/pig/branches/types/src/org/apache/pig/builtin/AVG.java incubator/pig/branches/types/src/org/apache/pig/builtin/BinStorage.java incubator/pig/branches/types/src/org/apache/pig/builtin/COUNT.java incubator/pig/branches/types/src/org/apache/pig/builtin/MAX.java incubator/pig/branches/types/src/org/apache/pig/builtin/MIN.java incubator/pig/branches/types/src/org/apache/pig/builtin/PigStorage.java incubator/pig/branches/types/src/org/apache/pig/builtin/SUM.java incubator/pig/branches/types/src/org/apache/pig/builtin/TOKENIZE.java incubator/pig/branches/types/src/org/apache/pig/builtin/TextLoader.java incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java incubator/pig/branches/types/test/org/apache/pig/test/TestOperatorPlan.java Modified: incubator/pig/branches/types/build.xml URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/build.xml?rev=642060r1=642059r2=642060view=diff == --- incubator/pig/branches/types/build.xml (original) +++ incubator/pig/branches/types/build.xml Thu Mar 27 19:02:43 2008 @@ -132,7 +132,8 @@ /target target name=compile-sources -javac encoding=${build.encoding} srcdir=${sources} includes=**/plan/*.java, **/data/*.java, **/test/TestOperatorPlan.java, **/logicalLayer/LogicalPlan.java, **/logicalLayer/LOEval.java, **/logicalLayer/LOSort.java, **/logicalLayer/LOGenerate.java, **/logicalLayer/LOVisitor.java, **/logicalLayer/schema/Schema.java destdir=${dist} debug=${javac.debug} optimize=${javac.optimize} target=${javac.version} source=${javac.version} deprecation=${javac.deprecation} +javac encoding=${build.encoding} srcdir=${sources} +includes=**/plan/*.java, **/data/*.java, **/pig/builtin/*.java, **/test/TestOperatorPlan.java, **/test/TestBuiltin.java, **/logicalLayer/LogicalPlan.java, **/logicalLayer/LOEval.java, **/logicalLayer/LOSort.java, **/logicalLayer/LOGenerate.java, **/logicalLayer/LOVisitor.java, **/logicalLayer/schema/Schema.java destdir=${dist} debug=${javac.debug} optimize=${javac.optimize} target=${javac.version} source=${javac.version} deprecation=${javac.deprecation} compilerarg line=${javac.args} ${javac.args.warnings} / classpath refid=${cp} / /javac @@ -214,11 +215,15 @@ formatter type=${test.junit.output.format} / batchtest fork=yes todir=${test.log.dir} fileset dir=test +include name=**/TestBuiltin.java / +include name=**/TestOperatorPlan.java / +!-- include name=**/*Test*.java / exclude name=**/TestLargeFile.java / exclude name=**/TestOrderBy.java / exclude name=**/TestPi.java / exclude name=**/nightly/** / +-- /fileset /batchtest /junit Modified: incubator/pig/branches/types/src/org/apache/pig/EvalFunc.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/EvalFunc.java?rev=642060r1=642059r2=642060view=diff == --- incubator/pig/branches/types/src/org/apache/pig/EvalFunc.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/EvalFunc.java Thu Mar 27 19:02:43 2008 @@ -23,9 +23,9 @@ import java.lang.reflect.Type; import org.apache.pig.data.Tuple; -import org.apache.pig.impl.PigContext; +// TODO FIX +// import org.apache.pig.impl.PigContext; import org.apache.pig.impl.logicalLayer.schema.Schema; -import org.apache.pig.impl.logicalLayer.schema.TupleSchema; /** @@ -73,6 +73,8 @@ //Type check the initial, intermediate, and final functions +// TODO FIX +/* if (this instanceof Algebraic){ Algebraic a = (Algebraic)this; @@ -84,13 +86,16 @@ if (getReturnTypeFromSpec(a.getFinal()) != returnType) throw new RuntimeException(Final + errMsg); } +*/ } private Type getReturnTypeFromSpec(String funcSpec){ try{ -return ((EvalFunc)PigContext.instantiateFuncFromSpec(funcSpec)).getReturnType(); +// TODO FIX +// return
svn commit: r639409 - in /incubator/pig/trunk: CHANGES.txt src/org/apache/pig/impl/util/SpillableMemoryManager.java
Author: gates Date: Thu Mar 20 12:02:00 2008 New Revision: 639409 URL: http://svn.apache.org/viewvc?rev=639409view=rev Log: PIG-164: Fix memory issue in SpillableMemoryManager to partially clean the list of bags each time a new bag is added rather than waiting until the garbage collector tells us we are out of memory. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/impl/util/SpillableMemoryManager.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=639409r1=639408r2=639409view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Thu Mar 20 12:02:00 2008 @@ -169,3 +169,7 @@ PIG-106: Change StringBuffer and String '+' to StringBuilder (francisoud via gates). + + PIG-164: Fix memory issue in SpillableMemoryManager to partially clean the list of + bags each time a new bag is added rather than waiting until the garbage + collector tells us we are out of memory (gates). Modified: incubator/pig/trunk/src/org/apache/pig/impl/util/SpillableMemoryManager.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/impl/util/SpillableMemoryManager.java?rev=639409r1=639408r2=639409view=diff == --- incubator/pig/trunk/src/org/apache/pig/impl/util/SpillableMemoryManager.java (original) +++ incubator/pig/trunk/src/org/apache/pig/impl/util/SpillableMemoryManager.java Thu Mar 20 12:02:00 2008 @@ -32,7 +32,7 @@ private final Log log = LogFactory.getLog(getClass()); -ListWeakReferenceSpillable spillables = new LinkedListWeakReferenceSpillable(); +LinkedListWeakReferenceSpillable spillables = new LinkedListWeakReferenceSpillable(); public SpillableMemoryManager() { ((NotificationEmitter)ManagementFactory.getMemoryMXBean()).addNotificationListener(this, null, null); @@ -142,6 +142,13 @@ */ public void registerSpillable(Spillable s) { synchronized(spillables) { +// Cleaing the entire list is too expensive. Just trim off the front while +// we can. +WeakReferenceSpillable first = spillables.peek(); +while (first != null first.get() == null) { +spillables.remove(); +first = spillables.peek(); +} spillables.add(new WeakReferenceSpillable(s)); } }
svn commit: r638706 - in /incubator/pig/branches/types: ./ lib/ src/org/apache/pig/impl/logicalLayer/schema/ src/org/apache/pig/impl/plan/ test/org/apache/pig/test/
Author: gates Date: Tue Mar 18 21:45:46 2008 New Revision: 638706 URL: http://svn.apache.org/viewvc?rev=638706view=rev Log: Moved OperatorPlan off of Apache Commons MultiValueMap to local implementation of MultiMap. This was necessary so that OperatorPlan could implement Serializable. Added: incubator/pig/branches/types/src/org/apache/pig/impl/plan/MultiMap.java Removed: incubator/pig/branches/types/lib/commons-collections-3.2.jar Modified: incubator/pig/branches/types/build.xml incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java incubator/pig/branches/types/src/org/apache/pig/impl/plan/OperatorPlan.java incubator/pig/branches/types/test/org/apache/pig/test/TestOperatorPlan.java Modified: incubator/pig/branches/types/build.xml URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/build.xml?rev=638706r1=638705r2=638706view=diff == --- incubator/pig/branches/types/build.xml (original) +++ incubator/pig/branches/types/build.xml Tue Mar 18 21:45:46 2008 @@ -132,7 +132,7 @@ /target target name=compile-sources -javac encoding=${build.encoding} srcdir=${sources} includes=**/plan/*.java, **/logicalLayer/LogicalPlan.java, **/logicalLayer/LOEval.java, **/logicalLayer/LOSort.java, **/logicalLayer/LOGenerate.java, **/logicalLayer/LOVisitor.java destdir=${dist} debug=${javac.debug} optimize=${javac.optimize} target=${javac.version} source=${javac.version} deprecation=${javac.deprecation} +javac encoding=${build.encoding} srcdir=${sources} includes=**/plan/*.java, **/test/TestOperatorPlan.java, **/logicalLayer/LogicalPlan.java, **/logicalLayer/LOEval.java, **/logicalLayer/LOSort.java, **/logicalLayer/LOGenerate.java, **/logicalLayer/LOVisitor.java, **/logicalLayer/schema/Schema.java destdir=${dist} debug=${javac.debug} optimize=${javac.optimize} target=${javac.version} source=${javac.version} deprecation=${javac.deprecation} compilerarg line=${javac.args} ${javac.args.warnings} / classpath refid=${cp} / !-- Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java?rev=638706r1=638705r2=638706view=diff == --- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java (original) +++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java Tue Mar 18 21:45:46 2008 @@ -66,6 +66,18 @@ type = DataType.TUPLE; schema = s; } + +@Override +public boolean equals(Object other) { +if (!(other instanceof FieldSchema)) return false; +FieldSchema fs = (FieldSchema)other; +// Fields can have different names and still be equal. But +// types and schemas (if they're a tuple) must match. +if (type != fs.type) return false; +if (schema != fs.schema) return false; + +return true; +} } private ListFieldSchema mFields; @@ -152,7 +164,22 @@ if (otherFs.type != DataType.UNKNOWN) ourFs.type = otherFs.type; if (otherFs.schema != null) ourFs.schema = otherFs.schema; } +} +@Override +public boolean equals(Object other) { +if (!(other instanceof Schema)) return false; + +Schema s = (Schema)other; + +if (s.size() != size()) return false; + +IteratorFieldSchema i = mFields.iterator(); +IteratorFieldSchema j = s.mFields.iterator(); +while (i.hasNext()) { +if (!(i.next().equals(j.next( return false; +} +return true; } } Added: incubator/pig/branches/types/src/org/apache/pig/impl/plan/MultiMap.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/plan/MultiMap.java?rev=638706view=auto == --- incubator/pig/branches/types/src/org/apache/pig/impl/plan/MultiMap.java (added) +++ incubator/pig/branches/types/src/org/apache/pig/impl/plan/MultiMap.java Tue Mar 18 21:45:46 2008 @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing
svn commit: r637987 - in /incubator/pig/trunk: CHANGES.txt bin/ bin/pig bin/startHOD.expect scripts/pig.pl scripts/startHOD.expect
Author: gates Date: Mon Mar 17 10:40:59 2008 New Revision: 637987 URL: http://svn.apache.org/viewvc?rev=637987view=rev Log: Removed Yahoo specific scripts/pig.pl, replaced with generic bash script bin/pig. Moved startHOD.expect to bin. Added: incubator/pig/trunk/bin/ incubator/pig/trunk/bin/pig incubator/pig/trunk/bin/startHOD.expect Removed: incubator/pig/trunk/scripts/pig.pl incubator/pig/trunk/scripts/startHOD.expect Modified: incubator/pig/trunk/CHANGES.txt Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=637987r1=637986r2=637987view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Mon Mar 17 10:40:59 2008 @@ -163,3 +163,6 @@ PIG-129: making sure that temp files are stored in task's home dir and cleaned up + +PIG-115: Removed Yahoo specific scripts/pig.pl, replaced with generic +bash script bin/pig. Moved startHOD.expect to bin (joa23 via gates). Added: incubator/pig/trunk/bin/pig URL: http://svn.apache.org/viewvc/incubator/pig/trunk/bin/pig?rev=637987view=auto == --- incubator/pig/trunk/bin/pig (added) +++ incubator/pig/trunk/bin/pig Mon Mar 17 10:40:59 2008 @@ -0,0 +1,175 @@ +#!/usr/bin/env bash +# +# The Pig command script +# +# Environment Variables +# +# JAVA_HOMEThe java implementation to use.Overrides JAVA_HOME. +# +# PIG_CLASSPATH Extra Java CLASSPATH entries. +# +# PIG_HEAPSIZEThe maximum amount of heap to use, in MB. +#Default is 1000. +# +# PIG_OPTSExtra Java runtime options. +# +# PIG_CONF_DIRAlternate conf dir. Default is ${PIG_HOME}/conf. +# +# PIG_ROOT_LOGGER The root appender. Default is INFO,console +# +# PIG_HADOOP_VERSION Version of hadoop to run with.Default is 15 (0.15). + +cygwin=false +case `uname` in +CYGWIN*) cygwin=true;; +esac +debug=false + +# filter command line parameter +for f in $@; do + if [[ $f = -secretDebugCmd ]]; then +debug=true + else +remaining=${remaining} $f + fi +done + +# resolve links - $0 may be a softlink +this=$0 +while [ -h $this ]; do +ls=`ls -ld $this` +link=`expr $ls : '.*- \(.*\)$'` +if expr $link : '.*/.*' /dev/null; then +this=$link +else +this=`dirname $this`/$link +fi +done + +# convert relative path to absolute path +bin=`dirname $this` +script=`basename $this` +bin=`unset CDPATH; cd $bin; pwd` +this=$bin/$script + +# the root of the Pig installation +export PIG_HOME=`dirname $this`/.. + +#check to see if the conf dir is given as an optional argument +if [ $# -gt 1 ] +then +if [ --config = $1 ] +then +shift +confdir=$1 +shift +PIG_CONF_DIR=$confdir +fi +fi + +# Allow alternate conf dir location. +PIG_CONF_DIR=${PIG_CONF_DIR:-$PIG_HOME/conf} + +if [ -f ${PIG_CONF_DIR}/pig-env.sh ]; then +. ${PIG_CONF_DIR}/pig-env.sh +fi + +# some Java parameters +if [ $JAVA_HOME != ]; then +#echo run java in $JAVA_HOME +JAVA_HOME=$JAVA_HOME +fi + +if [ $JAVA_HOME = ]; then +echo Error: JAVA_HOME is not set. +exit 1 +fi + +JAVA=$JAVA_HOME/bin/java +JAVA_HEAP_MAX=-Xmx1000m + +# check envvars which might override default args +if [ $PIG_HEAPSIZE != ]; then +JAVA_HEAP_MAX=-Xmx$PIG_HEAPSIZEm +fi + +# CLASSPATH initially contains $PIG_CONF_DIR +CLASSPATH=${PIG_CONF_DIR} +CLASSPATH=${CLASSPATH}:$JAVA_HOME/lib/tools.jar + +# for developers, add Pig classes to CLASSPATH +if [ -d $PIG_HOME/build/classes ]; then +CLASSPATH=${CLASSPATH}:$PIG_HOME/build/classes +fi +if [ -d $PIG_HOME/build/test/classes ]; then +CLASSPATH=${CLASSPATH}:$PIG_HOME/build/test/classes +fi + +# so that filenames w/ spaces are handled correctly in loops below +IFS= + +# for releases, add core pig to CLASSPATH +for f in $PIG_HOME/pig-*-core.jar; do +CLASSPATH=${CLASSPATH}:$f; +done + +# during development pig jar might be in build +for f in $PIG_HOME/build/pig-*-core.jar; do +CLASSPATH=${CLASSPATH}:$f; +done + +# Set the version for Hadoop, default to 15 +PIG_HADOOP_VERSION=${PIG_HADOOP_VERSION:-15} +# add libs to CLASSPATH.There can be more than one version of the hadoop +# libraries in the lib dir, so don't blindly add them all.Only add the one +# that matche PIG_HADOOP_VERSION. +for f in $PIG_HOME/lib/*.jar; do +IS_HADOOP=`echo $f | grep hadoop` +if [ ${IS_HADOOP}x == x ]; then +CLASSPATH=${CLASSPATH}:$f; +else +IS_RIGHT_VER=`echo $f | grep hadoop${PIG_HADOOP_VERSION}.jar` +if [ ${IS_RIGHT_VER}x != x ]; then +CLASSPATH=${CLASSPATH}:$f; +fi +fi +done + +# add user-specified CLASSPATH last +if [ $PIG_CLASSPATH != ]; then +CLASSPATH=${CLASSPATH
svn commit: r633652 - in /incubator/pig/trunk: CHANGES.txt src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java
Author: gates Date: Tue Mar 4 13:20:52 2008 New Revision: 633652 URL: http://svn.apache.org/viewvc?rev=633652view=rev Log: PIG-120: Support map reduce in local mode. To do this user needs to specify execution type as mapreduce and cluster name as local. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=633652r1=633651r2=633652view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Tue Mar 4 13:20:52 2008 @@ -149,3 +149,7 @@ PIG-13: adding version to the system (joa23 via olgan) PIG-113: Make explain output more understandable (pi_song via gates) + + PIG-120: Support map reduce in local mode. To do this user needs to + specify execution type as mapreduce and cluster name as local (joa23 via + gates). Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java?rev=633652r1=633651r2=633652view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java Tue Mar 4 13:20:52 2008 @@ -110,6 +110,8 @@ setSSHFactory(); String hodServer = System.getProperty(hod.server); +String cluster = System.getProperty(cluster); +String nameNode = System.getProperty(namenode); if (hodServer != null hodServer.length() 0) { String hdfsAndMapred[] = doHod(hodServer); @@ -117,17 +119,15 @@ setJobtrackerLocation(hdfsAndMapred[1]); } else { -String cluster = System.getProperty(cluster); if (cluster != null cluster.length() 0) { -if(cluster.indexOf(':') 0) { +if(cluster.indexOf(':') 0 !cluster.equalsIgnoreCase(local)) { cluster = cluster + :50020; } setJobtrackerLocation(cluster); } -String nameNode = System.getProperty(namenode); if (nameNode!=null nameNode.length() 0) { -if(nameNode.indexOf(':') 0) { +if(nameNode.indexOf(':') 0 !nameNode.equalsIgnoreCase(local)) { nameNode = nameNode + :8020; } setFilesystemLocation(nameNode); @@ -143,16 +143,18 @@ throw new ExecException(Failed to create DataStorage, e); } -log.info(Connecting to map-reduce job tracker at: + conf.get(mapred.job.tracker)); - -try { -jobTracker = (JobSubmissionProtocol) RPC.getProxy(JobSubmissionProtocol.class, - JobSubmissionProtocol.versionID, - JobTracker.getAddress(conf.getConfiguration()), - conf.getConfiguration()); -} -catch (IOException e) { -throw new ExecException(Failed to crate job tracker, e); +if(cluster != null !cluster.equalsIgnoreCase(local)){ + log.info(Connecting to map-reduce job tracker at: + conf.get(mapred.job.tracker)); + + try { + jobTracker = (JobSubmissionProtocol) RPC.getProxy(JobSubmissionProtocol.class, + JobSubmissionProtocol.versionID, + JobTracker.getAddress(conf.getConfiguration()), + conf.getConfiguration()); + } + catch (IOException e) { + throw new ExecException(Failed to crate job tracker, e); + } } try {
svn commit: r630997 - in /incubator/pig/trunk: CHANGES.txt src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java test/org/apache/pig/test/TestCombiner.java
Author: gates Date: Mon Feb 25 13:15:25 2008 New Revision: 630997 URL: http://svn.apache.org/viewvc?rev=630997view=rev Log: PIG-110: Replaced code accidently merged out in PIG-32 fix that handled flattening the combiner case. Added: incubator/pig/trunk/test/org/apache/pig/test/TestCombiner.java Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=630997r1=630996r2=630997view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Mon Feb 25 13:15:25 2008 @@ -127,3 +127,6 @@ be run w/o access to a hadoop cluster. (xuzh via gates) PIG-68: improvements to build.xml (joa23 via olgan) + + PIG-110: Replaced code accidently merged out in PIG-32 fix that handled + flattening the combiner case. (gates and oae) Modified: incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java?rev=630997r1=630996r2=630997view=diff == --- incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java (original) +++ incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/MapreducePlanCompiler.java Mon Feb 25 13:15:25 2008 @@ -498,11 +498,23 @@ private class CombineAdjuster extends EvalSpecVisitor { private int position = 0; +//We don't want to be performing any flattening in the combiner since the column numbers in +//the reduce spec assume that there is no combiner. If the combiner performs flattening, the column +//numbers get messed up. For now, since combiner works only with generate group, func1(), func2(),..., +//it suffices to write visitors for those eval spec types. + public void visitFuncEval(FuncEvalSpec fe) { // Reset the function to call the initial instance of itself // instead of the general instance. fe.resetFuncToInitial(); +fe.setFlatten(false); } + +@Override +public void visitProject(ProjectSpec p) { +p.setFlatten(false); +} + } private class CombineDeterminer extends EvalSpecVisitor { Added: incubator/pig/trunk/test/org/apache/pig/test/TestCombiner.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestCombiner.java?rev=630997view=auto == --- incubator/pig/trunk/test/org/apache/pig/test/TestCombiner.java (added) +++ incubator/pig/trunk/test/org/apache/pig/test/TestCombiner.java Mon Feb 25 13:15:25 2008 @@ -0,0 +1,63 @@ +package org.apache.pig.test; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.junit.Test; +import junit.framework.TestCase; + +import org.apache.pig.PigServer; +import org.apache.pig.builtin.PigStorage; +import org.apache.pig.data.Tuple; + +public class TestCombiner extends TestCase { + +@Test +public void testLocal() throws Exception { +// run the test locally +runTest(new PigServer(local)); +} + +@Test +public void testOnCluster() throws Exception { +// run the test on cluster +MiniCluster.buildCluster(); +runTest(new PigServer(mapreduce)); + +} + +private void runTest(PigServer pig) throws IOException { +ListString inputLines = new ArrayListString(); +inputLines.add(a,b,1); +inputLines.add(a,b,1); +inputLines.add(a,c,1); +loadWithTestLoadFunc(A, pig, inputLines); + +pig.registerQuery(B = group A by ($0, $1);); +pig.registerQuery(C = foreach B generate flatten(group), COUNT($1);); +IteratorTuple resultIterator = pig.openIterator(C); +Tuple tuple = resultIterator.next(); +assertEquals((a, b, 2), tuple.toString()); +tuple = resultIterator.next(); +assertEquals((a, c, 1), tuple.toString()); +} + +private void loadWithTestLoadFunc(String loadAlias, PigServer pig, +ListString inputLines) throws IOException { +File inputFile = File.createTempFile(test, txt); +PrintStream ps = new PrintStream(new FileOutputStream(inputFile)); +for (String line : inputLines) { +ps.println(line); +} +ps.close(); +pig.registerQuery(loadAlias + = load 'file: ++ inputFile + ' using
svn commit: r631029 - /incubator/pig/trunk/build.xml
Author: gates Date: Mon Feb 25 14:59:57 2008 New Revision: 631029 URL: http://svn.apache.org/viewvc?rev=631029view=rev Log: Added back copy of pig-X.jar from build directory to pig.jar in top directory for backward compatibility. Modified: incubator/pig/trunk/build.xml Modified: incubator/pig/trunk/build.xml URL: http://svn.apache.org/viewvc/incubator/pig/trunk/build.xml?rev=631029r1=631028r2=631029view=diff == --- incubator/pig/trunk/build.xml (original) +++ incubator/pig/trunk/build.xml Mon Feb 25 14:59:57 2008 @@ -45,6 +45,8 @@ !-- jar names. TODO we might want to use the svn reversion name in the name in case it is a dev version -- property name=output.jarfile value=${build.dir}/${final.name}.jar / property name=output.jarfile.core value=${build.dir}/${final.name}-core.jar / +!-- Maintain old pig.jar in top level directory. -- +property name=output.jarfile.backcompat value=${basedir}/${name}.jar / !-- test properties -- property name=test.src.dir value=${basedir}/test / @@ -187,6 +189,7 @@ zipfileset src=${lib.dir}/${hadoop.jarfile} / zipfileset src=${lib.dir}/jsch-0.1.33.jar / /jar +copy file=${output.jarfile} tofile=${output.jarfile.backcompat}/ /target !-- == -- @@ -280,4 +283,4 @@ /tar /target -/project \ No newline at end of file +/project
svn commit: r631045 - in /incubator/pig/branches/types/src/org/apache/pig/data: DataByteArray.java DataReaderWriter.java DefaultTuple.java
Author: gates Date: Mon Feb 25 15:56:02 2008 New Revision: 631045 URL: http://svn.apache.org/viewvc?rev=631045view=rev Log: Files that should have been added in previous checkins. Added: incubator/pig/branches/types/src/org/apache/pig/data/DataByteArray.java incubator/pig/branches/types/src/org/apache/pig/data/DataReaderWriter.java incubator/pig/branches/types/src/org/apache/pig/data/DefaultTuple.java Added: incubator/pig/branches/types/src/org/apache/pig/data/DataByteArray.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/data/DataByteArray.java?rev=631045view=auto == --- incubator/pig/branches/types/src/org/apache/pig/data/DataByteArray.java (added) +++ incubator/pig/branches/types/src/org/apache/pig/data/DataByteArray.java Mon Feb 25 15:56:02 2008 @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.pig.data; + +import java.io.IOException; +import java.lang.StringBuilder; +import java.util.ArrayList; +import java.util.Collection; + +/** + * An implementation of byte array. This is done as an object because we + * need to be able to implement compareTo, toString, hashCode, and some + * other methods. + */ +public class DataByteArray implements Comparable { +byte[] mData = null; + +/** + * Default constructor. The data array will not be allocated when this + * constructor is called. + */ +public DataByteArray() { +} + +/** + * Construct a byte array using the provided bytes as the content. + * @param b byte array to use as content. A reference to the bytes + * will be taken, the underlying bytes will not be copied. + */ +public DataByteArray(byte[] b) { +mData = b; +} + +/** + * Construct a byte array from a String. The contents of the string + * are copied. + * @param s String to make a byte array out of. + */ +public DataByteArray(String s) { +mData = s.getBytes(); +} + +/** + * Find the size of the byte array. + * @return number of bytes in the array. + */ +public int size() { +return mData.length; +} + +/** + * Get the underlying byte array. This is the real thing, not a copy, + * so don't mess with it! + * @return underlying byte[] + */ +public byte[] get() { +return mData; +} + +/** + * Set the internal byte array. This should not be called unless the + * default constructor was used. + * @param b byte array to store. The contents of the byte array are + * not copied. + */ +public void set(byte[] b) { +mData = b; +} + +/** + * Set the internal byte array. This should not be called unless the + * default constructor was used. + * @param s String to copy. The contents of the string are copied. + */ +public void set(String s) { +mData = s.getBytes(); +} + +@Override +public String toString() { +return new String(mData); +} + +public int compareTo(Object other) { +if (other instanceof DataByteArray) { +DataByteArray dba = (DataByteArray)other; +int mySz = mData.length; +int tSz = dba.mData.length; +if (tSz mySz) { +return 1; +} else if (tSz mySz) { +return -1; +} else { +for (int i = 0; i mySz; i++) { +if (mData[i] dba.mData[i]) return -1; +else if (mData[i] dba.mData[i]) return 1; +} +return 0; +} +} else { +return DataType.compare(this, other); +} +} + +@Override +public boolean equals(Object other) { +return (compareTo(other) == 0); +} + +@Override +public int hashCode() { +int hash = 1; +for (int i = 0; i mData.length; i++) { +// 29 chosen because hash uses 31 and bag 37, and a I want a +// prime. +hash = 29 * hash + mData[i]; +} +return hash; +} + +} Added
svn commit: r630357 [2/2] - in /incubator/pig/branches/types: ./ src/org/apache/pig/builtin/ src/org/apache/pig/data/ src/org/apache/pig/impl/builtin/ src/org/apache/pig/impl/eval/ src/org/apache/pig/
Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java?rev=630357r1=630356r2=630357view=diff == --- incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java (original) +++ incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java Fri Feb 22 15:44:37 2008 @@ -15,298 +15,307 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.pig.test; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.PrintStream; -import java.io.PrintWriter; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.StringTokenizer; - -import org.junit.Test; - -import org.apache.pig.EvalFunc; -import org.apache.pig.PigServer; -import org.apache.pig.builtin.BinStorage; -import org.apache.pig.builtin.PigStorage; -import org.apache.pig.builtin.TextLoader; -import org.apache.pig.data.*; -import org.apache.pig.impl.io.FileLocalizer; -import org.apache.pig.impl.io.PigFile; - -import junit.framework.TestCase; - -public class TestEvalPipeline extends TestCase { - - String initString = mapreduce; - - - static public class MyBagFunction extends EvalFuncDataBag{ - @Override - public void exec(Tuple input, DataBag output) throws IOException { - output.add(new Tuple(a)); - output.add(new Tuple(a)); - output.add(new Tuple(a)); - - } - } - - - private File createFile(String[] data) throws Exception{ - File f = File.createTempFile(tmp, ); - PrintWriter pw = new PrintWriter(f); - for (int i=0; idata.length; i++){ - pw.println(data[i]); - } - pw.close(); - return f; - } - - @Test - public void testFunctionInsideFunction() throws Exception{ - PigServer pigServer = new PigServer(initString); - - File f1 = createFile(new String[]{a:1,b:1,a:1}); - - pigServer.registerQuery(a = load 'file: + f1 + ' using + PigStorage.class.getName() + (':');); - pigServer.registerQuery(b = foreach a generate '1'-'1'/'1';); - IteratorTuple iter = pigServer.openIterator(b); - - for (int i=0 ;i3; i++){ - assertEquals(iter.next().getAtomField(0).numval(), 0.0); - } - - } - - @Test - public void testJoin() throws Exception{ - PigServer pigServer = new PigServer(initString); - - File f1 = createFile(new String[]{a:1,b:1,a:1}); - File f2 = createFile(new String[]{b,b,a}); - - pigServer.registerQuery(a = load 'file: + f1 + ' using + PigStorage.class.getName() + (':');); - pigServer.registerQuery(b = load 'file: + f2 + ';); - pigServer.registerQuery(c = cogroup a by $0, b by $0;); - pigServer.registerQuery(d = foreach c generate flatten($1),flatten($2);); - - IteratorTuple iter = pigServer.openIterator(d); - int count = 0; - while(iter.hasNext()){ - Tuple t = iter.next(); - assertTrue(t.getAtomField(0).strval().equals(t.getAtomField(2).strval())); - count++; - } - assertEquals(count, 4); - } - - @Test - public void testDriverMethod() throws Exception{ - PigServer pigServer = new PigServer(initString); - File f = File.createTempFile(tmp, ); - PrintWriter pw = new PrintWriter(f); - pw.println(a); - pw.println(a); - pw.close(); - pigServer.registerQuery(a = foreach (load 'file: + f + ') generate '1', flatten( + MyBagFunction.class.getName() + (*));); - pigServer.registerQuery(b = foreach a generate $0, flatten($1);); - IteratorTuple iter = pigServer.openIterator(a); - int count = 0; - while(iter.hasNext()){ - Tuple t = iter.next(); - assertTrue(t.getAtomField(0).strval().equals(1)); - assertTrue(t.getAtomField(1).strval().equals(a)); - count++; - } - assertEquals(count, 6); - f.delete();
svn commit: r629654 - in /incubator/pig/trunk: CHANGES.txt build.xml
Author: gates Date: Wed Feb 20 15:39:58 2008 New Revision: 629654 URL: http://svn.apache.org/viewvc?rev=629654view=rev Log: PIG-99: Fix to make unit tests not run out of memory. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/build.xml Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=629654r1=629653r2=629654view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Wed Feb 20 15:39:58 2008 @@ -112,3 +112,6 @@ (francisoud via gates). PIG-88: Remove unused HadoopExe import from Main. (pi_song via gates). + + PIG-99: Fix to make unit tests not run out of memory. (francisoud via + gates). Modified: incubator/pig/trunk/build.xml URL: http://svn.apache.org/viewvc/incubator/pig/trunk/build.xml?rev=629654r1=629653r2=629654view=diff == --- incubator/pig/trunk/build.xml (original) +++ incubator/pig/trunk/build.xml Wed Feb 20 15:39:58 2008 @@ -108,6 +108,7 @@ mkdir dir=test/reports/ junit printsummary=on haltonfailure=no errorProperty=tests.failed failureProperty=tests.failed +jvmarg line=-Xmx512m -XX:PermSize=128m -XX:MaxPermSize=512m/ sysproperty key=ssh.gateway value=${ssh.gateway}/ sysproperty key=hod.server value=${hod.server}/ !-- sysproperty key=hod.command value=${hod.command}/
svn commit: r627897 - in /incubator/pig/trunk: ./ lib-src/shock/org/apache/pig/shock/ src/org/apache/pig/ src/org/apache/pig/backend/hadoop/executionengine/ src/org/apache/pig/backend/hadoop/execution
Author: gates Date: Thu Feb 14 14:24:04 2008 New Revision: 627897 URL: http://svn.apache.org/viewvc?rev=627897view=rev Log: PIG-84: Converted printStackTrace calls to calls to the logger. Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/lib-src/shock/org/apache/pig/shock/SSHSocketImplFactory.java incubator/pig/trunk/src/org/apache/pig/Main.java incubator/pig/trunk/src/org/apache/pig/StandAloneParser.java incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HJob.java incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/SplitSpec.java incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigCombine.java incubator/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapreduceExec/PigMapReduce.java incubator/pig/trunk/src/org/apache/pig/backend/local/executionengine/POStore.java incubator/pig/trunk/src/org/apache/pig/impl/PigContext.java incubator/pig/trunk/src/org/apache/pig/impl/builtin/ShellBagEvalFunc.java incubator/pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/schema/Schema.java incubator/pig/trunk/src/org/apache/pig/impl/util/ObjectSerializer.java incubator/pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=627897r1=627896r2=627897view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Thu Feb 14 14:24:04 2008 @@ -107,3 +107,6 @@ exception being caught, and a different exception then thrown. All those locations have been changed so that the new exception now wraps the old. (francisoud via gates). + + PIG-84: Converted printStackTrace calls to calls to the logger. + (francisoud via gates). Modified: incubator/pig/trunk/lib-src/shock/org/apache/pig/shock/SSHSocketImplFactory.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/lib-src/shock/org/apache/pig/shock/SSHSocketImplFactory.java?rev=627897r1=627896r2=627897view=diff == --- incubator/pig/trunk/lib-src/shock/org/apache/pig/shock/SSHSocketImplFactory.java (original) +++ incubator/pig/trunk/lib-src/shock/org/apache/pig/shock/SSHSocketImplFactory.java Thu Feb 14 14:24:04 2008 @@ -401,7 +401,7 @@ log.error(EOF); } } catch (JSchException e) { - e.printStackTrace(); + log.error(e); IOException newE = new IOException(e.getMessage()); newE.setStackTrace(e.getStackTrace()); throw newE; @@ -460,7 +460,7 @@ connectTest(www.yahoo.com); log.error(Finished + this); } catch (Exception e) { - e.printStackTrace(); + log.error(e); } } }.start(); @@ -477,13 +477,13 @@ connectTest(www.flickr.com); log.error(Finished + this); } catch (Exception e) { - e.printStackTrace(); + log.error(e); } } }.start(); } } catch (Exception e) { - e.printStackTrace(); + log.error(e); } } Modified: incubator/pig/trunk/src/org/apache/pig/Main.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/Main.java?rev=627897r1=627896r2=627897view=diff == --- incubator/pig/trunk/src/org/apache/pig/Main.java (original) +++ incubator/pig/trunk/src/org/apache/pig/Main.java Thu Feb 14 14:24:04 2008 @@ -23,6 +23,8 @@ import java.util.jar.*; import java.text.ParseException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.util.HadoopExe; import org.apache.log4j.BasicConfigurator; @@ -42,6 +44,7 @@ public class Main { +private final static Log log = LogFactory.getLog(Main.class
svn commit: r612144 - in /incubator/pig/trunk: CHANGES.txt src/org/apache/pig/data/DataBag.java
Author: gates Date: Tue Jan 15 08:06:56 2008 New Revision: 612144 URL: http://svn.apache.org/viewvc?rev=612144view=rev Log: PIG-56: Made DataBag implement Iterable. (groves via gates) Modified: incubator/pig/trunk/CHANGES.txt incubator/pig/trunk/src/org/apache/pig/data/DataBag.java Modified: incubator/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=612144r1=612143r2=612144view=diff == --- incubator/pig/trunk/CHANGES.txt (original) +++ incubator/pig/trunk/CHANGES.txt Tue Jan 15 08:06:56 2008 @@ -65,3 +65,4 @@ PIG-61: Fixed MapreducePlanCompiler to use PigContext to load up the comparator function instead of Class.forName. (gates) + PIG-56: Made DataBag implement Iterable. (groves via gates) Modified: incubator/pig/trunk/src/org/apache/pig/data/DataBag.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/data/DataBag.java?rev=612144r1=612143r2=612144view=diff == --- incubator/pig/trunk/src/org/apache/pig/data/DataBag.java (original) +++ incubator/pig/trunk/src/org/apache/pig/data/DataBag.java Tue Jan 15 08:06:56 2008 @@ -65,7 +65,7 @@ * DataBag come in several types, default, sorted, and distinct. The type * must be chosen up front, there is no way to convert a bag on the fly. */ -public abstract class DataBag extends Datum implements Spillable { +public abstract class DataBag extends Datum implements Spillable, IterableTuple { // Container that holds the tuples. Actual object instantiated by // subclasses. protected CollectionTuple mContents; @@ -140,8 +140,9 @@ synchronized (mContents) { mMemSizeChanged = true; mSize += b.size(); -IteratorTuple i = b.iterator(); -while (i.hasNext()) mContents.add(i.next()); +for (Tuple t : b) { + mContents.add(t); +} } } @@ -278,10 +279,8 @@ // time re-sorting or re-applying distinct. out.write(BAG); out.writeLong(size()); -IteratorTuple it = iterator(); -while (it.hasNext()) { -Tuple item = it.next(); -item.write(out); +for (Tuple t : this) { +t.write(out); } }
svn commit: r610047 - in /incubator/pig/trunk/src/org/apache/pig/data: DataBag.java DefaultDataBag.java DistinctDataBag.java SortedDataBag.java
Author: gates Date: Tue Jan 8 08:20:45 2008 New Revision: 610047 URL: http://svn.apache.org/viewvc?rev=610047view=rev Log: PIG-30 Added hashCode() implementation to DataBag and added additional comments throughout DataBag and extenders implementation. Modified: incubator/pig/trunk/src/org/apache/pig/data/DataBag.java incubator/pig/trunk/src/org/apache/pig/data/DefaultDataBag.java incubator/pig/trunk/src/org/apache/pig/data/DistinctDataBag.java incubator/pig/trunk/src/org/apache/pig/data/SortedDataBag.java Modified: incubator/pig/trunk/src/org/apache/pig/data/DataBag.java URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/data/DataBag.java?rev=610047r1=610046r2=610047view=diff == --- incubator/pig/trunk/src/org/apache/pig/data/DataBag.java (original) +++ incubator/pig/trunk/src/org/apache/pig/data/DataBag.java Tue Jan 8 08:20:45 2008 @@ -32,7 +32,38 @@ import org.apache.pig.impl.mapreduceExec.PigMapReduce; /** - * A collection of Tuples + * A collection of Tuples. A DataBag may or may not fit into memory. + * DataBag extends spillable, which means that it registers with a memory + * manager. By default, it attempts to keep all of its contents in memory. + * If it is asked by the memory manager to spill to disk (by a call to + * spill()), it takes whatever it has in memory, opens a spill file, and + * writes the contents out. This may happen multiple times. The bag + * tracks all of the files it's spilled to. + * + * DataBag provides an Iterator interface, that allows callers to read + * through the contents. The iterators are aware of the data spilling. + * They have to be able to handle reading from files, as well as the fact + * that data they were reading from memory may have been spilled to disk + * underneath them. + * + * The DataBag interface assumes that all data is written before any is + * read. That is, a DataBag cannot be used as a queue. If data is written + * after data is read, the results are undefined. This condition is not + * checked on each add or read, for reasons of speed. Caveat emptor. + * + * Since spills are asynchronous (the memory manager requesting a spill + * runs in a separate thread), all operations dealing with the mContents + * Collection (which is the collection of tuples contained in the bag) have + * to be synchronized. This means that reading from a DataBag is currently + * serialized. This is ok for the moment because pig execution is + * currently single threaded. A ReadWriteLock was experimented with, but + * it was found to be about 10x slower than using the synchronize keyword. + * If pig changes its execution model to be multithreaded, we may need to + * return to this issue, as synchronizing reads will most likely defeat the + * purpose of multi-threading execution. + * + * DataBag come in several types, default, sorted, and distinct. The type + * must be chosen up front, there is no way to convert a bag on the fly. */ public abstract class DataBag extends Datum implements Spillable { // Container that holds the tuples. Actual object instantiated by @@ -170,6 +201,10 @@ } } +/** + * This method is potentially very expensive since it may require a + * sort of the bag; don't call it unless you have to. + */ public int compareTo(Object other) { // Do we really need to be able to compare to DataAtom and Tuple? // When does that happen? @@ -182,9 +217,31 @@ else return -1; } -// Don't sort them, just go tuple by tuple. -IteratorTuple thisIt = this.iterator(); -IteratorTuple otherIt = bOther.iterator(); +// Ugh, this is bogus. But I have to know if two bags have the +// same tuples, regardless of order. Hopefully most of the +// time the size check above will prevent this. +// If either bag isn't already sorted, create a sorted bag out +// of it so I can guarantee order. +DataBag thisClone; +DataBag otherClone; +if (this instanceof SortedDataBag || +this instanceof DistinctDataBag) { +thisClone = this; +} else { +thisClone = new SortedDataBag(null); +IteratorTuple i = iterator(); +while (i.hasNext()) thisClone.add(i.next()); +} +if (other instanceof SortedDataBag || +this instanceof DistinctDataBag) { +otherClone = bOther; +} else { +otherClone = new SortedDataBag(null); +IteratorTuple i = bOther.iterator(); +while (i.hasNext()) otherClone.add(i.next()); +} +IteratorTuple thisIt = thisClone.iterator(); +IteratorTuple otherIt = otherClone.iterator