svn commit: r966325 - in /hadoop/pig/trunk: CHANGES.txt src/org/apache/pig/data/DefaultAbstractBag.java src/org/apache/pig/data/DefaultTuple.java test/org/apache/pig/test/TestTuple.java
Author: thejas Date: Wed Jul 21 17:33:42 2010 New Revision: 966325 URL: http://svn.apache.org/viewvc?rev=966325view=rev Log: PIG-1492 : DefaultTuple and DefaultMemory understimate their memory footprint Modified: hadoop/pig/trunk/CHANGES.txt hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java hadoop/pig/trunk/test/org/apache/pig/test/TestTuple.java Modified: hadoop/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=966325r1=966324r2=966325view=diff == --- hadoop/pig/trunk/CHANGES.txt (original) +++ hadoop/pig/trunk/CHANGES.txt Wed Jul 21 17:33:42 2010 @@ -100,6 +100,8 @@ PIG-1309: Map-side Cogroup (ashutoshc) BUG FIXES +PIG-1492: DefaultTuple and DefaultMemory understimate their memory footprint (thejas) + PIG-1409: Fix up javadocs for org.apache.pig.builtin (gates) PIG-1490: Make Pig storers work with remote HDFS in secure mode (rding) Modified: hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java?rev=966325r1=966324r2=966325view=diff == --- hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java Wed Jul 21 17:33:42 2010 @@ -140,20 +140,49 @@ public abstract class DefaultAbstractBag used *= numInMem; } -// add up the overhead for this object, mContents object, references to tuples, -// and other object variables -used += 12 + 12 + numInMem*4 + 8 + 4 + 8; +// add up the overhead for this object and other object variables +int bag_fix_size = 8 /* object header */ ++ 4 + 8 + 8 /* mLastContentsSize + mMemSize + mSize */ ++ 8 + 8 /* mContents ref + mSpillFiles ref*/ ++ 4 /* +4 to round it to eight*/ ++ 36 /* mContents fixed */ +; +long mFields_size = roundToEight(4 + numInMem*4); /* mContents fixed + per entry */ +// in java hotspot 32bit vm, there seems to be a minimum bag size of 188 bytes +// some of the extra bytes is probably from a minimum size of this array list +mFields_size = Math.max(40, mFields_size); +used += bag_fix_size + mFields_size; + // add up overhead for mSpillFiles ArrayList, Object[] inside ArrayList, // object variable inside ArrayList and references to spill files if (mSpillFiles != null) { - used += 12 + 12 + 4 + mSpillFiles.size()*4; +used += roundToEight(36 /* mSpillFiles fixed overhead*/ + mSpillFiles.size()*4); + +if(mSpillFiles.size() 0){ +//a rough estimate of memory used by each file entry +// the auto generated files are likely to have same length +long approx_per_entry_size = +roundToEight(mSpillFiles.get(0).toString().length() * 2 + 38); + +used += mSpillFiles.size() * approx_per_entry_size; +} } mMemSize = used; return used; } + +/** + * Memory size of objects are rounded to multiple of 8 bytes + * @param i + * @return i rounded to a equal of higher multiple of 8 + */ +private long roundToEight(long i) { +return 8 * ((i+7)/8); // integer division rounds the result down +} + /** * Clear out the contents of the bag, both on disk and in memory. * Any attempts to read after this is called will produce undefined Modified: hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java?rev=966325r1=966324r2=966325view=diff == --- hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java Wed Jul 21 17:33:42 2010 @@ -31,17 +31,15 @@ import org.apache.pig.backend.executione import org.apache.pig.impl.util.TupleFormat; /** - * This was the old default implementation of Tuple. The new default is - * {...@link BinSedesTuple} . - * Zebra and BinStorage load/store functions use the .write(..) and .readFields(..) - * functions here for (de)serialization. + * A default implementation of Tuple. This class will be created by the + * DefaultTupleFactory. */ public class DefaultTuple implements Tuple { protected boolean isNull = false; private static final long serialVersionUID = 2L; protected ListObject mFields; - + /** * Default constructor. This constructor
svn commit: r966326 - in /hadoop/pig/trunk: ./ src/org/apache/pig/ src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/ src/org/apache/pig/tools/pigstats/ test/org/apache/pig/test/
Author: rding Date: Wed Jul 21 17:39:14 2010 New Revision: 966326 URL: http://svn.apache.org/viewvc?rev=966326view=rev Log: PIG-1478: Add progress notification listener to PigRunner API Added: hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/PigProgressNotificationListener.java Modified: hadoop/pig/trunk/CHANGES.txt hadoop/pig/trunk/src/org/apache/pig/Main.java hadoop/pig/trunk/src/org/apache/pig/PigRunner.java hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/JobStats.java hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/PigStatsUtil.java hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/ScriptState.java hadoop/pig/trunk/test/org/apache/pig/test/TestPigRunner.java Modified: hadoop/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=966326r1=966325r2=966326view=diff == --- hadoop/pig/trunk/CHANGES.txt (original) +++ hadoop/pig/trunk/CHANGES.txt Wed Jul 21 17:39:14 2010 @@ -24,6 +24,8 @@ INCOMPATIBLE CHANGES IMPROVEMENTS +PIG-1478: Add progress notification listener to PigRunner API (rding) + PIG-1472: Optimize serialization/deserialization between Map and Reduce and between MR jobs (thejas) PIG-1389: Implement Pig counter to track number of rows for each input files Modified: hadoop/pig/trunk/src/org/apache/pig/Main.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/Main.java?rev=966326r1=966325r2=966326view=diff == --- hadoop/pig/trunk/src/org/apache/pig/Main.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/Main.java Wed Jul 21 17:39:14 2010 @@ -62,6 +62,7 @@ import org.apache.pig.impl.util.JarManag import org.apache.pig.impl.util.ObjectSerializer; import org.apache.pig.impl.util.PropertiesUtil; import org.apache.pig.impl.util.UDFContext; +import org.apache.pig.tools.pigstats.PigProgressNotificationListener; import org.apache.pig.tools.pigstats.PigStatsUtil; import org.apache.pig.tools.pigstats.ScriptState; import org.apache.pig.tools.cmdline.CmdLineParser; @@ -100,10 +101,10 @@ public class Main { public static void main(String args[]) { GenericOptionsParser parser = new GenericOptionsParser(args); String[] pigArgs = parser.getRemainingArgs(); -System.exit(run(pigArgs)); +System.exit(run(pigArgs, null)); } -static int run(String args[]) { +static int run(String args[], PigProgressNotificationListener listener) { int rc = 1; Properties properties = new Properties(); PropertiesUtil.loadDefaultProperties(properties); @@ -287,6 +288,10 @@ static int run(String args[]) { // create the static script state object String commandLine = LoadFunc.join((AbstractListString)Arrays.asList(args), ); ScriptState scriptState = ScriptState.start(commandLine); +if (listener != null) { +scriptState.registerListener(listener); +} + if(logFileName == null !userSpecifiedLog) { logFileName = validateLogFile(properties.getProperty(pig.logfile), null); Modified: hadoop/pig/trunk/src/org/apache/pig/PigRunner.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/PigRunner.java?rev=966326r1=966325r2=966326view=diff == --- hadoop/pig/trunk/src/org/apache/pig/PigRunner.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/PigRunner.java Wed Jul 21 17:39:14 2010 @@ -20,6 +20,7 @@ package org.apache.pig; import org.apache.hadoop.util.GenericOptionsParser; import org.apache.pig.classification.InterfaceAudience; import org.apache.pig.classification.InterfaceStability; +import org.apache.pig.tools.pigstats.PigProgressNotificationListener; import org.apache.pig.tools.pigstats.PigStats; import org.apache.pig.tools.pigstats.PigStatsUtil; @@ -45,10 +46,10 @@ public abstract class PigRunner { public final static int THROWABLE_EXCEPTION = 8; } -public static PigStats run(String[] args) { +public static PigStats run(String[] args, PigProgressNotificationListener listener) { GenericOptionsParser parser = new GenericOptionsParser(args); String[] pigArgs = parser.getRemainingArgs(); -return PigStatsUtil.getPigStats(Main.run(pigArgs)); +return PigStatsUtil.getPigStats(Main.run(pigArgs, listener)); } } Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java?rev=966326r1=966325r2=966326view=diff
svn commit: r966413 - in /hadoop/pig/trunk: .gitignore CHANGES.txt
Author: gates Date: Wed Jul 21 20:49:03 2010 New Revision: 966413 URL: http://svn.apache.org/viewvc?rev=966413view=rev Log: PIG-1509: Add .gitignore file Added: hadoop/pig/trunk/.gitignore Modified: hadoop/pig/trunk/CHANGES.txt Added: hadoop/pig/trunk/.gitignore URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/.gitignore?rev=966413view=auto == --- hadoop/pig/trunk/.gitignore (added) +++ hadoop/pig/trunk/.gitignore Wed Jul 21 20:49:03 2010 @@ -0,0 +1,7 @@ +*~ +build/ +src-gen/ +test/org/apache/pig/test/utils/dotGraph/parser/ +ivy/*.jar +pig.jar + Modified: hadoop/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=966413r1=966412r2=966413view=diff == --- hadoop/pig/trunk/CHANGES.txt (original) +++ hadoop/pig/trunk/CHANGES.txt Wed Jul 21 20:49:03 2010 @@ -24,6 +24,8 @@ INCOMPATIBLE CHANGES IMPROVEMENTS +PIG-1509: Add .gitignore file (cwsteinbach via gates) + PIG-1478: Add progress notification listener to PigRunner API (rding) PIG-1472: Optimize serialization/deserialization between Map and Reduce and between MR jobs (thejas)
svn commit: r966470 - in /hadoop/pig/branches/branch-0.7: CHANGES.txt src/org/apache/pig/impl/logicalLayer/optimizer/PushUpFilter.java test/org/apache/pig/test/TestPushUpFilter.java
Author: daijy Date: Thu Jul 22 01:10:42 2010 New Revision: 966470 URL: http://svn.apache.org/viewvc?rev=966470view=rev Log: PIG-1507: Full outer join fails while doing a filter on joined data Modified: hadoop/pig/branches/branch-0.7/CHANGES.txt hadoop/pig/branches/branch-0.7/src/org/apache/pig/impl/logicalLayer/optimizer/PushUpFilter.java hadoop/pig/branches/branch-0.7/test/org/apache/pig/test/TestPushUpFilter.java Modified: hadoop/pig/branches/branch-0.7/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.7/CHANGES.txt?rev=966470r1=966469r2=966470view=diff == --- hadoop/pig/branches/branch-0.7/CHANGES.txt (original) +++ hadoop/pig/branches/branch-0.7/CHANGES.txt Thu Jul 22 01:10:42 2010 @@ -198,6 +198,8 @@ OPTIMIZATIONS BUG FIXES +PIG-1507: Full outer join fails while doing a filter on joined data (daijy) + PIG-1493: Column Pruner throw exception inconsistent pruning (daijy) PIG-1490: Make Pig storers work with remote HDFS in secure mode (rding) Modified: hadoop/pig/branches/branch-0.7/src/org/apache/pig/impl/logicalLayer/optimizer/PushUpFilter.java URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.7/src/org/apache/pig/impl/logicalLayer/optimizer/PushUpFilter.java?rev=966470r1=966469r2=966470view=diff == --- hadoop/pig/branches/branch-0.7/src/org/apache/pig/impl/logicalLayer/optimizer/PushUpFilter.java (original) +++ hadoop/pig/branches/branch-0.7/src/org/apache/pig/impl/logicalLayer/optimizer/PushUpFilter.java Thu Jul 22 01:10:42 2010 @@ -244,6 +244,7 @@ public class PushUpFilter extends Logica // Also in LOJoin, innerFlag==true indicate that branch is the outer join side // which has the exact opposite semantics // If all innerFlag is true, that implies a regular join +// If all innerFlag is false, means a outer join, in this case, we can not push up filter for any path (See PIG-1507) if (i!=mPushBeforeInput ((LOJoin)predecessor).getInnerFlags()[i]) { otherBranchContainOuter = true; } @@ -251,7 +252,13 @@ public class PushUpFilter extends Logica sawInner = true; } } -if (otherBranchContainOuter sawInner) { +if (!otherBranchContainOuter ((LOJoin)predecessor).getInnerFlags()[mPushBeforeInput]==false) // all innerFlag is false, implies an outer join +{ +mPushBeforeInput = -1; +return false; +} +if (otherBranchContainOuter sawInner) // If it is not a regular join and the path we push is on inner side +{ mPushBeforeInput = -1; return false; } Modified: hadoop/pig/branches/branch-0.7/test/org/apache/pig/test/TestPushUpFilter.java URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.7/test/org/apache/pig/test/TestPushUpFilter.java?rev=966470r1=966469r2=966470view=diff == --- hadoop/pig/branches/branch-0.7/test/org/apache/pig/test/TestPushUpFilter.java (original) +++ hadoop/pig/branches/branch-0.7/test/org/apache/pig/test/TestPushUpFilter.java Thu Jul 22 01:10:42 2010 @@ -1069,6 +1069,7 @@ public class TestPushUpFilter extends ju assertTrue(pushUpFilter.getPushBeforeInput() == -1); } +// See PIG-1289 @Test public void testOutJoin() throws Exception { planTester.buildPlan(A = load 'myfile' as (name, age, gpa);); @@ -1086,6 +1087,25 @@ public class TestPushUpFilter extends ju assertTrue(pushUpFilter.getPushBefore() == false); assertTrue(pushUpFilter.getPushBeforeInput() == -1); } + +// See PIG-1507 +@Test +public void testFullOutJoin() throws Exception { +planTester.buildPlan(A = load 'myfile' as (d1:int);); +planTester.buildPlan(B = load 'anotherfile' as (d2:int);); +planTester.buildPlan(c = join A by d1 full outer, B by d2;); +LogicalPlan lp = planTester.buildPlan(d = filter c by d2 is null;); + +planTester.setPlan(lp); +planTester.setProjectionMap(lp); + +PushUpFilter pushUpFilter = new PushUpFilter(lp); + +assertTrue(!pushUpFilter.check(lp.getLeaves())); +assertTrue(pushUpFilter.getSwap() == false); +assertTrue(pushUpFilter.getPushBefore() == false); +assertTrue(pushUpFilter.getPushBeforeInput() == -1); +} }
svn commit: r966471 - in /hadoop/pig/trunk: CHANGES.txt src/org/apache/pig/impl/logicalLayer/optimizer/PushUpFilter.java test/org/apache/pig/test/TestPushUpFilter.java
Author: daijy Date: Thu Jul 22 01:13:13 2010 New Revision: 966471 URL: http://svn.apache.org/viewvc?rev=966471view=rev Log: PIG-1507: Full outer join fails while doing a filter on joined data Modified: hadoop/pig/trunk/CHANGES.txt hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PushUpFilter.java hadoop/pig/trunk/test/org/apache/pig/test/TestPushUpFilter.java Modified: hadoop/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=966471r1=966470r2=966471view=diff == --- hadoop/pig/trunk/CHANGES.txt (original) +++ hadoop/pig/trunk/CHANGES.txt Thu Jul 22 01:13:13 2010 @@ -347,6 +347,8 @@ OPTIMIZATIONS BUG FIXES +PIG-1507: Full outer join fails while doing a filter on joined data (daijy) + PIG-1493: Column Pruner throw exception inconsistent pruning (daijy) PIG-1484: BinStorage should support comma seperated path (daijy) Modified: hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PushUpFilter.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PushUpFilter.java?rev=966471r1=966470r2=966471view=diff == --- hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PushUpFilter.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PushUpFilter.java Thu Jul 22 01:13:13 2010 @@ -244,6 +244,7 @@ public class PushUpFilter extends Logica // Also in LOJoin, innerFlag==true indicate that branch is the outer join side // which has the exact opposite semantics // If all innerFlag is true, that implies a regular join +// If all innerFlag is false, means a outer join, in this case, we can not push up filter for any path (See PIG-1507) if (i!=mPushBeforeInput ((LOJoin)predecessor).getInnerFlags()[i]) { otherBranchContainOuter = true; } @@ -251,7 +252,13 @@ public class PushUpFilter extends Logica sawInner = true; } } -if (otherBranchContainOuter sawInner) { +if (!otherBranchContainOuter ((LOJoin)predecessor).getInnerFlags()[mPushBeforeInput]==false) // all innerFlag is false, implies an outer join +{ +mPushBeforeInput = -1; +return false; +} +if (otherBranchContainOuter sawInner) // If it is not a regular join and the path we push is on inner side +{ mPushBeforeInput = -1; return false; } Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestPushUpFilter.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestPushUpFilter.java?rev=966471r1=966470r2=966471view=diff == --- hadoop/pig/trunk/test/org/apache/pig/test/TestPushUpFilter.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestPushUpFilter.java Thu Jul 22 01:13:13 2010 @@ -1074,6 +1074,7 @@ public class TestPushUpFilter extends ju assertTrue(pushUpFilter.getPushBeforeInput() == -1); } +// See PIG-1289 @Test public void testOutJoin() throws Exception { planTester.buildPlan(A = load 'myfile' as (name, age, gpa);); @@ -1091,6 +1092,25 @@ public class TestPushUpFilter extends ju assertTrue(pushUpFilter.getPushBefore() == false); assertTrue(pushUpFilter.getPushBeforeInput() == -1); } + +// See PIG-1507 +@Test +public void testFullOutJoin() throws Exception { +planTester.buildPlan(A = load 'myfile' as (d1:int);); +planTester.buildPlan(B = load 'anotherfile' as (d2:int);); +planTester.buildPlan(c = join A by d1 full outer, B by d2;); +LogicalPlan lp = planTester.buildPlan(d = filter c by d2 is null;); + +planTester.setPlan(lp); +planTester.setProjectionMap(lp); + +PushUpFilter pushUpFilter = new PushUpFilter(lp); + +assertTrue(!pushUpFilter.check(lp.getLeaves())); +assertTrue(pushUpFilter.getSwap() == false); +assertTrue(pushUpFilter.getPushBefore() == false); +assertTrue(pushUpFilter.getPushBeforeInput() == -1); +} }