[jira] Created: (PIG-1835) Pig 0.9 new logical plan throws class cast exception
Pig 0.9 new logical plan throws class cast exception Key: PIG-1835 URL: https://issues.apache.org/jira/browse/PIG-1835 Project: Pig Issue Type: Bug Affects Versions: 0.9.0 Reporter: Vivek Padmanabhan I have the below script which is throwing class cast exception while doing SUM. Even though all the fields are properly typed, while computing sum in m_agg0 and m_agg02 the record from tuple is coming as java.lang.Long instead of Double. The problem is happening in Pig 0.9. It works fine with 0.9 if I flag off new logical plan by -Dpig.usenewlogicalplan=false. {code} A0 = load 'inputA' using PigStorage('\t') as ( group_id, r_id:long, is_phase2:int, roi_value:double,roi_cost:double,ecpm, prob:double,pixel_id, pixel_type, val:long,f3, f4,type:long, amount:double,item_id:long); A0 = foreach A0 generate r_id, is_phase2, ((val==257 or val==258)? 1: 0) as imps, ((val==257 or val==258)? amount: 0.0) as a_out, ((val==257 or val==258)? item_id: 0) as a_item_id, ((val==257 or val==258)? roi_value: 0.0) as roi_value,((val==257 or val==258)? roi_cost: 0.0) as roi_cost, ((val==257 or val==513)? ecpm: 0.0) as ecpm, ((val==257 or val==513)? prob: 0.0) as prob, ((val==257 or val==513)? amount: 0.0) as pub_rev, ((val==257 or val==513)? item_id: 0) as pub_line_id,((val==257 or val==513)? type: 0) as pub_pt; - B0 = load 'inputB' using PigStorage('\t') as ( group_id:long, r_id:long, roi_value:double,roi_cost:double,receive_time, host_name,site_id,rm_has_cookies,rm_pearl_id, f1,f2,pixel_id:long,pixel_type:int, xcookie,val:long,f3, f4,type:long,amount:double,item_id:long); B0 = foreach B0 generate r_id, ((val==257 or val==258)? 1: 0) as B0,((val==257 or val==258)? amount: 0.0) as a_out, ((val==257 or val==258)? item_id: 0) as a_item_id,((val==257 or val==258)? roi_value: 0.0) as roi_value, ((val==257 or val==258)? roi_cost: 0.0) as roi_cost, ((val==257 or val==513)? amount: 0.0) as pub_rev, ((val==257 or val==513)? item_id: 0) as pub_line_id, ((val==257 or val==513)? type: 0) as pub_pt; C0 = load 'inputC' using PigStorage('\t') as ( group_id:long, r_id:long, roi_value:double, roi_cost:double, receive_time:long, host_name:chararray, site_id:long, rm_has_cookies:int,rm_pearl_id:long,f1,f2, pixel_id:long, pixel_type:int,rm_is_post_click:int, rm_conversion_id,xcookie:chararray,val:long,f3:long,f4:long,type:long,amount:double,item_id:long); C0 = foreach C0 generate r_id,((val==257 or val==258)? 1: 0) as C0, ((val==257 or val==258)? amount: 0.0) as a_out, ((val==257 or val==258)? item_id: 0) as a_item_id,((val==257 or val==513)? amount: 0.0) as pub_rev, ((val==257 or val==513)? item_id: 0) as pub_line_id, ((val==257 or val==513)? type: 0) as pub_pt; m_all = cogroup A0 by (r_id) outer, B0 by (r_id) outer, C0 by (r_id) outer ; m_agg01 = foreach m_all generate (double)(IsEmpty(C0) ? 0.0 : SUM(C0.pub_rev)) as conv_pub_rev; store m_agg01 into 'out1' USING PigStorage(','); m_all = cogroup A0 by (r_id) outer, B0 by (r_id) outer, C0 by (r_id) outer ; m_agg02 = foreach m_all generate (double)(IsEmpty(C0) ? 0.0 : SUM(C0.pub_rev)) as conv_pub_rev; store m_agg02 into 'out2' USING PigStorage(','); {code} The below are the inputs to the script (all single record and tab seperated) inputA -- 1 1.1 1.1 1.1 1.1 1 1.1 inputB -- 1.1 1.1 a1 1 b1 1 1 c1 1.1 inputC -- 1.1 1.1 a1 1 b1 1 1 1 c1 1.1 Exception from reducers ___ org.apache.pig.backend.executionengine.ExecException: ERROR 2103: Problem while computing sum of doubles. at org.apache.pig.builtin.DoubleSum.sum(DoubleSum.java:147) at org.apache.pig.builtin.DoubleSum.exec(DoubleSum.java:46) at org.apache.pig.builtin.DoubleSum.exec(DoubleSum.java:41) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc.getNext(POUserFunc.java:230) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc.getNext(POUserFunc.java:302)
[jira] Updated: (PIG-1717) pig needs to call setPartitionFilter if schema is null but getPartitionKeys is not
[ https://issues.apache.org/jira/browse/PIG-1717?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gerrit Jansen van Vuuren updated PIG-1717: -- Tags: PIG-1717.v2.patch (was: PIG-1717.v1.patch) Status: Patch Available (was: Open) Hi, finally I got to do this. What I did was add in a check in the LOLoad for conf == null, this will allow any current tests that pass the conf as null to succeed. pig needs to call setPartitionFilter if schema is null but getPartitionKeys is not -- Key: PIG-1717 URL: https://issues.apache.org/jira/browse/PIG-1717 Project: Pig Issue Type: Improvement Components: impl Affects Versions: 0.9.0 Reporter: Gerrit Jansen van Vuuren Assignee: Gerrit Jansen van Vuuren Priority: Minor Fix For: 0.9.0 Attachments: PIG-1717.patch, PIG-1717.v1.patch, PIG-1717.v2.patch, patchReleaseAuditWarnings.txt.gz, testlog.tgz, trunkReleaseAuditWarnings.txt.gz I'm writing a loader that works with hive style partitioning e.g. /logs/type1/daydate=2010-11-01 The loader does not know the schema upfront and this is something that the user adds in the script using the AS clause. The problem is that this user defined schema is not available to the loader, so the loader cannot return any schema, the Loader does know what the partition keys are and pig needs in some way to know about these partition keys. Currently if the schema is null pig never calls the LoadMetaData:getPartitionKeys method or the setPartitionFilter method. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (PIG-240) Support launching concurrent Pig jobs from one VM
[ https://issues.apache.org/jira/browse/PIG-240?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12989230#comment-12989230 ] Thomas Memenga commented on PIG-240: Any news on this? We are trying to trigger a lot of pig-jobs simultaneously from a single jvm and we are recieving Unable to find clone for op errors from time to time (pig 0.8.0). It's a pitty PigServer isn't threadsafe, spawning jvms for each script seems to be our only option. Support launching concurrent Pig jobs from one VM - Key: PIG-240 URL: https://issues.apache.org/jira/browse/PIG-240 Project: Pig Issue Type: Improvement Components: impl Reporter: Tom White Assignee: Jeff Zhang Attachments: patch_240.txt, pig-240-1.patch, pig-240-2.patch, pig-240.patch For some applications it would be convenient to launch concurrent Pig jobs from a single VM. This is currently not possible since Pig has static mutable state. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (PIG-1618) Switch to new parser generator technology
[ https://issues.apache.org/jira/browse/PIG-1618?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12989280#comment-12989280 ] Xuefu Zhang commented on PIG-1618: -- Test-patch run result for patch NewParser-15.patch: (release warning is due to new files added.) [exec] -1 overall. [exec] [exec] +1 @author. The patch does not contain any @author tags. [exec] [exec] +1 tests included. The patch appears to include 6 new or modified tests. [exec] [exec] +1 javadoc. The javadoc tool did not generate any warning messages. [exec] [exec] +1 javac. The applied patch does not increase the total number of javac compiler warnings. [exec] [exec] +1 findbugs. The patch does not introduce any new Findbugs warnings. [exec] [exec] -1 release audit. The applied patch generated 499 release audit warnings (more than the trunk's current 496 warnings). Ant test passed. Switch to new parser generator technology - Key: PIG-1618 URL: https://issues.apache.org/jira/browse/PIG-1618 Project: Pig Issue Type: Improvement Affects Versions: 0.8.0 Reporter: Alan Gates Assignee: Xuefu Zhang Fix For: 0.9.0 Attachments: NewParser-1.patch, NewParser-10.patch, NewParser-11.patch, NewParser-12.patch, NewParser-13.2.patch, NewParser-13.patch, NewParser-14.patch, NewParser-15.patch, NewParser-2.patch, NewParser-3.patch, NewParser-3.patch, NewParser-4.patch, NewParser-5.patch, NewParser-6.patch, NewParser-7.patch, NewParser-8.patches, NewParser-9.patch, antlr-3.2.jar, javadoc.patch There are many bugs in Pig related to the parser, particularly to bad error messages. After review of Java CC we feel these will be difficult to address using that tool. Also, the .jjt files used by JavaCC are hard to understand and maintain. ANTLR is being reviewed as the most likely choice to move to, but other parsers will be reviewed as well. This JIRA will act as an umbrella issue for other parser issues. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (PIG-240) Support launching concurrent Pig jobs from one VM
[ https://issues.apache.org/jira/browse/PIG-240?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12989305#comment-12989305 ] Thomas Memenga commented on PIG-240: Additional Note: Applied latest patch to 0.8.0 (release), but executing the following script results in nullpointerexception: {code} data = load 'hdfs://host.local:9000/user/hadoop/input/part-r-0' using PigStorage() as (thed,ts:int,cId,ai,si); input_filtered = foreach data generate ts, ai, si; pipe_4965 = filter input_filtered by ts 1288043999 and ts 1288047599; pipe_4965_grouped = group pipe_4965 by (ai, si); pipe_4965_flat = foreach pipe_4965_grouped generate FLATTEN($0) , COUNT($1) , MIN(pipe_4965.ts) as ts_min, MAX(pipe_4965.ts) as ts_max; store pipe_4965_flat INTO '/user/hadoop/output/4965' USING PigStorage(); {code} {code} org.apache.pig.impl.logicalLayer.FrontendException: ERROR 2042: Error in new logical plan. Try -Dpig.usenewlogicalplan=false. at org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.compile(HExecutionEngine.java:309) at org.apache.pig.PigServer.compilePp(PigServer.java:1354) at org.apache.pig.PigServer.executeCompiledLogicalPlan(PigServer.java:1196) at org.apache.pig.PigServer.execute(PigServer.java:1190) at org.apache.pig.PigServer.access$100(PigServer.java:128) at org.apache.pig.PigServer$Graph.execute(PigServer.java:1517) at org.apache.pig.PigServer.executeBatchEx(PigServer.java:362) at org.apache.pig.PigServer.executeBatch(PigServer.java:329) at org.apache.pig.tools.grunt.GruntParser.executeBatch(GruntParser.java:112) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:169) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:141) at org.apache.pig.tools.grunt.Grunt.exec(Grunt.java:90) at org.apache.pig.Main.run(Main.java:510) at org.apache.pig.Main.main(Main.java:107) Caused by: java.lang.NullPointerException at org.apache.pig.newplan.ReverseDependencyOrderWalker.walk(ReverseDependencyOrderWalker.java:70) at org.apache.pig.newplan.PlanVisitor.visit(PlanVisitor.java:50) at org.apache.pig.newplan.logical.optimizer.SchemaResetter.visit(SchemaResetter.java:105) at org.apache.pig.newplan.logical.relational.LOGenerate.accept(LOGenerate.java:229) at org.apache.pig.newplan.DependencyOrderWalker.walk(DependencyOrderWalker.java:75) at org.apache.pig.newplan.logical.optimizer.SchemaResetter.visit(SchemaResetter.java:94) at org.apache.pig.newplan.logical.relational.LOForEach.accept(LOForEach.java:71) at org.apache.pig.newplan.DependencyOrderWalker.walk(DependencyOrderWalker.java:75) at org.apache.pig.newplan.PlanVisitor.visit(PlanVisitor.java:50) at org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.compile(HExecutionEngine.java:261) ... 13 more {code} Script works fine with unpatched 0.8.0 Support launching concurrent Pig jobs from one VM - Key: PIG-240 URL: https://issues.apache.org/jira/browse/PIG-240 Project: Pig Issue Type: Improvement Components: impl Reporter: Tom White Assignee: Jeff Zhang Attachments: patch_240.txt, pig-240-1.patch, pig-240-2.patch, pig-240.patch For some applications it would be convenient to launch concurrent Pig jobs from a single VM. This is currently not possible since Pig has static mutable state. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (PIG-1717) pig needs to call setPartitionFilter if schema is null but getPartitionKeys is not
[ https://issues.apache.org/jira/browse/PIG-1717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12989327#comment-12989327 ] Alan Gates commented on PIG-1717: - Re-running test-patch and unit tests. pig needs to call setPartitionFilter if schema is null but getPartitionKeys is not -- Key: PIG-1717 URL: https://issues.apache.org/jira/browse/PIG-1717 Project: Pig Issue Type: Improvement Components: impl Affects Versions: 0.9.0 Reporter: Gerrit Jansen van Vuuren Assignee: Gerrit Jansen van Vuuren Priority: Minor Fix For: 0.9.0 Attachments: PIG-1717.patch, PIG-1717.v1.patch, PIG-1717.v2.patch, patchReleaseAuditWarnings.txt.gz, testlog.tgz, trunkReleaseAuditWarnings.txt.gz I'm writing a loader that works with hive style partitioning e.g. /logs/type1/daydate=2010-11-01 The loader does not know the schema upfront and this is something that the user adds in the script using the AS clause. The problem is that this user defined schema is not available to the loader, so the loader cannot return any schema, the Loader does know what the partition keys are and pig needs in some way to know about these partition keys. Currently if the schema is null pig never calls the LoadMetaData:getPartitionKeys method or the setPartitionFilter method. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (PIG-1717) pig needs to call setPartitionFilter if schema is null but getPartitionKeys is not
[ https://issues.apache.org/jira/browse/PIG-1717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12989329#comment-12989329 ] Alan Gates commented on PIG-1717: - [exec] -1 overall. [exec] [exec] +1 @author. The patch does not contain any @author tags. [exec] [exec] +1 tests included. The patch appears to include 7 new or modified tests. [exec] [exec] +1 javadoc. The javadoc tool did not generate any warning messages. [exec] [exec] +1 javac. The applied patch does not increase the total number of javac compiler warnings. [exec] [exec] +1 findbugs. The patch does not introduce any new Findbugs warnings. [exec] [exec] -1 release audit. The applied patch generated 509 release audit warnings (more than the trunk's current 507 warnings). [exec] As far as I can tell the release audit complaints are just because two new files got added which made changes in the jdiff output, so we can ignore those. pig needs to call setPartitionFilter if schema is null but getPartitionKeys is not -- Key: PIG-1717 URL: https://issues.apache.org/jira/browse/PIG-1717 Project: Pig Issue Type: Improvement Components: impl Affects Versions: 0.9.0 Reporter: Gerrit Jansen van Vuuren Assignee: Gerrit Jansen van Vuuren Priority: Minor Fix For: 0.9.0 Attachments: PIG-1717.patch, PIG-1717.v1.patch, PIG-1717.v2.patch, patchReleaseAuditWarnings.txt.gz, testlog.tgz, trunkReleaseAuditWarnings.txt.gz I'm writing a loader that works with hive style partitioning e.g. /logs/type1/daydate=2010-11-01 The loader does not know the schema upfront and this is something that the user adds in the script using the AS clause. The problem is that this user defined schema is not available to the loader, so the loader cannot return any schema, the Loader does know what the partition keys are and pig needs in some way to know about these partition keys. Currently if the schema is null pig never calls the LoadMetaData:getPartitionKeys method or the setPartitionFilter method. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (PIG-1825) ability to turn off the write ahead log for pig's HBaseStorage
[ https://issues.apache.org/jira/browse/PIG-1825?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12989330#comment-12989330 ] Alan Gates commented on PIG-1825: - Dmitriy, is this something we should check in? You seemed to indicate that this was no longer necessary after we moved to HBase 0.89 or above. ability to turn off the write ahead log for pig's HBaseStorage -- Key: PIG-1825 URL: https://issues.apache.org/jira/browse/PIG-1825 Project: Pig Issue Type: Improvement Affects Versions: 0.8.0 Reporter: Corbin Hoenes Priority: Minor Fix For: 0.8.0 Attachments: HBaseStorage_noWAL.patch Added an option to allow a caller of HBaseStorage to turn off the WriteAheadLog feature while doing bulk loads into hbase. From the performance tuning wikipage: http://wiki.apache.org/hadoop/PerformanceTuning To speed up the inserts in a non critical job (like an import job), you can use Put.writeToWAL(false) to bypass writing to the write ahead log. We've tested this on HBase 0.20.6 and it helps dramatically. The -noWAL options is passed in just like other options for hbase storage: STORE myalias INTO 'MyTable' USING org.apache.pig.backend.hadoop.hbase.HBaseStorage('mycolumnfamily:field1 mycolumnfamily:field2','-noWAL'); This would be my first patch so please educate me with any steps I need to do. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Updated: (PIG-1831) Indeterministic behavior in local mode due to static variable PigMapReduce.sJobConf
[ https://issues.apache.org/jira/browse/PIG-1831?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Daniel Dai updated PIG-1831: Attachment: PIG-1831-1.patch Richard suggest a better fix using ThreadLocal variable instead of static variable. Still keep static sJobConf for backward compatibility though it is already marked as deprecate in 0.7. In theory, if UDF still use deprecated sJobConf, they might see the same issue. But the chance of it should be very low. Indeterministic behavior in local mode due to static variable PigMapReduce.sJobConf --- Key: PIG-1831 URL: https://issues.apache.org/jira/browse/PIG-1831 Project: Pig Issue Type: Bug Affects Versions: 0.8.0 Reporter: Vivek Padmanabhan Assignee: Daniel Dai Attachments: PIG-1831-0.patch, PIG-1831-1.patch The below script when run in local mode gives me a different output. It looks like in local mode I have to store a relation obtained through streaming in order to use it afterwards. For example consider the below script : DEFINE MySTREAMUDF `test.sh`; A = LOAD 'myinput' USING PigStorage() AS (myId:chararray, data2, data3,data4 ); B = STREAM A THROUGH MySTREAMUDF AS (wId:chararray, num:int); --STORE B into 'output.B'; C = JOIN B by wId LEFT OUTER, A by myId; D = FOREACH C GENERATE B::wId,B::num,data4 ; D = STREAM D THROUGH MySTREAMUDF AS (f1:chararray,f2:int); --STORE D into 'output.D'; E = foreach B GENERATE wId,num; F = DISTINCT E; G = GROUP F ALL; H = FOREACH G GENERATE COUNT_STAR(F) as TotalCount; I = CROSS D,H; STORE I into 'output.I'; test.sh - #/bin/bash cut -f1,3 And input is abcdlabel1 11 feature1 acbdlabel2 22 feature2 adbclabel3 33 feature3 Here if I store relation B and D then everytime i get the result : acbd3 abcd3 adbc3 But if i dont store relations B and D then I get an empty output. Here again I have observed that this behaviour is random ie sometimes like 1out of 5 runs there will be output. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira
How to debug Pig exception
Hi List, How can I set up Eclipse or IntelliJ to debug Pig like a non-just-in-time-compiled Java app? Thanks, Andrew
Review Request: Indeterministic behavior in local mode due to static variable PigMapReduce.sJobConf
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/376/ --- Review request for pig and Richard Ding. Summary --- The below script when run in local mode gives me a different output. It looks like in local mode I have to store a relation obtained through streaming in order to use it afterwards. For example consider the below script : DEFINE MySTREAMUDF `test.sh`; A = LOAD 'myinput' USING PigStorage() AS (myId:chararray, data2, data3,data4 ); B = STREAM A THROUGH MySTREAMUDF AS (wId:chararray, num:int); --STORE B into 'output.B'; C = JOIN B by wId LEFT OUTER, A by myId; D = FOREACH C GENERATE B::wId,B::num,data4 ; D = STREAM D THROUGH MySTREAMUDF AS (f1:chararray,f2:int); --STORE D into 'output.D'; E = foreach B GENERATE wId,num; F = DISTINCT E; G = GROUP F ALL; H = FOREACH G GENERATE COUNT_STAR(F) as TotalCount; I = CROSS D,H; STORE I into 'output.I'; This addresses bug PIG-1831. https://issues.apache.org/jira/browse/PIG-1831 Diffs - http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POCollectedGroup.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POCombinerPackage.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/PODistinct.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POJoinPackage.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeCogroup.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeJoin.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPackage.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPartitionRearrange.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POSort.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/builtin/Distinct.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/data/InternalCachedBag.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/data/InternalDistinctBag.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/data/InternalSortedBag.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/impl/builtin/DefaultIndexableLoader.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/test/org/apache/pig/test/TestFRJoin.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/test/org/apache/pig/test/TestFinish.java 1065894 http://svn.apache.org/repos/asf/pig/trunk/test/org/apache/pig/test/utils/FILTERFROMFILE.java 1065894 Diff: https://reviews.apache.org/r/376/diff Testing --- Thanks, Daniel
[jira] Created: (PIG-1836) Accumulator like interface should be used with Pig operators after (co)group in certain cases
Accumulator like interface should be used with Pig operators after (co)group in certain cases - Key: PIG-1836 URL: https://issues.apache.org/jira/browse/PIG-1836 Project: Pig Issue Type: Improvement Reporter: Alan Gates There are a number of cases where people (co)group their data, and then pass it to an operator other than foreach with a UDF, but where an accumulator like interface would still make sense. A few examples: {code} C = group B by $0; D = foreach C generate flatten(B); ... C = group B by $0; D = stream C through 'script.py'; ... C = group B by $0; store C into 'output'; {code} In all these cases the following operator does not require all the data to be held in memory at once. There may be others beyond this. Changing this part of the pipeline would greatly speed these types of queries and make them less likely to die with out of memory errors. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (PIG-1825) ability to turn off the write ahead log for pig's HBaseStorage
[ https://issues.apache.org/jira/browse/PIG-1825?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12989436#comment-12989436 ] Dmitriy V. Ryaboy commented on PIG-1825: Sounds fine to me (though I haven't read the patch yet). HBase 0.90 has significant speed improvements but I imagine it still writes a WAL and you can still turn it off. ability to turn off the write ahead log for pig's HBaseStorage -- Key: PIG-1825 URL: https://issues.apache.org/jira/browse/PIG-1825 Project: Pig Issue Type: Improvement Affects Versions: 0.8.0 Reporter: Corbin Hoenes Priority: Minor Fix For: 0.8.0 Attachments: HBaseStorage_noWAL.patch Added an option to allow a caller of HBaseStorage to turn off the WriteAheadLog feature while doing bulk loads into hbase. From the performance tuning wikipage: http://wiki.apache.org/hadoop/PerformanceTuning To speed up the inserts in a non critical job (like an import job), you can use Put.writeToWAL(false) to bypass writing to the write ahead log. We've tested this on HBase 0.20.6 and it helps dramatically. The -noWAL options is passed in just like other options for hbase storage: STORE myalias INTO 'MyTable' USING org.apache.pig.backend.hadoop.hbase.HBaseStorage('mycolumnfamily:field1 mycolumnfamily:field2','-noWAL'); This would be my first patch so please educate me with any steps I need to do. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Created: (PIG-1838) On a large farm, some pigs die of /tmp starvation
On a large farm, some pigs die of /tmp starvation - Key: PIG-1838 URL: https://issues.apache.org/jira/browse/PIG-1838 Project: Pig Issue Type: Wish Components: impl Affects Versions: 0.8.0 Reporter: Allen Wittenauer We're starting to issues where interactive/command line pig users blow up due to so many large jar creations in /tmp. (In other words, pig execution prior to the java.io.tmpdir fix that Hadoop makes can kick in.) Pig should probably not depend upon users being savvy enough to override java.io.tmpdir on their own in these situations and/or a better steward of the space it does use. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (PIG-1838) On a large farm, some pigs die of /tmp starvation
[ https://issues.apache.org/jira/browse/PIG-1838?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12989481#comment-12989481 ] Allen Wittenauer commented on PIG-1838: --- I have a few thoughts on a better/programmatic way for Pig to be better behaved without depending on users doing the right thing. One or more of these would probably work: a) Redefine java.io.tmpdir itself after it gets the Hadoop property files loaded b) In the pig wrapper script, parse mapred-site.xml and pull out the mapred tmp space c) Override Java's createTempFile method to use Hadoop's tmp location/$TEMPDIR/$TMPDIR/$TEMP/some other value d) Change the jar assembly such that it goes into a create-submit-delete-repeat pattern. (From a casual glance, it appears to create all the jars at once rather than just when needed.) On a large farm, some pigs die of /tmp starvation - Key: PIG-1838 URL: https://issues.apache.org/jira/browse/PIG-1838 Project: Pig Issue Type: Wish Components: impl Affects Versions: 0.8.0 Reporter: Allen Wittenauer We're starting to issues where interactive/command line pig users blow up due to so many large jar creations in /tmp. (In other words, pig execution prior to the java.io.tmpdir fix that Hadoop makes can kick in.) Pig should probably not depend upon users being savvy enough to override java.io.tmpdir on their own in these situations and/or a better steward of the space it does use. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Updated: (PIG-1838) On a large farm, some pigs die of /tmp starvation
[ https://issues.apache.org/jira/browse/PIG-1838?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Allen Wittenauer updated PIG-1838: -- Description: We're starting to see issues where interactive/command line pig users blow up due to so many large jar creations in /tmp. (In other words, pig execution prior to the java.io.tmpdir fix that Hadoop makes can kick in.) Pig should probably not depend upon users being savvy enough to override java.io.tmpdir on their own in these situations and/or a better steward of the space it does use.(was: We're starting to issues where interactive/command line pig users blow up due to so many large jar creations in /tmp. (In other words, pig execution prior to the java.io.tmpdir fix that Hadoop makes can kick in.) Pig should probably not depend upon users being savvy enough to override java.io.tmpdir on their own in these situations and/or a better steward of the space it does use. ) On a large farm, some pigs die of /tmp starvation - Key: PIG-1838 URL: https://issues.apache.org/jira/browse/PIG-1838 Project: Pig Issue Type: Wish Components: impl Affects Versions: 0.8.0 Reporter: Allen Wittenauer We're starting to see issues where interactive/command line pig users blow up due to so many large jar creations in /tmp. (In other words, pig execution prior to the java.io.tmpdir fix that Hadoop makes can kick in.) Pig should probably not depend upon users being savvy enough to override java.io.tmpdir on their own in these situations and/or a better steward of the space it does use. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira
Re: Question about the hadoop-core-0.20.2.jar
Hi Qi, Can you try cleaning the build and then build jar afresh, by using 'ant clean jar'? As per what i see in ant target, it takes the hadoop jar from build directory. So, cleaning it up and building afresh might work. Hope this helps, Thanks, Chaitanya. On Wed, Feb 2, 2011 at 9:43 AM, qi zheng qzhe...@gmail.com wrote: Hi all, I made some changes in hadoop and generated the updated version of hadoop-core-0.20.2.jar to replace the one under pig/build/ivy/lib/pig. But when I tried to use ant to get new pig.jar, it still told me that the new method I added in hadoop counldn't be found. Any idea? Thanks -- Qi School of Engineering and Applied Science University of Pennsylvania