[jira] Commented: (PIG-1016) Reading in map data seems broken
[ https://issues.apache.org/jira/browse/PIG-1016?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771308#action_12771308 ] hc busy commented on PIG-1016: -- Well, I'd like to start by thanking everyone for the attention and support! As a first time contributor, I feel my heart warmed by the encouraging comments and serious time everyone is spending on my problem. I also greatly appreciate the patience everybody has, and of course I am perpetually grateful for everybody's work in making this all work. Line by line, {code} +// find bug is complaining about nulls. This check sequence will prevent nulls from being dereferenced. +if(o1!=null && o2!=null){ ... +}else{ + if(o1==null && o2==null){rc=0;} + else if(o1==null) {rc=-1;} + else{ rc=1; } {code} Does what it says, it prevents a findbug warning. non-null is greater than null by convention. {code} +// In case the objects are comparable +if((o1 instanceof NullableBytesWritable && o2 instanceof NullableBytesWritable)|| + !(o1 instanceof PigNullableWritable && o2 instanceof PigNullableWritable) +){ + + NullableBytesWritable nbw1 = (NullableBytesWritable)o1; + NullableBytesWritable nbw2 = (NullableBytesWritable)o2; + + // If either are null, handle differently. + if (!nbw1.isNull() && !nbw2.isNull()) { + rc = ((DataByteArray)nbw1.getValueAsPigType()).compareTo((DataByteArray)nbw2.getValueAsPigType()); + } else { + // For sorting purposes two nulls are equal. + if (nbw1.isNull() && nbw2.isNull()) rc = 0; + else if (nbw1.isNull()) rc = -1; + else rc = 1; + } +} {code} The if statement takes us outside of original comparison code (enclosed in outer if above) ONLY if both compratee are PigNullableWritable that are not NullableBytesWritable. This may seem confusing at first glance, but what it does is do the identical thing as before the patch except for the new case that I introduced by allowing other types. The code is awkward, as Santhosh noted. But I am not too sure I understand the original implementation. But certainly, this way, we preserve original behavior and for new cases that this patch introduces, they are handled in the remaining else: {code} else{ + // enter here only if both o1 and o2 are non-NullableByteWritable PigNullableWritable's + PigNullableWritable nbw1 = (PigNullableWritable)o1; + PigNullableWritable nbw2 = (PigNullableWritable)o2; + // If either are null, handle differently. + if (!nbw1.isNull() && !nbw2.isNull()) { + rc = nbw1.compareTo(nbw2); + } else { + // For sorting purposes two nulls are equal. + if (nbw1.isNull() && nbw2.isNull()) rc = 0; + else if (nbw1.isNull()) rc = -1; + else rc = 1; + } +} {code} This is the safest way I can think of writing this code, and I have been able to order by a value begotten out of a map. Also, join and then sort keyed on values of maps both works. I guess something that flows better might be the following: {code} if(o1!=null && o2!=null){ if((o1 instanceof PigNullableWritable && o2 instanceof PigNullableWritable ){ PigNullableWritable nbw1 = (PigNullableWritable)o1; PigNullableWritable nbw2 = (PigNullableWritable)o2; // If either are null, handle differently. if (!nbw1.isNull() && !nbw2.isNull()) { rc = nbw1.compareTo(nbw2); } else { // For sorting purposes two nulls are equal. if (nbw1.isNull() && nbw2.isNull()) rc = 0; else if (nbw1.isNull()) rc = -1; else rc = 1; } }else{ throw new Exception("bad compare"); } }else{ if(o1==null && o2==null){rc=0;} else if(o1==null) {rc=-1;} else{ rc=1; } {code} But I must admit that I don't know what the right thing to do is. I don't know the design well enough to know if throwing an exception is the appropriate thing? Or something else? And would the last code block perform the right comparison in place of the original function? lmk of your thoughts on improvements to the patch. > Reading in map data seems broken > > > Key: PIG-1016 > URL: https://issues.apache.org/jira/browse/PIG-1016 > Project: Pig > Issue Type: Improvement > Components: data >Affects Versions: 0.4.0 >Reporter: hc busy >
[jira] Updated: (PIG-953) Enable merge join in pig to work with loaders and store functions which can internally index sorted data
[ https://issues.apache.org/jira/browse/PIG-953?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Pradeep Kamath updated PIG-953: --- Attachment: PIG-953-9.patch - New patch addresses all javadoc, unit test and findbugs issues. - The release audit warnings are unrelated issues relating to html files and not code related. - I tried supressing deprecated related javac warning in code but looks like there is an existing javac [bug|http://bugs.sun.com/view_bug.do?bug_id=6594914] - so there is no way I am aware of to supress this in code and we may need to live with these warnings till we move to the new hadoop api > Enable merge join in pig to work with loaders and store functions which can > internally index sorted data > - > > Key: PIG-953 > URL: https://issues.apache.org/jira/browse/PIG-953 > Project: Pig > Issue Type: Improvement >Affects Versions: 0.3.0 >Reporter: Pradeep Kamath >Assignee: Pradeep Kamath > Attachments: PIG-953-2.patch, PIG-953-3.patch, PIG-953-4.patch, > PIG-953-5.patch, PIG-953-6.patch, PIG-953-7.patch, PIG-953-8.patch, > PIG-953-9.patch, PIG-953.patch > > > Currently merge join implementation in pig includes construction of an index > on sorted data and use of that index to seek into the "right input" to > efficiently perform the join operation. Some loaders (notably the zebra > loader) internally implement an index on sorted data and can perform this > seek efficiently using their index. So the use of the index needs to be > abstracted in such a way that when the loader supports indexing, pig uses it > (indirectly through the loader) and does not construct an index. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1016) Reading in map data seems broken
[ https://issues.apache.org/jira/browse/PIG-1016?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771287#action_12771287 ] Santhosh Srinivasan commented on PIG-1016: -- I am summarizing my understanding of the patch that has been submitted by hc busy. Root cause: PIG-880 changed the value type of maps in PigStorage from native Java types to DataByteArray. As a result of this change, parsing of complex types as map values was disabled. Proposed fix: Revert the changes made as part of PIG-880 to interpret map values as Java types. In addition, change the comparison method to check for the object type and call the appropriate compareTo method. The latter is required to workaround the fact that the front-end assigns the value type to be DataByteArray whereas the backend sees the actual type (Integer, Long, Tuple, DataBag, etc.) Based on this understanding I have the following review comment(s). Index: src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigBytesRawComparator.java === Can you explain the checks in the if and the else? Specifically, NullableBytesWritable is a subclass of PigNullableWritable. As a result, in the if part, the check for both o1 and o2 not being PigNullableWritable is confusing as nbw1 and nbw2 are cast to NullableBytesWritable if o1 and o2 are not PigNullableWritable. {code} +// find bug is complaining about nulls. This check sequence will prevent nulls from being dereferenced. +if(o1!=null && o2!=null){ + +// In case the objects are comparable +if((o1 instanceof NullableBytesWritable && o2 instanceof NullableBytesWritable)|| + !(o1 instanceof PigNullableWritable && o2 instanceof PigNullableWritable) +){ + + NullableBytesWritable nbw1 = (NullableBytesWritable)o1; + NullableBytesWritable nbw2 = (NullableBytesWritable)o2; + + // If either are null, handle differently. + if (!nbw1.isNull() && !nbw2.isNull()) { + rc = ((DataByteArray)nbw1.getValueAsPigType()).compareTo((DataByteArray)nbw2.getValueAsPigType()); + } else { + // For sorting purposes two nulls are equal. + if (nbw1.isNull() && nbw2.isNull()) rc = 0; + else if (nbw1.isNull()) rc = -1; + else rc = 1; + } +}else{ + // enter here only if both o1 and o2 are non-NullableByteWritable PigNullableWritable's + PigNullableWritable nbw1 = (PigNullableWritable)o1; + PigNullableWritable nbw2 = (PigNullableWritable)o2; + // If either are null, handle differently. + if (!nbw1.isNull() && !nbw2.isNull()) { + rc = nbw1.compareTo(nbw2); + } else { + // For sorting purposes two nulls are equal. + if (nbw1.isNull() && nbw2.isNull()) rc = 0; + else if (nbw1.isNull()) rc = -1; + else rc = 1; + } +} +}else{ + if(o1==null && o2==null){rc=0;} + else if(o1==null) {rc=-1;} + else{ rc=1; } {code} > Reading in map data seems broken > > > Key: PIG-1016 > URL: https://issues.apache.org/jira/browse/PIG-1016 > Project: Pig > Issue Type: Improvement > Components: data >Affects Versions: 0.4.0 >Reporter: hc busy > Fix For: 0.5.0 > > Attachments: PIG-1016.patch > > > Hi, I'm trying to load a map that has a tuple for value. The read fails in > 0.4.0 because of a misconfiguration in the parser. Where as in almost all > documentation it is stated that value of the map can be any time. > I've attached a patch that allows us to read in complex objects as value as > documented. I've done simple verification of loading in maps with tuple/map > values and writing them back out using LOAD and STORE. All seems to work fine. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1048) inner join using 'skewed' produces multiple rows for keys with single row in both input relations
[ https://issues.apache.org/jira/browse/PIG-1048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771249#action_12771249 ] Hadoop QA commented on PIG-1048: +1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12423389/pig_1048.patch against trunk revision 830757. +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 3 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. +1 core tests. The patch passed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/124/testReport/ Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/124/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/124/console This message is automatically generated. > inner join using 'skewed' produces multiple rows for keys with single row in > both input relations > - > > Key: PIG-1048 > URL: https://issues.apache.org/jira/browse/PIG-1048 > Project: Pig > Issue Type: Bug >Reporter: Thejas M Nair >Assignee: Sriranjan Manjunath > Attachments: pig_1048.patch > > > ${code} > grunt> cat students.txt > asdfxc M 23 12.44 > qwerF 21 14.44 > uhsdf M 34 12.11 > zxldf M 21 12.56 > qwerF 23 145.5 > oiueM 54 23.33 > l1 = load 'students.txt'; > l2 = load 'students.txt'; > j = join l1 by $0, l2 by $0 ; > store j into 'tmp.txt' > grunt> cat tmp.txt > oiueM 54 23.33 oiueM 54 23.33 > oiueM 54 23.33 oiueM 54 23.33 > qwerF 21 14.44 qwerF 21 14.44 > qwerF 21 14.44 qwerF 23 145.5 > qwerF 23 145.5 qwerF 21 14.44 > qwerF 23 145.5 qwerF 23 145.5 > uhsdf M 34 12.11 uhsdf M 34 12.11 > uhsdf M 34 12.11 uhsdf M 34 12.11 > zxldf M 21 12.56 zxldf M 21 12.56 > zxldf M 21 12.56 zxldf M 21 12.56 > asdfxc M 23 12.44 asdfxc M 23 12.44 > asdfxc M 23 12.44 asdfxc M 23 12.44$ > ${code} -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1057) [Zebra] Zebra does not support concurrent deletions of column groups now.
[ https://issues.apache.org/jira/browse/PIG-1057?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chao Wang updated PIG-1057: --- Attachment: patch_1057 > [Zebra] Zebra does not support concurrent deletions of column groups now. > - > > Key: PIG-1057 > URL: https://issues.apache.org/jira/browse/PIG-1057 > Project: Pig > Issue Type: Bug >Affects Versions: 0.4.0 >Reporter: Chao Wang >Assignee: Chao Wang > Fix For: 0.6.0 > > Attachments: patch_1057 > > > Zebra does not support concurrent deletions of column groups now. As a > result, the TestDropColumnGroup testcase can sometimes fail due to this. > In this testcase, multiple threads will be launched together, with each one > deleting one particular column group. The following exception can be thrown > (with callstack): > /*/ > ... > java.io.FileNotFoundException: File > /.../pig-trunk/build/contrib/zebra/test/data/DropCGTest/CG02 does not exist. > at > org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:361) > at > org.apache.hadoop.fs.RawLocalFileSystem.listStatus(RawLocalFileSystem.java:290) > at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:716) > at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:741) > at > org.apache.hadoop.fs.ChecksumFileSystem.listStatus(ChecksumFileSystem.java:465) > at > org.apache.hadoop.zebra.io.BasicTable$SchemaFile.setCGDeletedFlags(BasicTable.java:1610) > at > org.apache.hadoop.zebra.io.BasicTable$SchemaFile.readSchemaFile(BasicTable.java:1593) > at > org.apache.hadoop.zebra.io.BasicTable$SchemaFile.(BasicTable.java:1416) > at > org.apache.hadoop.zebra.io.BasicTable.dropColumnGroup(BasicTable.java:133) > at > org.apache.hadoop.zebra.io.TestDropColumnGroup$DropThread.run(TestDropColumnGroup.java:772) > ... > /*/ > We plan to fix this in Zebra to support concurrent deletions of column > groups. The root cause is that a thread or process reads in some stale file > system information (e.g., it sees /CG0 first) and then can fail later on (it > tries to access /CG0, however /CG0 may be deleted by another thread or > process). Therefore, we plan to adopt a retry logic to resolve this issue. > More detailed, we allow a dropping column group thread to retry n times when > doing its deleting job - n is the total number of column groups. > Note that here we do NOT try to resolve the more general concurrent column > group deletions + reads issue. If a process is reading some data that could > be deleted by another process, it can fail as we expect. > Here we only try to resolve the concurrent column group deletions issue. If > you have multiple threads or processes to delete column groups, they should > succeed. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-859) Optimizer throw error on self-joins
[ https://issues.apache.org/jira/browse/PIG-859?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771223#action_12771223 ] Daniel Dai commented on PIG-859: I was wondering whether Pig Latin should allow self-join. In the script: A = load 'a' as (a0, a1); B = Join A by $0, A by $0; The output schema for B is (A.a0, A.a1, A.a0, A.a1). It is doom to cause a schema alias conflict. > Optimizer throw error on self-joins > --- > > Key: PIG-859 > URL: https://issues.apache.org/jira/browse/PIG-859 > Project: Pig > Issue Type: Bug > Components: impl >Affects Versions: 0.3.0 >Reporter: Ashutosh Chauhan > > Doing self-join results in exception thrown by Optimizer. Consider the > following query > {code} > grunt> A = load 'a'; > grunt> B = Join A by $0, A by $0; > grunt> explain B; > 2009-06-20 15:51:38,303 [main] ERROR org.apache.pig.tools.grunt.Grunt - > ERROR 1094: Attempt to insert between two nodes that were not connected. > Details at logfile: pig_1245538027026.log > {code} > Relevant stack-trace from log-file: > {code} > Caused by: org.apache.pig.impl.plan.optimizer.OptimizerException: ERROR > 2047: Internal error. Unable to introduce split operators. > at > org.apache.pig.impl.logicalLayer.optimizer.ImplicitSplitInserter.transform(ImplicitSplitInserter.java:163) > at > org.apache.pig.impl.logicalLayer.optimizer.LogicalOptimizer.optimize(LogicalOptimizer.java:163) > at org.apache.pig.PigServer.compileLp(PigServer.java:844) > at org.apache.pig.PigServer.compileLp(PigServer.java:781) > at org.apache.pig.PigServer.getStorePlan(PigServer.java:723) > at org.apache.pig.PigServer.explain(PigServer.java:566) > ... 8 more > Caused by: org.apache.pig.impl.plan.PlanException: ERROR 1094: Attempt > to insert between two nodes that were not connected. > at > org.apache.pig.impl.plan.OperatorPlan.doInsertBetween(OperatorPlan.java:500) > at > org.apache.pig.impl.plan.OperatorPlan.insertBetween(OperatorPlan.java:480) > at > org.apache.pig.impl.logicalLayer.optimizer.ImplicitSplitInserter.transform(ImplicitSplitInserter.java:139) > ... 13 more > {code} > A possible workaround is: > {code} > grunt> A = load 'a'; > grunt> B = load 'a'; > grunt> C = join A by $0, B by $0; > grunt> explain C; > {code} -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-859) Optimizer throw error on self-joins
[ https://issues.apache.org/jira/browse/PIG-859?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771220#action_12771220 ] Jing Huang commented on PIG-859: if we do joina = join rec1 by (a), rec1 by (a) using "merge" ; New error message is thrown by parser, Error message is : Caused by: org.apache.pig.impl.plan.PlanValidationException: ERROR 1108: Duplicate schema alias: rec1::a in "joina" at org.apache.pig.impl.logicalLayer.validators.SchemaAliasVisitor.validate(SchemaAliasVisitor.java:69) at org.apache.pig.impl.logicalLayer.validators.SchemaAliasVisitor.visit(SchemaAliasVisitor.java:115) at org.apache.pig.impl.logicalLayer.LOJoin.visit(LOJoin.java:203) at org.apache.pig.impl.logicalLayer.LOJoin.visit(LOJoin.java:45) at org.apache.pig.impl.plan.DepthFirstWalker.depthFirst(DepthFirstWalker.java:67) at org.apache.pig.impl.plan.DepthFirstWalker.depthFirst(DepthFirstWalker.java:69) at org.apache.pig.impl.plan.DepthFirstWalker.walk(DepthFirstWalker.java:50) at org.apache.pig.impl.plan.PlanVisitor.visit(PlanVisitor.java:51) at org.apache.pig.impl.plan.PlanValidator.validateSkipCollectException(PlanValidator.java:101) ... 13 more > Optimizer throw error on self-joins > --- > > Key: PIG-859 > URL: https://issues.apache.org/jira/browse/PIG-859 > Project: Pig > Issue Type: Bug > Components: impl >Affects Versions: 0.3.0 >Reporter: Ashutosh Chauhan > > Doing self-join results in exception thrown by Optimizer. Consider the > following query > {code} > grunt> A = load 'a'; > grunt> B = Join A by $0, A by $0; > grunt> explain B; > 2009-06-20 15:51:38,303 [main] ERROR org.apache.pig.tools.grunt.Grunt - > ERROR 1094: Attempt to insert between two nodes that were not connected. > Details at logfile: pig_1245538027026.log > {code} > Relevant stack-trace from log-file: > {code} > Caused by: org.apache.pig.impl.plan.optimizer.OptimizerException: ERROR > 2047: Internal error. Unable to introduce split operators. > at > org.apache.pig.impl.logicalLayer.optimizer.ImplicitSplitInserter.transform(ImplicitSplitInserter.java:163) > at > org.apache.pig.impl.logicalLayer.optimizer.LogicalOptimizer.optimize(LogicalOptimizer.java:163) > at org.apache.pig.PigServer.compileLp(PigServer.java:844) > at org.apache.pig.PigServer.compileLp(PigServer.java:781) > at org.apache.pig.PigServer.getStorePlan(PigServer.java:723) > at org.apache.pig.PigServer.explain(PigServer.java:566) > ... 8 more > Caused by: org.apache.pig.impl.plan.PlanException: ERROR 1094: Attempt > to insert between two nodes that were not connected. > at > org.apache.pig.impl.plan.OperatorPlan.doInsertBetween(OperatorPlan.java:500) > at > org.apache.pig.impl.plan.OperatorPlan.insertBetween(OperatorPlan.java:480) > at > org.apache.pig.impl.logicalLayer.optimizer.ImplicitSplitInserter.transform(ImplicitSplitInserter.java:139) > ... 13 more > {code} > A possible workaround is: > {code} > grunt> A = load 'a'; > grunt> B = load 'a'; > grunt> C = join A by $0, B by $0; > grunt> explain C; > {code} -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-747) Logical to Physical Plan Translation fails when temporary alias are created within foreach
[ https://issues.apache.org/jira/browse/PIG-747?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771209#action_12771209 ] Daniel Dai commented on PIG-747: Please hold on this patch. Seems the calculation is not good. Looks like supportsMultipleOutputs is useful to prevent repeated calculation. We need to investigate more before submit new patch. > Logical to Physical Plan Translation fails when temporary alias are created > within foreach > -- > > Key: PIG-747 > URL: https://issues.apache.org/jira/browse/PIG-747 > Project: Pig > Issue Type: Bug >Affects Versions: 0.4.0 >Reporter: Viraj Bhat >Assignee: Daniel Dai > Fix For: 0.6.0 > > Attachments: physicalplan.txt, physicalplanprob.pig, PIG-747-1.patch > > > Consider a the pig script which calculates a new column F inside the foreach > as: > {code} > A = load 'physicalplan.txt' as (col1,col2,col3); > B = foreach A { >D = col1/col2; >E = col3/col2; >F = E - (D*D); >generate >F as newcol; > }; > dump B; > {code} > This gives the following error: > === > Caused by: > org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogicalToPhysicalTranslatorException: > ERROR 2015: Invalid physical operators in the physical plan > at > org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTranslationVisitor.visit(LogToPhyTranslationVisitor.java:377) > at > org.apache.pig.impl.logicalLayer.LOMultiply.visit(LOMultiply.java:63) > at > org.apache.pig.impl.logicalLayer.LOMultiply.visit(LOMultiply.java:29) > at > org.apache.pig.impl.plan.DependencyOrderWalkerWOSeenChk.walk(DependencyOrderWalkerWOSeenChk.java:68) > at > org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTranslationVisitor.visit(LogToPhyTranslationVisitor.java:908) > at > org.apache.pig.impl.logicalLayer.LOForEach.visit(LOForEach.java:122) > at org.apache.pig.impl.logicalLayer.LOForEach.visit(LOForEach.java:41) > at > org.apache.pig.impl.plan.DependencyOrderWalker.walk(DependencyOrderWalker.java:68) > at org.apache.pig.impl.plan.PlanVisitor.visit(PlanVisitor.java:51) > at > org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.compile(HExecutionEngine.java:246) > ... 10 more > Caused by: org.apache.pig.impl.plan.PlanException: ERROR 0: Attempt to give > operator of type > org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.Divide > multiple outputs. This operator does not support multiple outputs. > at > org.apache.pig.impl.plan.OperatorPlan.connect(OperatorPlan.java:158) > at > org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan.connect(PhysicalPlan.java:89) > at > org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTranslationVisitor.visit(LogToPhyTranslationVisitor.java:373) > ... 19 more > === -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1048) inner join using 'skewed' produces multiple rows for keys with single row in both input relations
[ https://issues.apache.org/jira/browse/PIG-1048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771203#action_12771203 ] Alan Gates commented on PIG-1048: - Could you describe briefly the cause of the problem and how the one line change fixes it? > inner join using 'skewed' produces multiple rows for keys with single row in > both input relations > - > > Key: PIG-1048 > URL: https://issues.apache.org/jira/browse/PIG-1048 > Project: Pig > Issue Type: Bug >Reporter: Thejas M Nair >Assignee: Sriranjan Manjunath > Attachments: pig_1048.patch > > > ${code} > grunt> cat students.txt > asdfxc M 23 12.44 > qwerF 21 14.44 > uhsdf M 34 12.11 > zxldf M 21 12.56 > qwerF 23 145.5 > oiueM 54 23.33 > l1 = load 'students.txt'; > l2 = load 'students.txt'; > j = join l1 by $0, l2 by $0 ; > store j into 'tmp.txt' > grunt> cat tmp.txt > oiueM 54 23.33 oiueM 54 23.33 > oiueM 54 23.33 oiueM 54 23.33 > qwerF 21 14.44 qwerF 21 14.44 > qwerF 21 14.44 qwerF 23 145.5 > qwerF 23 145.5 qwerF 21 14.44 > qwerF 23 145.5 qwerF 23 145.5 > uhsdf M 34 12.11 uhsdf M 34 12.11 > uhsdf M 34 12.11 uhsdf M 34 12.11 > zxldf M 21 12.56 zxldf M 21 12.56 > zxldf M 21 12.56 zxldf M 21 12.56 > asdfxc M 23 12.44 asdfxc M 23 12.44 > asdfxc M 23 12.44 asdfxc M 23 12.44$ > ${code} -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1016) Reading in map data seems broken
[ https://issues.apache.org/jira/browse/PIG-1016?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771200#action_12771200 ] Alan Gates commented on PIG-1016: - I am keeping an eye on this ticket. But at this point I'd like to get Santhosh's feedback on your changes before proceeding, as he had comments on your earlier patch and I want to make sure your new patch addresses them. Santhosh, can you provide feedback soon, or let one of the other committers know what to look for so we can move forward on this? > Reading in map data seems broken > > > Key: PIG-1016 > URL: https://issues.apache.org/jira/browse/PIG-1016 > Project: Pig > Issue Type: Improvement > Components: data >Affects Versions: 0.4.0 >Reporter: hc busy > Fix For: 0.5.0 > > Attachments: PIG-1016.patch > > > Hi, I'm trying to load a map that has a tuple for value. The read fails in > 0.4.0 because of a misconfiguration in the parser. Where as in almost all > documentation it is stated that value of the map can be any time. > I've attached a patch that allows us to read in complex objects as value as > documented. I've done simple verification of loading in maps with tuple/map > values and writing them back out using LOAD and STORE. All seems to work fine. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1001) Generate more meaningful error message when one input file does not exist
[ https://issues.apache.org/jira/browse/PIG-1001?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Daniel Dai updated PIG-1001: Status: Patch Available (was: Open) > Generate more meaningful error message when one input file does not exist > - > > Key: PIG-1001 > URL: https://issues.apache.org/jira/browse/PIG-1001 > Project: Pig > Issue Type: Bug >Affects Versions: 0.4.0 >Reporter: Daniel Dai > Fix For: 0.6.0 > > Attachments: PIG-1001-1.patch, PIG-1001-2.patch > > > In the following query, if 1.txt does not exist, > a = load '1.txt'; > b = group a by $0; > c = group b all; > dump c; > Pig throws error message "ERROR 2100: file:/tmp/temp155054664/tmp1144108421 > does not exist.", Pig should deal with it with the error message "Input file > 1.txt not exist" instead of those confusing messages. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1001) Generate more meaningful error message when one input file does not exist
[ https://issues.apache.org/jira/browse/PIG-1001?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Daniel Dai updated PIG-1001: Attachment: PIG-1001-2.patch Reattach the patch per Alan's comment. > Generate more meaningful error message when one input file does not exist > - > > Key: PIG-1001 > URL: https://issues.apache.org/jira/browse/PIG-1001 > Project: Pig > Issue Type: Bug >Affects Versions: 0.4.0 >Reporter: Daniel Dai > Fix For: 0.6.0 > > Attachments: PIG-1001-1.patch, PIG-1001-2.patch > > > In the following query, if 1.txt does not exist, > a = load '1.txt'; > b = group a by $0; > c = group b all; > dump c; > Pig throws error message "ERROR 2100: file:/tmp/temp155054664/tmp1144108421 > does not exist.", Pig should deal with it with the error message "Input file > 1.txt not exist" instead of those confusing messages. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1001) Generate more meaningful error message when one input file does not exist
[ https://issues.apache.org/jira/browse/PIG-1001?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Daniel Dai updated PIG-1001: Status: Open (was: Patch Available) > Generate more meaningful error message when one input file does not exist > - > > Key: PIG-1001 > URL: https://issues.apache.org/jira/browse/PIG-1001 > Project: Pig > Issue Type: Bug >Affects Versions: 0.4.0 >Reporter: Daniel Dai > Fix For: 0.6.0 > > Attachments: PIG-1001-1.patch, PIG-1001-2.patch > > > In the following query, if 1.txt does not exist, > a = load '1.txt'; > b = group a by $0; > c = group b all; > dump c; > Pig throws error message "ERROR 2100: file:/tmp/temp155054664/tmp1144108421 > does not exist.", Pig should deal with it with the error message "Input file > 1.txt not exist" instead of those confusing messages. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1036) Fragment-replicate left outer join
[ https://issues.apache.org/jira/browse/PIG-1036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771195#action_12771195 ] Olga Natkovich commented on PIG-1036: - Actually, in a separate findbugs patch, I am removing all the extra constructors because they do cause NPE. > Fragment-replicate left outer join > -- > > Key: PIG-1036 > URL: https://issues.apache.org/jira/browse/PIG-1036 > Project: Pig > Issue Type: New Feature >Reporter: Olga Natkovich >Assignee: Ankit Modi > Attachments: LeftOuterFRJoin.patch > > -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1036) Fragment-replicate left outer join
[ https://issues.apache.org/jira/browse/PIG-1036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771192#action_12771192 ] Dmitriy V. Ryaboy commented on PIG-1036: This is in the old POFRJoin too, but looking at the constructors, there's a lurking NPE: {code} public POFRJoin(OperatorKey k) throws PlanException, ExecException { this(k,-1,null, null, null, null, -1, false); } [...] public POFRJoin(OperatorKey k, int rp, List inp, List> ppLists, List> keyTypes, FileSpec[] replFiles, int fragment, boolean isLeftOuter) throws ExecException{ super(k,rp,inp); phyPlanLists = ppLists; this.fragment = fragment; this.keyTypes = keyTypes; this.replFiles = replFiles; replicates = new Map[ppLists.size()]; // BANG [...] {code} size() is getting called on an object that might be null. Perhaps input list, ppLists, keyTypes, and FileSpec[] should be initialized to empty objects instead of nulls when they are not set? > Fragment-replicate left outer join > -- > > Key: PIG-1036 > URL: https://issues.apache.org/jira/browse/PIG-1036 > Project: Pig > Issue Type: New Feature >Reporter: Olga Natkovich >Assignee: Ankit Modi > Attachments: LeftOuterFRJoin.patch > > -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-953) Enable merge join in pig to work with loaders and store functions which can internally index sorted data
[ https://issues.apache.org/jira/browse/PIG-953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771188#action_12771188 ] Olga Natkovich commented on PIG-953: +1 on the patch assuming that the all the failures and warning from test-patch are addressed. > Enable merge join in pig to work with loaders and store functions which can > internally index sorted data > - > > Key: PIG-953 > URL: https://issues.apache.org/jira/browse/PIG-953 > Project: Pig > Issue Type: Improvement >Affects Versions: 0.3.0 >Reporter: Pradeep Kamath >Assignee: Pradeep Kamath > Attachments: PIG-953-2.patch, PIG-953-3.patch, PIG-953-4.patch, > PIG-953-5.patch, PIG-953-6.patch, PIG-953-7.patch, PIG-953-8.patch, > PIG-953.patch > > > Currently merge join implementation in pig includes construction of an index > on sorted data and use of that index to seek into the "right input" to > efficiently perform the join operation. Some loaders (notably the zebra > loader) internally implement an index on sorted data and can perform this > seek efficiently using their index. So the use of the index needs to be > abstracted in such a way that when the loader supports indexing, pig uses it > (indirectly through the loader) and does not construct an index. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1036) Fragment-replicate left outer join
[ https://issues.apache.org/jira/browse/PIG-1036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771184#action_12771184 ] Olga Natkovich commented on PIG-1036: - Any findbug warnings that can't be fixed, must go into the exclusion file. We can't commit any patches that increase the count for findbugs or javac warnings. > Fragment-replicate left outer join > -- > > Key: PIG-1036 > URL: https://issues.apache.org/jira/browse/PIG-1036 > Project: Pig > Issue Type: New Feature >Reporter: Olga Natkovich >Assignee: Ankit Modi > Attachments: LeftOuterFRJoin.patch > > -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1036) Fragment-replicate left outer join
[ https://issues.apache.org/jira/browse/PIG-1036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771175#action_12771175 ] Ankit Modi commented on PIG-1036: - This patch fails in findBugs as I had modified ***lines (4 lines of constructors)*** that contained findBugs warnings earlier. > Fragment-replicate left outer join > -- > > Key: PIG-1036 > URL: https://issues.apache.org/jira/browse/PIG-1036 > Project: Pig > Issue Type: New Feature >Reporter: Olga Natkovich >Assignee: Ankit Modi > Attachments: LeftOuterFRJoin.patch > > -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1036) Fragment-replicate left outer join
[ https://issues.apache.org/jira/browse/PIG-1036?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ankit Modi updated PIG-1036: Status: Patch Available (was: Open) > Fragment-replicate left outer join > -- > > Key: PIG-1036 > URL: https://issues.apache.org/jira/browse/PIG-1036 > Project: Pig > Issue Type: New Feature >Reporter: Olga Natkovich >Assignee: Ankit Modi > Attachments: LeftOuterFRJoin.patch > > -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1016) Reading in map data seems broken
[ https://issues.apache.org/jira/browse/PIG-1016?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771170#action_12771170 ] hc busy commented on PIG-1016: -- Okay, trying to get this into a release of pig... I noticed 0.4 came , but nothing has happened on this ticket. > Reading in map data seems broken > > > Key: PIG-1016 > URL: https://issues.apache.org/jira/browse/PIG-1016 > Project: Pig > Issue Type: Improvement > Components: data >Affects Versions: 0.4.0 >Reporter: hc busy > Fix For: 0.5.0 > > Attachments: PIG-1016.patch > > > Hi, I'm trying to load a map that has a tuple for value. The read fails in > 0.4.0 because of a misconfiguration in the parser. Where as in almost all > documentation it is stated that value of the map can be any time. > I've attached a patch that allows us to read in complex objects as value as > documented. I've done simple verification of loading in maps with tuple/map > values and writing them back out using LOAD and STORE. All seems to work fine. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1036) Fragment-replicate left outer join
[ https://issues.apache.org/jira/browse/PIG-1036?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ankit Modi updated PIG-1036: Attachment: LeftOuterFRJoin.patch This patch fails in findBugs as I had modified the line that contained findBugs warnings earlier. It also fails on ReleaseAudit for html ( doc ) file for POFRJoin > Fragment-replicate left outer join > -- > > Key: PIG-1036 > URL: https://issues.apache.org/jira/browse/PIG-1036 > Project: Pig > Issue Type: New Feature >Reporter: Olga Natkovich >Assignee: Ankit Modi > Attachments: LeftOuterFRJoin.patch > > -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1016) Reading in map data seems broken
[ https://issues.apache.org/jira/browse/PIG-1016?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] hc busy updated PIG-1016: - Fix Version/s: (was: 0.4.0) 0.5.0 Status: Patch Available (was: Open) > Reading in map data seems broken > > > Key: PIG-1016 > URL: https://issues.apache.org/jira/browse/PIG-1016 > Project: Pig > Issue Type: Improvement > Components: data >Affects Versions: 0.4.0 >Reporter: hc busy > Fix For: 0.5.0 > > Attachments: PIG-1016.patch > > > Hi, I'm trying to load a map that has a tuple for value. The read fails in > 0.4.0 because of a misconfiguration in the parser. Where as in almost all > documentation it is stated that value of the map can be any time. > I've attached a patch that allows us to read in complex objects as value as > documented. I've done simple verification of loading in maps with tuple/map > values and writing them back out using LOAD and STORE. All seems to work fine. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Assigned: (PIG-1036) Fragment-replicate left outer join
[ https://issues.apache.org/jira/browse/PIG-1036?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ankit Modi reassigned PIG-1036: --- Assignee: Ankit Modi > Fragment-replicate left outer join > -- > > Key: PIG-1036 > URL: https://issues.apache.org/jira/browse/PIG-1036 > Project: Pig > Issue Type: New Feature >Reporter: Olga Natkovich >Assignee: Ankit Modi > -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1016) Reading in map data seems broken
[ https://issues.apache.org/jira/browse/PIG-1016?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] hc busy updated PIG-1016: - Fix Version/s: 0.4.0 Status: Open (was: Patch Available) > Reading in map data seems broken > > > Key: PIG-1016 > URL: https://issues.apache.org/jira/browse/PIG-1016 > Project: Pig > Issue Type: Improvement > Components: data >Affects Versions: 0.4.0 >Reporter: hc busy > Fix For: 0.4.0 > > Attachments: PIG-1016.patch > > > Hi, I'm trying to load a map that has a tuple for value. The read fails in > 0.4.0 because of a misconfiguration in the parser. Where as in almost all > documentation it is stated that value of the map can be any time. > I've attached a patch that allows us to read in complex objects as value as > documented. I've done simple verification of loading in maps with tuple/map > values and writing them back out using LOAD and STORE. All seems to work fine. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1055) FINDBUGS: remaining "Dodgy Warnings"
[ https://issues.apache.org/jira/browse/PIG-1055?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771165#action_12771165 ] Daniel Dai commented on PIG-1055: - +1 > FINDBUGS: remaining "Dodgy Warnings" > > > Key: PIG-1055 > URL: https://issues.apache.org/jira/browse/PIG-1055 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich >Assignee: Olga Natkovich > Attachments: PIG-1055.patch > > > BCQuestionable cast from java.util.List to java.util.ArrayList in new > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit(PigContext, > FileSystem, Path, String, List, long, long) > Eqorg.apache.pig.data.AmendableTuple doesn't override > DefaultTuple.equals(Object) > Eqorg.apache.pig.data.TimestampedTuple doesn't override > DefaultTuple.equals(Object) > IAAmbiguous invocation of either an outer or inherited method > org.apache.pig.impl.plan.DotPlanDumper.getName(Operator) in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.DotMRPrinter$InnerPrinter.getAttributes(DotMRPrinter$InnerOperator) > IMComputation of average could overflow in > org.apache.tools.bzip2r.CBZip2OutputStream.qSort3(int, int, int) > IMCheck for oddness that won't work for negative numbers in > org.apache.tools.bzip2r.CBZip2OutputStream.sendMTFValues() > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.doHod(String, > Properties) > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.CombinerOptimizer.visitMROp(MapReduceOper) > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler.visitDistinct(PODistinct) > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler.visitFRJoin(POFRJoin) > REC Exception is caught when Exception is not thrown in > org.apache.pig.impl.logicalLayer.optimizer.OpLimitOptimizer.processNode(LOLimit) > REC Exception is caught when Exception is not thrown in > org.apache.pig.tools.streams.StreamGenerator.actionPerformed(ActionEvent) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner.sJobConf > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner$Combine.configure(JobConf) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.activeSplit > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.getRecordReader(InputSplit, > JobConf, Reporter) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.sJob > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.getRecordReader(InputSplit, > JobConf, Reporter) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce.sJobConf > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.configure(JobConf) > STWrite to static field org.apache.pig.data.BagFactory.gMemMgr from > instance method new org.apache.pig.data.BagFactory() > STWrite to static field > org.apache.pig.impl.logicalLayer.LogicalPlanCloneHelper.mOpToCloneMap from > instance method new > org.apache.pig.impl.logicalLayer.LogicalPlanCloneHelper(LogicalPlan, Map) > STWrite to static field > org.apache.pig.impl.logicalLayer.LogicalPlanBuilder.classloader from instance > method org.apache.pig.impl.PigContext.addJar(URL) -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1022) optimizer pushes filter before the foreach that generates column used by filter
[ https://issues.apache.org/jira/browse/PIG-1022?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771157#action_12771157 ] Daniel Dai commented on PIG-1022: - The core test failure is temporal due to "port conflict" > optimizer pushes filter before the foreach that generates column used by > filter > --- > > Key: PIG-1022 > URL: https://issues.apache.org/jira/browse/PIG-1022 > Project: Pig > Issue Type: Bug > Components: impl >Affects Versions: 0.4.0 >Reporter: Thejas M Nair >Assignee: Daniel Dai > Fix For: 0.6.0 > > Attachments: PIG-1022-1.patch > > > grunt> l = load 'students.txt' using PigStorage() as (name:chararray, > gender:chararray, age:chararray, score:chararray); > grunt> f = foreach l generate name, gender, age,score, '200' as > gid:chararray; > grunt> g = group f by (name, gid); > grunt> f2 = foreach g generate group.name as name: chararray, group.gid as > gid: chararray; > grunt> filt = filter f2 by gid == '200'; > grunt> explain filt; > In the plan generated filt is pushed up after the load and before the first > foreach, even though the filter is on gid which is generated in first foreach. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1001) Generate more meaningful error message when one input file does not exist
[ https://issues.apache.org/jira/browse/PIG-1001?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771152#action_12771152 ] Daniel Dai commented on PIG-1001: - Hi, Alan, The code do remove B and C. It uses fifo to achieve that to avoid recursion. But you reminds me that we have trimBlow(). We should use trimBlow() instead of implementing a duplicate logic. > Generate more meaningful error message when one input file does not exist > - > > Key: PIG-1001 > URL: https://issues.apache.org/jira/browse/PIG-1001 > Project: Pig > Issue Type: Bug >Affects Versions: 0.4.0 >Reporter: Daniel Dai > Fix For: 0.6.0 > > Attachments: PIG-1001-1.patch > > > In the following query, if 1.txt does not exist, > a = load '1.txt'; > b = group a by $0; > c = group b all; > dump c; > Pig throws error message "ERROR 2100: file:/tmp/temp155054664/tmp1144108421 > does not exist.", Pig should deal with it with the error message "Input file > 1.txt not exist" instead of those confusing messages. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-953) Enable merge join in pig to work with loaders and store functions which can internally index sorted data
[ https://issues.apache.org/jira/browse/PIG-953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771136#action_12771136 ] Hadoop QA commented on PIG-953: --- -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12423385/PIG-953-8.patch against trunk revision 830664. +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 6 new or modified tests. -1 javadoc. The javadoc tool appears to have generated 1 warning messages. -1 javac. The applied patch generated 202 javac compiler warnings (more than the trunk's current 197 warnings). -1 findbugs. The patch appears to introduce 5 new Findbugs warnings. -1 release audit. The applied patch generated 320 release audit warnings (more than the trunk's current 313 warnings). -1 core tests. The patch failed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/123/testReport/ Release audit warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/123/artifact/trunk/patchprocess/releaseAuditDiffWarnings.txt Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/123/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/123/console This message is automatically generated. > Enable merge join in pig to work with loaders and store functions which can > internally index sorted data > - > > Key: PIG-953 > URL: https://issues.apache.org/jira/browse/PIG-953 > Project: Pig > Issue Type: Improvement >Affects Versions: 0.3.0 >Reporter: Pradeep Kamath >Assignee: Pradeep Kamath > Attachments: PIG-953-2.patch, PIG-953-3.patch, PIG-953-4.patch, > PIG-953-5.patch, PIG-953-6.patch, PIG-953-7.patch, PIG-953-8.patch, > PIG-953.patch > > > Currently merge join implementation in pig includes construction of an index > on sorted data and use of that index to seek into the "right input" to > efficiently perform the join operation. Some loaders (notably the zebra > loader) internally implement an index on sorted data and can perform this > seek efficiently using their index. So the use of the index needs to be > abstracted in such a way that when the loader supports indexing, pig uses it > (indirectly through the loader) and does not construct an index. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1052) FINDBUGS: remaining performance warnings
[ https://issues.apache.org/jira/browse/PIG-1052?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Olga Natkovich updated PIG-1052: Resolution: Fixed Status: Resolved (was: Patch Available) patch committed > FINDBUGS: remaining performance warnings > > > Key: PIG-1052 > URL: https://issues.apache.org/jira/browse/PIG-1052 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich >Assignee: Olga Natkovich > Attachments: PIG-1052.patch > > > SBSC Method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.Launcher.getStackTraceElement(String) > concatenates strings using + in a loop > SBSC Method org.apache.pig.impl.logicalLayer.LOCross.getSchema() > concatenates strings using + in a loop > SBSC Method org.apache.pig.impl.logicalLayer.LOForEach.getSchema() > concatenates strings using + in a loop > SBSC Method org.apache.pig.PigServer.locateJarFromResources(String) > concatenates strings using + in a loop > SBSC Method org.apache.pig.tools.parameters.ParseException.initialise(Token, > int[][], String[]) concatenates strings using + in a loop > SBSC Method > org.apache.pig.tools.parameters.PreprocessorContext.executeShellCommand(String) > concatenates strings using + in a loop > SSUnread field: > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.Launcher.OOM_ERR; > should this field be static? > SSUnread field: > org.apache.pig.impl.io.BufferedPositionedInputStream.bufSize; should this > field be static? > UPM Private method > org.apache.pig.impl.plan.optimizer.RulePlanPrinter.planString(List) is never > called > UPM Private method org.apache.pig.impl.plan.PlanPrinter.planString(List) is > never called > WMI Method org.apache.pig.builtin.PigStorage.putField(Object) makes > inefficient use of keySet iterator instead of entrySet iterator > WMI Method org.apache.pig.data.DataType.mapToString(Map) makes inefficient > use of keySet iterator instead of entrySet iterator > WMI Method org.apache.pig.impl.logicalLayer.LOCross.getSchema() makes > inefficient use of keySet iterator instead of entrySet iterator > WMI Method org.apache.pig.impl.logicalLayer.LOForEach.getSchema() makes > inefficient use of keySet iterator instead of entrySet iterator > WMI Method > org.apache.pig.impl.logicalLayer.validators.TypeCheckingVisitor.getLoadFuncSpec(Schema$FieldSchema, > String) makes inefficient use of keySet iterator instead of entrySet iterator > WMI Method > org.apache.pig.impl.plan.CompilationMessageCollector.logAggregate(Map, > CompilationMessageCollector$MessageType, Log) makes inefficient use of keySet > iterator instead of entrySet iterator > WMI Method org.apache.pig.StandAloneParser.tryParse(String) makes > inefficient use of keySet iterator instead of entrySet iterator -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-790) Error message should indicate in which line number in the Pig script the error occured (debugging BinCond)
[ https://issues.apache.org/jira/browse/PIG-790?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Daniel Dai updated PIG-790: --- Resolution: Fixed Hadoop Flags: [Reviewed] Status: Resolved (was: Patch Available) Patch committed. No test included in patch since it is only about error message and there is no good way to test automatically. > Error message should indicate in which line number in the Pig script the > error occured (debugging BinCond) > -- > > Key: PIG-790 > URL: https://issues.apache.org/jira/browse/PIG-790 > Project: Pig > Issue Type: Bug >Affects Versions: 0.4.0 >Reporter: Viraj Bhat >Assignee: Daniel Dai >Priority: Minor > Fix For: 0.6.0 > > Attachments: error_rerport.pig, myerrordata.txt, PIG-790-1.patch, > PIG-790-2.patch, pig_1240972895275.log > > > I have a simple Pig script which loads integer data and does a Bincond, where > it compares, (col1 eq ''). There is an error message that is generated in > this case, but it does not specify the line number in the script. > {code} > MYDATA = load '/user/viraj/myerrordata.txt' using PigStorage() as (col1:int, > col2:int); > MYDATA_PROJECT = FOREACH MYDATA GENERATE ((col1 eq '') ? 1 : 0) as newcol1, > ((col1 neq '') ? col1 - col2 : > 16) > as time_diff; > dump MYDATA_PROJECT; > {code} > == > 2009-04-29 02:33:07,182 [main] INFO > org.apache.pig.backend.hadoop.executionengine.HExecutionEngine - Connecting > to hadoop file system at: hdfs://localhost:9000 > 2009-04-29 02:33:08,584 [main] INFO > org.apache.pig.backend.hadoop.executionengine.HExecutionEngine - Connecting > to map-reduce job tracker at: localhost:9001 > 2009-04-29 02:33:08,836 [main] INFO org.apache.pig.PigServer - Create a new > graph. > 2009-04-29 02:33:10,040 [main] ERROR org.apache.pig.tools.grunt.Grunt - ERROR > 1039: Incompatible types in EqualTo Operator left hand side:int right hand > side:chararray > Details at logfile: /home/viraj/pig-svn/trunk/pig_1240972386081.log > == > It would be good if the error message has a line number and a copy of the > line in the script which is causing the problem. > Attaching data, script and log file. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1052) FINDBUGS: remaining performance warnings
[ https://issues.apache.org/jira/browse/PIG-1052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771102#action_12771102 ] Daniel Dai commented on PIG-1052: - +1 > FINDBUGS: remaining performance warnings > > > Key: PIG-1052 > URL: https://issues.apache.org/jira/browse/PIG-1052 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich >Assignee: Olga Natkovich > Attachments: PIG-1052.patch > > > SBSC Method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.Launcher.getStackTraceElement(String) > concatenates strings using + in a loop > SBSC Method org.apache.pig.impl.logicalLayer.LOCross.getSchema() > concatenates strings using + in a loop > SBSC Method org.apache.pig.impl.logicalLayer.LOForEach.getSchema() > concatenates strings using + in a loop > SBSC Method org.apache.pig.PigServer.locateJarFromResources(String) > concatenates strings using + in a loop > SBSC Method org.apache.pig.tools.parameters.ParseException.initialise(Token, > int[][], String[]) concatenates strings using + in a loop > SBSC Method > org.apache.pig.tools.parameters.PreprocessorContext.executeShellCommand(String) > concatenates strings using + in a loop > SSUnread field: > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.Launcher.OOM_ERR; > should this field be static? > SSUnread field: > org.apache.pig.impl.io.BufferedPositionedInputStream.bufSize; should this > field be static? > UPM Private method > org.apache.pig.impl.plan.optimizer.RulePlanPrinter.planString(List) is never > called > UPM Private method org.apache.pig.impl.plan.PlanPrinter.planString(List) is > never called > WMI Method org.apache.pig.builtin.PigStorage.putField(Object) makes > inefficient use of keySet iterator instead of entrySet iterator > WMI Method org.apache.pig.data.DataType.mapToString(Map) makes inefficient > use of keySet iterator instead of entrySet iterator > WMI Method org.apache.pig.impl.logicalLayer.LOCross.getSchema() makes > inefficient use of keySet iterator instead of entrySet iterator > WMI Method org.apache.pig.impl.logicalLayer.LOForEach.getSchema() makes > inefficient use of keySet iterator instead of entrySet iterator > WMI Method > org.apache.pig.impl.logicalLayer.validators.TypeCheckingVisitor.getLoadFuncSpec(Schema$FieldSchema, > String) makes inefficient use of keySet iterator instead of entrySet iterator > WMI Method > org.apache.pig.impl.plan.CompilationMessageCollector.logAggregate(Map, > CompilationMessageCollector$MessageType, Log) makes inefficient use of keySet > iterator instead of entrySet iterator > WMI Method org.apache.pig.StandAloneParser.tryParse(String) makes > inefficient use of keySet iterator instead of entrySet iterator -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (PIG-1059) FINDBUGS: remaining Bad practice + Multithreaded correctness Warning
FINDBUGS: remaining Bad practice + Multithreaded correctness Warning Key: PIG-1059 URL: https://issues.apache.org/jira/browse/PIG-1059 Project: Pig Issue Type: Improvement Reporter: Olga Natkovich IS Inconsistent synchronization of org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.hodConfDir; locked 66% of time IS Inconsistent synchronization of org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.hodProcess; locked 80% of time IS Inconsistent synchronization of org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.remoteHodConfDir; locked 88% of time IS Inconsistent synchronization of org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStream.initialized; locked 50% of time UG org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigHadoopLogger.getAggregate() is unsynchronized, org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigHadoopLogger.setAggregate(boolean) is synchronized UG org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigHadoopLogger.getReporter() is unsynchronized, org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigHadoopLogger.setReporter(Reporter) is synchronized BC Equals method for org.apache.pig.builtin.PigStorage assumes the argument is of type PigStorage BC Equals method for org.apache.pig.impl.streaming.StreamingCommand$HandleSpec assumes the argument is of type StreamingCommand$HandleSpec DP org.apache.pig.data.BagFactory.getInstance() creates a java.net.URLClassLoader classloader, which should be performed within a doPrivileged block DP org.apache.pig.data.TupleFactory.getInstance() creates a java.net.URLClassLoader classloader, which should be performed within a doPrivileged block DP org.apache.pig.impl.PigContext.createCl(String) creates a java.net.URLClassLoader classloader, which should be performed within a doPrivileged block DP org.apache.pig.impl.util.JarManager.createCl(String, PigContext) creates a java.net.URLClassLoader classloader, which should be performed within a doPrivileged block Eq org.apache.pig.data.DistinctDataBag$DistinctDataBagIterator$TContainer defines compareTo(DistinctDataBag$DistinctDataBagIterator$TContainer) and uses Object.equals() Eq org.apache.pig.data.SingleTupleBag defines compareTo(Object) and uses Object.equals() Eq org.apache.pig.data.SortedDataBag$SortedDataBagIterator$PQContainer defines compareTo(SortedDataBag$SortedDataBagIterator$PQContainer) and uses Object.equals() Eq org.apache.pig.data.TargetedTuple defines compareTo(Object) and uses Object.equals() HE org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan defines equals and uses Object.hashCode() HE org.apache.pig.backend.local.executionengine.physicalLayer.relationalOperators.POCogroup$groupComparator defines equals and uses Object.hashCode() HE org.apache.pig.builtin.BinaryStorage defines equals and uses Object.hashCode() HE org.apache.pig.builtin.BinStorage defines equals and uses Object.hashCode() HE org.apache.pig.builtin.PigStorage defines equals and uses Object.hashCode() HE org.apache.pig.data.InternalSortedBag$DefaultComparator defines equals and uses Object.hashCode() HE org.apache.pig.data.NonSpillableDataBag defines equals and uses Object.hashCode() HE org.apache.pig.data.SortedDataBag$DefaultComparator defines equals and uses Object.hashCode() HE org.apache.pig.impl.streaming.StreamingCommand$HandleSpec defines equals and uses Object.hashCode() Nm org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PhyPlanSetter.visitSplit(POSplit) doesn't override method in superclass because parameter type org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit doesn't match superclass parameter type org.apache.pig.backend.local.executionengine.physicalLayer.relationalOperators.POSplit Nm org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PhyPlanSetter.visitSplit(POSplit) doesn't override method in superclass because parameter type org.apache.pig.backend.local.executionengine.physicalLayer.relationalOperators.POSplit doesn't match superclass parameter type org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit RV org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.deleteLocalDir(File) ignores exceptional return value of java.io.File.delete() RV org.apache.pig.backend.local.datastorage.LocalPath.delete() ignores exceptional return value of java.io.File.delete() RV org.apache.pig.data.DefaultAbstractBag.clear() ignores exceptional return value of java.io.File.delete() RV org.apache.pig.data.DefaultAbstractBag.finalize()
[jira] Assigned: (PIG-1059) FINDBUGS: remaining Bad practice + Multithreaded correctness Warning
[ https://issues.apache.org/jira/browse/PIG-1059?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Olga Natkovich reassigned PIG-1059: --- Assignee: Olga Natkovich > FINDBUGS: remaining Bad practice + Multithreaded correctness Warning > > > Key: PIG-1059 > URL: https://issues.apache.org/jira/browse/PIG-1059 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich >Assignee: Olga Natkovich > > ISInconsistent synchronization of > org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.hodConfDir; > locked 66% of time > ISInconsistent synchronization of > org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.hodProcess; > locked 80% of time > ISInconsistent synchronization of > org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.remoteHodConfDir; > locked 88% of time > ISInconsistent synchronization of > org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStream.initialized; > locked 50% of time > UG > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigHadoopLogger.getAggregate() > is unsynchronized, > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigHadoopLogger.setAggregate(boolean) > is synchronized > UG > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigHadoopLogger.getReporter() > is unsynchronized, > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigHadoopLogger.setReporter(Reporter) > is synchronized > BCEquals method for org.apache.pig.builtin.PigStorage assumes the > argument is of type PigStorage > BCEquals method for > org.apache.pig.impl.streaming.StreamingCommand$HandleSpec assumes the > argument is of type StreamingCommand$HandleSpec > DPorg.apache.pig.data.BagFactory.getInstance() creates a > java.net.URLClassLoader classloader, which should be performed within a > doPrivileged block > DPorg.apache.pig.data.TupleFactory.getInstance() creates a > java.net.URLClassLoader classloader, which should be performed within a > doPrivileged block > DPorg.apache.pig.impl.PigContext.createCl(String) creates a > java.net.URLClassLoader classloader, which should be performed within a > doPrivileged block > DPorg.apache.pig.impl.util.JarManager.createCl(String, PigContext) > creates a java.net.URLClassLoader classloader, which should be performed > within a doPrivileged block > Eqorg.apache.pig.data.DistinctDataBag$DistinctDataBagIterator$TContainer > defines compareTo(DistinctDataBag$DistinctDataBagIterator$TContainer) and > uses Object.equals() > Eqorg.apache.pig.data.SingleTupleBag defines compareTo(Object) and uses > Object.equals() > Eqorg.apache.pig.data.SortedDataBag$SortedDataBagIterator$PQContainer > defines compareTo(SortedDataBag$SortedDataBagIterator$PQContainer) and uses > Object.equals() > Eqorg.apache.pig.data.TargetedTuple defines compareTo(Object) and uses > Object.equals() > HE > org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan > defines equals and uses Object.hashCode() > HE > org.apache.pig.backend.local.executionengine.physicalLayer.relationalOperators.POCogroup$groupComparator > defines equals and uses Object.hashCode() > HEorg.apache.pig.builtin.BinaryStorage defines equals and uses > Object.hashCode() > HEorg.apache.pig.builtin.BinStorage defines equals and uses > Object.hashCode() > HEorg.apache.pig.builtin.PigStorage defines equals and uses > Object.hashCode() > HEorg.apache.pig.data.InternalSortedBag$DefaultComparator defines equals > and uses Object.hashCode() > HEorg.apache.pig.data.NonSpillableDataBag defines equals and uses > Object.hashCode() > HEorg.apache.pig.data.SortedDataBag$DefaultComparator defines equals and > uses Object.hashCode() > HEorg.apache.pig.impl.streaming.StreamingCommand$HandleSpec defines > equals and uses Object.hashCode() > Nm > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PhyPlanSetter.visitSplit(POSplit) > doesn't override method in superclass because parameter type > org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit > doesn't match superclass parameter type > org.apache.pig.backend.local.executionengine.physicalLayer.relationalOperators.POSplit > Nm > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PhyPlanSetter.visitSplit(POSplit) > doesn't override method in superclass because parameter type > org.apache.pig.backend.local.executionengine.physicalLayer.relationalOperators.POSplit > doesn't match superclass parameter type > org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit > RV > org.apache.pig.backend.hadoop.executionengine.HExecutio
[jira] Assigned: (PIG-1052) FINDBUGS: remaining performance warnings
[ https://issues.apache.org/jira/browse/PIG-1052?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Olga Natkovich reassigned PIG-1052: --- Assignee: Olga Natkovich > FINDBUGS: remaining performance warnings > > > Key: PIG-1052 > URL: https://issues.apache.org/jira/browse/PIG-1052 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich >Assignee: Olga Natkovich > Attachments: PIG-1052.patch > > > SBSC Method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.Launcher.getStackTraceElement(String) > concatenates strings using + in a loop > SBSC Method org.apache.pig.impl.logicalLayer.LOCross.getSchema() > concatenates strings using + in a loop > SBSC Method org.apache.pig.impl.logicalLayer.LOForEach.getSchema() > concatenates strings using + in a loop > SBSC Method org.apache.pig.PigServer.locateJarFromResources(String) > concatenates strings using + in a loop > SBSC Method org.apache.pig.tools.parameters.ParseException.initialise(Token, > int[][], String[]) concatenates strings using + in a loop > SBSC Method > org.apache.pig.tools.parameters.PreprocessorContext.executeShellCommand(String) > concatenates strings using + in a loop > SSUnread field: > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.Launcher.OOM_ERR; > should this field be static? > SSUnread field: > org.apache.pig.impl.io.BufferedPositionedInputStream.bufSize; should this > field be static? > UPM Private method > org.apache.pig.impl.plan.optimizer.RulePlanPrinter.planString(List) is never > called > UPM Private method org.apache.pig.impl.plan.PlanPrinter.planString(List) is > never called > WMI Method org.apache.pig.builtin.PigStorage.putField(Object) makes > inefficient use of keySet iterator instead of entrySet iterator > WMI Method org.apache.pig.data.DataType.mapToString(Map) makes inefficient > use of keySet iterator instead of entrySet iterator > WMI Method org.apache.pig.impl.logicalLayer.LOCross.getSchema() makes > inefficient use of keySet iterator instead of entrySet iterator > WMI Method org.apache.pig.impl.logicalLayer.LOForEach.getSchema() makes > inefficient use of keySet iterator instead of entrySet iterator > WMI Method > org.apache.pig.impl.logicalLayer.validators.TypeCheckingVisitor.getLoadFuncSpec(Schema$FieldSchema, > String) makes inefficient use of keySet iterator instead of entrySet iterator > WMI Method > org.apache.pig.impl.plan.CompilationMessageCollector.logAggregate(Map, > CompilationMessageCollector$MessageType, Log) makes inefficient use of keySet > iterator instead of entrySet iterator > WMI Method org.apache.pig.StandAloneParser.tryParse(String) makes > inefficient use of keySet iterator instead of entrySet iterator -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (PIG-1058) FINDBUGS: remaining "Correctness Warnings"
FINDBUGS: remaining "Correctness Warnings" -- Key: PIG-1058 URL: https://issues.apache.org/jira/browse/PIG-1058 Project: Pig Issue Type: Improvement Reporter: Olga Natkovich Assignee: Olga Natkovich BC Impossible cast from java.lang.Object[] to java.lang.String[] in org.apache.pig.PigServer.listPaths(String) EC Call to equals() comparing different types in org.apache.pig.impl.plan.Operator.equals(Object) GC java.lang.Byte is incompatible with expected argument type java.lang.Integer in org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.POPackageAnnotator$LoRearrangeDiscoverer.visitLocalRearrange(POLocalRearrange) IL There is an apparent infinite recursive loop in org.apache.pig.backend.local.executionengine.physicalLayer.relationalOperators.POCogroup$groupComparator.equals(Object) INT Bad comparison of nonnegative value with -1 in org.apache.tools.bzip2r.CBZip2InputStream.bsR(int) INT Bad comparison of nonnegative value with -1 in org.apache.tools.bzip2r.CBZip2InputStream.getAndMoveToFrontDecode() INT Bad comparison of nonnegative value with -1 in org.apache.tools.bzip2r.CBZip2InputStream.getAndMoveToFrontDecode() MF Field ConstantExpression.res masks field in superclass org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator Nm org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler.visitSplit(POSplit) doesn't override method in superclass because parameter type org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit doesn't match superclass parameter type org.apache.pig.backend.local.executionengine.physicalLayer.relationalOperators.POSplit Nm org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.NoopStoreRemover$PhysicalRemover.visitSplit(POSplit) doesn't override method in superclass because parameter type org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit doesn't match superclass parameter type org.apache.pig.backend.local.executionengine.physicalLayer.relationalOperators.POSplit NP Possible null pointer dereference of ? in org.apache.pig.impl.logicalLayer.optimizer.PushDownForeachFlatten.check(List) NP Possible null pointer dereference of lo in org.apache.pig.impl.logicalLayer.optimizer.StreamOptimizer.transform(List) NP Possible null pointer dereference of Schema$FieldSchema.Schema$FieldSchema.alias in org.apache.pig.impl.logicalLayer.schema.Schema.equals(Schema, Schema, boolean, boolean) NP Possible null pointer dereference of Schema$FieldSchema.alias in org.apache.pig.impl.logicalLayer.schema.Schema$FieldSchema.equals(Schema$FieldSchema, Schema$FieldSchema, boolean, boolean) NP Possible null pointer dereference of inp in org.apache.pig.impl.streaming.ExecutableManager$ProcessInputThread.run() RCN Nullcheck of pigContext at line 123 of value previously dereferenced in org.apache.pig.impl.util.JarManager.createJar(OutputStream, List, PigContext) RV org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.fixUpDomain(String, Properties) ignores return value of java.net.InetAddress.getByName(String) RV Bad attempt to compute absolute value of signed 32-bit hashcode in org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.SkewedPartitioner.getPartition(PigNullableWritable, Writable, int) RV Bad attempt to compute absolute value of signed 32-bit hashcode in org.apache.pig.impl.plan.DotPlanDumper.getID(Operator) UwF Field only ever set to null: org.apache.pig.impl.builtin.MergeJoinIndexer.dummyTuple -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Assigned: (PIG-1055) FINDBUGS: remaining "Dodgy Warnings"
[ https://issues.apache.org/jira/browse/PIG-1055?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Olga Natkovich reassigned PIG-1055: --- Assignee: Olga Natkovich > FINDBUGS: remaining "Dodgy Warnings" > > > Key: PIG-1055 > URL: https://issues.apache.org/jira/browse/PIG-1055 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich >Assignee: Olga Natkovich > Attachments: PIG-1055.patch > > > BCQuestionable cast from java.util.List to java.util.ArrayList in new > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit(PigContext, > FileSystem, Path, String, List, long, long) > Eqorg.apache.pig.data.AmendableTuple doesn't override > DefaultTuple.equals(Object) > Eqorg.apache.pig.data.TimestampedTuple doesn't override > DefaultTuple.equals(Object) > IAAmbiguous invocation of either an outer or inherited method > org.apache.pig.impl.plan.DotPlanDumper.getName(Operator) in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.DotMRPrinter$InnerPrinter.getAttributes(DotMRPrinter$InnerOperator) > IMComputation of average could overflow in > org.apache.tools.bzip2r.CBZip2OutputStream.qSort3(int, int, int) > IMCheck for oddness that won't work for negative numbers in > org.apache.tools.bzip2r.CBZip2OutputStream.sendMTFValues() > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.doHod(String, > Properties) > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.CombinerOptimizer.visitMROp(MapReduceOper) > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler.visitDistinct(PODistinct) > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler.visitFRJoin(POFRJoin) > REC Exception is caught when Exception is not thrown in > org.apache.pig.impl.logicalLayer.optimizer.OpLimitOptimizer.processNode(LOLimit) > REC Exception is caught when Exception is not thrown in > org.apache.pig.tools.streams.StreamGenerator.actionPerformed(ActionEvent) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner.sJobConf > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner$Combine.configure(JobConf) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.activeSplit > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.getRecordReader(InputSplit, > JobConf, Reporter) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.sJob > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.getRecordReader(InputSplit, > JobConf, Reporter) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce.sJobConf > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.configure(JobConf) > STWrite to static field org.apache.pig.data.BagFactory.gMemMgr from > instance method new org.apache.pig.data.BagFactory() > STWrite to static field > org.apache.pig.impl.logicalLayer.LogicalPlanCloneHelper.mOpToCloneMap from > instance method new > org.apache.pig.impl.logicalLayer.LogicalPlanCloneHelper(LogicalPlan, Map) > STWrite to static field > org.apache.pig.impl.logicalLayer.LogicalPlanBuilder.classloader from instance > method org.apache.pig.impl.PigContext.addJar(URL) -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Assigned: (PIG-1051) FINFBUGS: NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE: Possible null pointer dereference due to return value of called method
[ https://issues.apache.org/jira/browse/PIG-1051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Olga Natkovich reassigned PIG-1051: --- Assignee: Olga Natkovich > FINFBUGS: NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE: Possible null pointer > dereference due to return value of called method > > > Key: PIG-1051 > URL: https://issues.apache.org/jira/browse/PIG-1051 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich >Assignee: Olga Natkovich > Attachments: PIG-1051.patch > > > NPPossible null pointer dereference in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.CountingMap.put(Object, > Integer) due to return value of called method > NPLoad of known null value in > org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan.clone() > NPLoad of known null value in > org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan.clone() > NPLoad of known null value in > org.apache.pig.impl.logicalLayer.optimizer.OpLimitOptimizer.check(List) > NPLoad of known null value in > org.apache.pig.impl.logicalLayer.optimizer.PushDownForeachFlatten.getOperator(List) > NPLoad of known null value in > org.apache.pig.impl.logicalLayer.optimizer.PushUpFilter.getOperator(List) > NPLoad of known null value in > org.apache.pig.impl.logicalLayer.optimizer.StreamOptimizer.check(List) > NPLoad of known null value in > org.apache.pig.impl.logicalLayer.optimizer.TypeCastInserter.getOperator(List) > NPLoad of known null value in > org.apache.pig.impl.logicalLayer.optimizer.TypeCastInserter.getOperator(List) > NPLoad of known null value in > org.apache.pig.impl.logicalLayer.schema.Schema.mergeSchema(Schema, Schema, > boolean, boolean, boolean) > NPLoad of known null value in > org.apache.pig.impl.logicalLayer.schema.Schema.mergeSchema(Schema, Schema, > boolean, boolean, boolean) > NPPossible null pointer dereference in > org.apache.pig.impl.util.LineageTracer.getWeightedCounts(IdentityHashSet, > int) due to return value of called method > NPPossible null pointer dereference in > org.apache.pig.impl.util.LineageTracer.getWeightedCounts(IdentityHashSet, > int) due to return value of called method > NPPossible null pointer dereference in > org.apache.pig.impl.util.LineageTracer.insert(Tuple) due to return value of > called method > NPPossible null pointer dereference in > org.apache.pig.impl.util.LineageTracer.link(Tuple, Tuple) due to return value > of called method > NPPossible null pointer dereference in > org.apache.pig.impl.util.LineageTracer.link(Tuple, Tuple) due to return value > of called method > NPPossible null pointer dereference in > org.apache.pig.pen.LineageTrimmingVisitor.PruneBaseDataConstrainedCoverage(Map, > DataBag, LineageTracer, Map) due to return value of called method > NPPossible null pointer dereference in > org.apache.pig.pen.LineageTrimmingVisitor.PruneBaseDataConstrainedCoverage(Map, > DataBag, LineageTracer, Map) due to return value of called method > NPPossible null pointer dereference in > org.apache.pig.pen.LineageTrimmingVisitor.PruneBaseDataConstrainedCoverage(Map, > DataBag, LineageTracer, Map) due to return value of called method > NPPossible null pointer dereference in > org.apache.pig.pen.LineageTrimmingVisitor.PruneBaseDataConstrainedCoverage(Map, > DataBag, LineageTracer, Map) due to return value of called method > NPPossible null pointer dereference in > org.apache.pig.pen.util.LineageTracer.getWeightedCounts(float, float) due to > return value of called method > NPPossible null pointer dereference in > org.apache.pig.pen.util.LineageTracer.getWeightedCounts(float, float) due to > return value of called method > NPPossible null pointer dereference in > org.apache.pig.pen.util.LineageTracer.insert(Tuple) due to return value of > called method > NPPossible null pointer dereference in > org.apache.pig.pen.util.LineageTracer.link(Tuple, Tuple) due to return value > of called method > NPPossible null pointer dereference in > org.apache.pig.pen.util.LineageTracer.link(Tuple, Tuple) due to return value > of called method > NPPossible null pointer dereference in > org.apache.pig.StandAloneParser.main(String[]) due to return value of called > method -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1055) FINDBUGS: remaining "Dodgy Warnings"
[ https://issues.apache.org/jira/browse/PIG-1055?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Olga Natkovich updated PIG-1055: Status: Patch Available (was: Open) > FINDBUGS: remaining "Dodgy Warnings" > > > Key: PIG-1055 > URL: https://issues.apache.org/jira/browse/PIG-1055 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich > Attachments: PIG-1055.patch > > > BCQuestionable cast from java.util.List to java.util.ArrayList in new > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit(PigContext, > FileSystem, Path, String, List, long, long) > Eqorg.apache.pig.data.AmendableTuple doesn't override > DefaultTuple.equals(Object) > Eqorg.apache.pig.data.TimestampedTuple doesn't override > DefaultTuple.equals(Object) > IAAmbiguous invocation of either an outer or inherited method > org.apache.pig.impl.plan.DotPlanDumper.getName(Operator) in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.DotMRPrinter$InnerPrinter.getAttributes(DotMRPrinter$InnerOperator) > IMComputation of average could overflow in > org.apache.tools.bzip2r.CBZip2OutputStream.qSort3(int, int, int) > IMCheck for oddness that won't work for negative numbers in > org.apache.tools.bzip2r.CBZip2OutputStream.sendMTFValues() > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.doHod(String, > Properties) > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.CombinerOptimizer.visitMROp(MapReduceOper) > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler.visitDistinct(PODistinct) > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler.visitFRJoin(POFRJoin) > REC Exception is caught when Exception is not thrown in > org.apache.pig.impl.logicalLayer.optimizer.OpLimitOptimizer.processNode(LOLimit) > REC Exception is caught when Exception is not thrown in > org.apache.pig.tools.streams.StreamGenerator.actionPerformed(ActionEvent) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner.sJobConf > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner$Combine.configure(JobConf) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.activeSplit > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.getRecordReader(InputSplit, > JobConf, Reporter) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.sJob > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.getRecordReader(InputSplit, > JobConf, Reporter) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce.sJobConf > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.configure(JobConf) > STWrite to static field org.apache.pig.data.BagFactory.gMemMgr from > instance method new org.apache.pig.data.BagFactory() > STWrite to static field > org.apache.pig.impl.logicalLayer.LogicalPlanCloneHelper.mOpToCloneMap from > instance method new > org.apache.pig.impl.logicalLayer.LogicalPlanCloneHelper(LogicalPlan, Map) > STWrite to static field > org.apache.pig.impl.logicalLayer.LogicalPlanBuilder.classloader from instance > method org.apache.pig.impl.PigContext.addJar(URL) -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1055) FINDBUGS: remaining "Dodgy Warnings"
[ https://issues.apache.org/jira/browse/PIG-1055?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Olga Natkovich updated PIG-1055: Attachment: PIG-1055.patch > FINDBUGS: remaining "Dodgy Warnings" > > > Key: PIG-1055 > URL: https://issues.apache.org/jira/browse/PIG-1055 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich > Attachments: PIG-1055.patch > > > BCQuestionable cast from java.util.List to java.util.ArrayList in new > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit(PigContext, > FileSystem, Path, String, List, long, long) > Eqorg.apache.pig.data.AmendableTuple doesn't override > DefaultTuple.equals(Object) > Eqorg.apache.pig.data.TimestampedTuple doesn't override > DefaultTuple.equals(Object) > IAAmbiguous invocation of either an outer or inherited method > org.apache.pig.impl.plan.DotPlanDumper.getName(Operator) in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.DotMRPrinter$InnerPrinter.getAttributes(DotMRPrinter$InnerOperator) > IMComputation of average could overflow in > org.apache.tools.bzip2r.CBZip2OutputStream.qSort3(int, int, int) > IMCheck for oddness that won't work for negative numbers in > org.apache.tools.bzip2r.CBZip2OutputStream.sendMTFValues() > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.doHod(String, > Properties) > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.CombinerOptimizer.visitMROp(MapReduceOper) > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler.visitDistinct(PODistinct) > REC Exception is caught when Exception is not thrown in > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler.visitFRJoin(POFRJoin) > REC Exception is caught when Exception is not thrown in > org.apache.pig.impl.logicalLayer.optimizer.OpLimitOptimizer.processNode(LOLimit) > REC Exception is caught when Exception is not thrown in > org.apache.pig.tools.streams.StreamGenerator.actionPerformed(ActionEvent) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner.sJobConf > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner$Combine.configure(JobConf) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.activeSplit > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.getRecordReader(InputSplit, > JobConf, Reporter) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.sJob > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat.getRecordReader(InputSplit, > JobConf, Reporter) > STWrite to static field > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce.sJobConf > from instance method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.configure(JobConf) > STWrite to static field org.apache.pig.data.BagFactory.gMemMgr from > instance method new org.apache.pig.data.BagFactory() > STWrite to static field > org.apache.pig.impl.logicalLayer.LogicalPlanCloneHelper.mOpToCloneMap from > instance method new > org.apache.pig.impl.logicalLayer.LogicalPlanCloneHelper(LogicalPlan, Map) > STWrite to static field > org.apache.pig.impl.logicalLayer.LogicalPlanBuilder.classloader from instance > method org.apache.pig.impl.PigContext.addJar(URL) -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1001) Generate more meaningful error message when one input file does not exist
[ https://issues.apache.org/jira/browse/PIG-1001?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771072#action_12771072 ] Alan Gates commented on PIG-1001: - I have a question on this code in JobControlCompiler.updateMROpPlan: {code} for (Job job : completeFailedJobs) // remove all subsequent jobs { List fifo = new ArrayList(); fifo.add(jobMroMap.get(job)); while (!fifo.isEmpty()) { MapReduceOper mro = fifo.remove(0); List succs = plan.getSuccessors(mro); if (succs != null) fifo.addAll(succs); plan.remove(mro); numRemoved++; } } {code} If we have a dependency graph like: A->B->C and A fails, won't the above code only remove B and not C? OperatorPlan.getSuccessors() only gets immediate successors, not all successors. I think you want OperatorPlan.trimBelow() instead. > Generate more meaningful error message when one input file does not exist > - > > Key: PIG-1001 > URL: https://issues.apache.org/jira/browse/PIG-1001 > Project: Pig > Issue Type: Bug >Affects Versions: 0.4.0 >Reporter: Daniel Dai > Fix For: 0.6.0 > > Attachments: PIG-1001-1.patch > > > In the following query, if 1.txt does not exist, > a = load '1.txt'; > b = group a by $0; > c = group b all; > dump c; > Pig throws error message "ERROR 2100: file:/tmp/temp155054664/tmp1144108421 > does not exist.", Pig should deal with it with the error message "Input file > 1.txt not exist" instead of those confusing messages. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-746) Works in --exectype local, fails on grid - ERROR 2113: SingleTupleBag should never be serialized
[ https://issues.apache.org/jira/browse/PIG-746?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Richard Ding updated PIG-746: - Attachment: PIG-746.patch As Pradeep suggested, the combiner optimizer code in this patch identifies projection of bags in the foreach following the group and in such cases decides not to use the combiner. > Works in --exectype local, fails on grid - ERROR 2113: SingleTupleBag should > never be serialized > > > Key: PIG-746 > URL: https://issues.apache.org/jira/browse/PIG-746 > Project: Pig > Issue Type: Bug >Reporter: David Ciemiewicz >Assignee: Richard Ding > Attachments: PIG-746.patch > > > The script below works on Pig 2.0 local mode but fails when I run the same > program on the grid. > I was attempting to create a workaround for PIG-710. > Here's the error: > {code} > Caused by: org.apache.pig.backend.executionengine.ExecException: ERROR 2113: > SingleTupleBag should never be serialized > or serialized. > at org.apache.pig.data.SingleTupleBag.write(SingleTupleBag.java:129) > at > org.apache.pig.data.DataReaderWriter.writeDatum(DataReaderWriter.java:147) > at org.apache.pig.data.DefaultTuple.write(DefaultTuple.java:291) > at > org.apache.pig.impl.io.PigNullableWritable.write(PigNullableWritable.java:83) > at > org.apache.hadoop.io.serializer.WritableSerialization$WritableSerializer.serialize(WritableSerialization.java:90) > at > org.apache.hadoop.io.serializer.WritableSerialization$WritableSerializer.serialize(WritableSerialization.java:77) > at > org.apache.hadoop.mapred.MapTask$MapOutputBuffer.collect(MapTask.java:439) > at > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce$Map.collect(PigMapReduce.java:101) > at > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.runPipeline(PigMapBase.java:219) > at > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.map(PigMapBase.java:208) > at > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce$Map.map(PigMapReduce.java:86) > at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:47) > at org.apache.hadoop.mapred.MapTask.run(MapTask.java:227) > {code} > Here's the program: > {code} > A = load 'filterbug.data' using PigStorage() as ( id, str ); > A = foreach A generate > id, > str, > ( > str matches 'hello' or > str matches 'hello' > ? 1 : 0 > ) as matched; > describe A; > B = group A by ( id ); > describe B; > D = foreach B generate > group, > SUM(A.matched) as matchedcount, > A; > describe D; > E = filter D by matchedcount > 0; > describe E; > F = foreach E generate > FLATTEN(A); > describe F; > dump F; > {code} > Here's the data filterbug.data > {code} > a hello > a goodbye > b goodbye > c hello > c hello > c hello > e what > {code} > -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-746) Works in --exectype local, fails on grid - ERROR 2113: SingleTupleBag should never be serialized
[ https://issues.apache.org/jira/browse/PIG-746?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Richard Ding updated PIG-746: - Status: Patch Available (was: Open) > Works in --exectype local, fails on grid - ERROR 2113: SingleTupleBag should > never be serialized > > > Key: PIG-746 > URL: https://issues.apache.org/jira/browse/PIG-746 > Project: Pig > Issue Type: Bug >Reporter: David Ciemiewicz >Assignee: Richard Ding > Attachments: PIG-746.patch > > > The script below works on Pig 2.0 local mode but fails when I run the same > program on the grid. > I was attempting to create a workaround for PIG-710. > Here's the error: > {code} > Caused by: org.apache.pig.backend.executionengine.ExecException: ERROR 2113: > SingleTupleBag should never be serialized > or serialized. > at org.apache.pig.data.SingleTupleBag.write(SingleTupleBag.java:129) > at > org.apache.pig.data.DataReaderWriter.writeDatum(DataReaderWriter.java:147) > at org.apache.pig.data.DefaultTuple.write(DefaultTuple.java:291) > at > org.apache.pig.impl.io.PigNullableWritable.write(PigNullableWritable.java:83) > at > org.apache.hadoop.io.serializer.WritableSerialization$WritableSerializer.serialize(WritableSerialization.java:90) > at > org.apache.hadoop.io.serializer.WritableSerialization$WritableSerializer.serialize(WritableSerialization.java:77) > at > org.apache.hadoop.mapred.MapTask$MapOutputBuffer.collect(MapTask.java:439) > at > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce$Map.collect(PigMapReduce.java:101) > at > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.runPipeline(PigMapBase.java:219) > at > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.map(PigMapBase.java:208) > at > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce$Map.map(PigMapReduce.java:86) > at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:47) > at org.apache.hadoop.mapred.MapTask.run(MapTask.java:227) > {code} > Here's the program: > {code} > A = load 'filterbug.data' using PigStorage() as ( id, str ); > A = foreach A generate > id, > str, > ( > str matches 'hello' or > str matches 'hello' > ? 1 : 0 > ) as matched; > describe A; > B = group A by ( id ); > describe B; > D = foreach B generate > group, > SUM(A.matched) as matchedcount, > A; > describe D; > E = filter D by matchedcount > 0; > describe E; > F = foreach E generate > FLATTEN(A); > describe F; > dump F; > {code} > Here's the data filterbug.data > {code} > a hello > a goodbye > b goodbye > c hello > c hello > c hello > e what > {code} > -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (PIG-1057) [Zebra] Zebra does not support concurrent deletions of column groups now.
[Zebra] Zebra does not support concurrent deletions of column groups now. - Key: PIG-1057 URL: https://issues.apache.org/jira/browse/PIG-1057 Project: Pig Issue Type: Bug Affects Versions: 0.4.0 Reporter: Chao Wang Assignee: Chao Wang Fix For: 0.6.0 Zebra does not support concurrent deletions of column groups now. As a result, the TestDropColumnGroup testcase can sometimes fail due to this. In this testcase, multiple threads will be launched together, with each one deleting one particular column group. The following exception can be thrown (with callstack): /*/ ... java.io.FileNotFoundException: File /.../pig-trunk/build/contrib/zebra/test/data/DropCGTest/CG02 does not exist. at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:361) at org.apache.hadoop.fs.RawLocalFileSystem.listStatus(RawLocalFileSystem.java:290) at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:716) at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:741) at org.apache.hadoop.fs.ChecksumFileSystem.listStatus(ChecksumFileSystem.java:465) at org.apache.hadoop.zebra.io.BasicTable$SchemaFile.setCGDeletedFlags(BasicTable.java:1610) at org.apache.hadoop.zebra.io.BasicTable$SchemaFile.readSchemaFile(BasicTable.java:1593) at org.apache.hadoop.zebra.io.BasicTable$SchemaFile.(BasicTable.java:1416) at org.apache.hadoop.zebra.io.BasicTable.dropColumnGroup(BasicTable.java:133) at org.apache.hadoop.zebra.io.TestDropColumnGroup$DropThread.run(TestDropColumnGroup.java:772) ... /*/ We plan to fix this in Zebra to support concurrent deletions of column groups. The root cause is that a thread or process reads in some stale file system information (e.g., it sees /CG0 first) and then can fail later on (it tries to access /CG0, however /CG0 may be deleted by another thread or process). Therefore, we plan to adopt a retry logic to resolve this issue. More detailed, we allow a dropping column group thread to retry n times when doing its deleting job - n is the total number of column groups. Note that here we do NOT try to resolve the more general concurrent column group deletions + reads issue. If a process is reading some data that could be deleted by another process, it can fail as we expect. Here we only try to resolve the concurrent column group deletions issue. If you have multiple threads or processes to delete column groups, they should succeed. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1052) FINDBUGS: remaining performance warnings
[ https://issues.apache.org/jira/browse/PIG-1052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12771040#action_12771040 ] Hadoop QA commented on PIG-1052: -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12423381/PIG-1052.patch against trunk revision 830335. +1 @author. The patch does not contain any @author tags. -1 tests included. The patch doesn't appear to include any new or modified tests. Please justify why no tests are needed for this patch. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. +1 core tests. The patch passed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/122/testReport/ Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/122/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/122/console This message is automatically generated. > FINDBUGS: remaining performance warnings > > > Key: PIG-1052 > URL: https://issues.apache.org/jira/browse/PIG-1052 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich > Attachments: PIG-1052.patch > > > SBSC Method > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.Launcher.getStackTraceElement(String) > concatenates strings using + in a loop > SBSC Method org.apache.pig.impl.logicalLayer.LOCross.getSchema() > concatenates strings using + in a loop > SBSC Method org.apache.pig.impl.logicalLayer.LOForEach.getSchema() > concatenates strings using + in a loop > SBSC Method org.apache.pig.PigServer.locateJarFromResources(String) > concatenates strings using + in a loop > SBSC Method org.apache.pig.tools.parameters.ParseException.initialise(Token, > int[][], String[]) concatenates strings using + in a loop > SBSC Method > org.apache.pig.tools.parameters.PreprocessorContext.executeShellCommand(String) > concatenates strings using + in a loop > SSUnread field: > org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.Launcher.OOM_ERR; > should this field be static? > SSUnread field: > org.apache.pig.impl.io.BufferedPositionedInputStream.bufSize; should this > field be static? > UPM Private method > org.apache.pig.impl.plan.optimizer.RulePlanPrinter.planString(List) is never > called > UPM Private method org.apache.pig.impl.plan.PlanPrinter.planString(List) is > never called > WMI Method org.apache.pig.builtin.PigStorage.putField(Object) makes > inefficient use of keySet iterator instead of entrySet iterator > WMI Method org.apache.pig.data.DataType.mapToString(Map) makes inefficient > use of keySet iterator instead of entrySet iterator > WMI Method org.apache.pig.impl.logicalLayer.LOCross.getSchema() makes > inefficient use of keySet iterator instead of entrySet iterator > WMI Method org.apache.pig.impl.logicalLayer.LOForEach.getSchema() makes > inefficient use of keySet iterator instead of entrySet iterator > WMI Method > org.apache.pig.impl.logicalLayer.validators.TypeCheckingVisitor.getLoadFuncSpec(Schema$FieldSchema, > String) makes inefficient use of keySet iterator instead of entrySet iterator > WMI Method > org.apache.pig.impl.plan.CompilationMessageCollector.logAggregate(Map, > CompilationMessageCollector$MessageType, Log) makes inefficient use of keySet > iterator instead of entrySet iterator > WMI Method org.apache.pig.StandAloneParser.tryParse(String) makes > inefficient use of keySet iterator instead of entrySet iterator -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1030) explain and dump not working with two UDFs inside inner plan of foreach
[ https://issues.apache.org/jira/browse/PIG-1030?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Richard Ding updated PIG-1030: -- Attachment: PIG-1030.patch The problem is that the code assums to combine only in the case where there is only one PODistinct which is the only input to an aggregate function. This patch disables the combiner if an aggregate function in foreach statements have multiple inputs and one of them is PODistinct. > explain and dump not working with two UDFs inside inner plan of foreach > --- > > Key: PIG-1030 > URL: https://issues.apache.org/jira/browse/PIG-1030 > Project: Pig > Issue Type: Bug >Reporter: Ying He >Assignee: Richard Ding > Attachments: PIG-1030.patch > > > this scprit does not work > register /homes/yinghe/owl/string.jar; > a = load '/user/yinghe/a.txt' as (id, color); > b = group a all; > c = foreach b { > d = distinct a.color; > generate group, string.BagCount2(d), string.ColumnLen2(d, 0); > } > the udfs are regular, not algebraic. > then if I call "dump c;" or "explain c", I would get this error message. > ERROR org.apache.pig.tools.grunt.Grunt - ERROR 2019: Expected to find plan > with single leaf. Found 2 leaves. > The error only occurs for the first time, after getting this error, if I call > "dump c" or "explain c" again, it would succeed. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1030) explain and dump not working with two UDFs inside inner plan of foreach
[ https://issues.apache.org/jira/browse/PIG-1030?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Richard Ding updated PIG-1030: -- Status: Patch Available (was: Open) > explain and dump not working with two UDFs inside inner plan of foreach > --- > > Key: PIG-1030 > URL: https://issues.apache.org/jira/browse/PIG-1030 > Project: Pig > Issue Type: Bug >Reporter: Ying He >Assignee: Richard Ding > Attachments: PIG-1030.patch > > > this scprit does not work > register /homes/yinghe/owl/string.jar; > a = load '/user/yinghe/a.txt' as (id, color); > b = group a all; > c = foreach b { > d = distinct a.color; > generate group, string.BagCount2(d), string.ColumnLen2(d, 0); > } > the udfs are regular, not algebraic. > then if I call "dump c;" or "explain c", I would get this error message. > ERROR org.apache.pig.tools.grunt.Grunt - ERROR 2019: Expected to find plan > with single leaf. Found 2 leaves. > The error only occurs for the first time, after getting this error, if I call > "dump c" or "explain c" again, it would succeed. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-598) Parameter substitution ($PARAMETER) should not be performed in comments
[ https://issues.apache.org/jira/browse/PIG-598?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Thejas M Nair updated PIG-598: -- Status: Patch Available (was: Open) > Parameter substitution ($PARAMETER) should not be performed in comments > --- > > Key: PIG-598 > URL: https://issues.apache.org/jira/browse/PIG-598 > Project: Pig > Issue Type: Bug > Components: impl >Affects Versions: 0.2.0 >Reporter: David Ciemiewicz >Assignee: Thejas M Nair > Attachments: PIG-598.patch > > > Compiling the following code example will generate an error that > $NOT_A_PARAMETER is an Undefined Parameter. > This is problematic as sometimes you want to comment out parts of your code, > including parameters so that you don't have to define them. > This I think it would be really good if parameter substitution was not > performed in comments. > {code} > -- $NOT_A_PARAMETER > {code} > {code} > -bash-3.00$ pig -exectype local -latest comment.pig > USING: /grid/0/gs/pig/current > java.lang.RuntimeException: Undefined parameter : NOT_A_PARAMETER > at > org.apache.pig.tools.parameters.PreprocessorContext.substitute(PreprocessorContext.java:221) > at > org.apache.pig.tools.parameters.ParameterSubstitutionPreprocessor.parsePigFile(ParameterSubstitutionPreprocessor.java:106) > at > org.apache.pig.tools.parameters.ParameterSubstitutionPreprocessor.genSubstitutedFile(ParameterSubstitutionPreprocessor.java:86) > at org.apache.pig.Main.runParamPreprocessor(Main.java:394) > at org.apache.pig.Main.main(Main.java:296) > {code} -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
RE: Custom Loadfunc problem!
I think that you are right. But the most weird thing is that there is no LOG outputed. I can see that the last method invoked was getLocations. And anything after that was not recorded anywhere. Even in the pseudo-distributed mode of Hadoop, if I use System.out.println, the stdout under userlogs should contain a job's log, right? But that did not happen after getLocations. It drove me crazy. I have no idea of if an important method like PigInputFormat.getRecordReader has ever been invoked. Please enlighten me! Thanks, Richard > Date: Wed, 28 Oct 2009 10:57:54 -0400 > Subject: Re: Custom Loadfunc problem! > From: dvrya...@gmail.com > To: pig-dev@hadoop.apache.org > > Richard, > I would strongly advise you against trying to mess with PigInputFormat > and JobControlCompiler. We know that slicers work (after all, people > manage to write hbase loaders and such). Plus the whole > input/output/slice/slicer thing is being revised right now, there is a > proposal in PIG-966 you can look at if you would like to comment. > > In fact I just took your code and substituted some placeholders for > various methods and classes (like HSSchema, etc), and I can see Slice > methods getting called and everything. > > What do your location strings look like? Pig tries to be helpful by > transforming locations into HDFS paths if the locations are not > qualified by a scheme ( something:/ ). This makes file-based LoadFuncs > easier to write, but it also means that you have to specify a scheme > if you are doing something that's not file-based. Try changing your > code to expect the location string to start with some scheme, such as > abcd:/ (or perhaps something more meaningful) and things should work. > > -D > > 2009/10/28 RichardGUO Fei : > > > > It seems that the real problem is JobControlCompiler.getJob, because this > > method removes POLoad and sets PigInputFormat as the input format but the > > path passed to my storage is actually a table name rather than a file path! > > I think that the LoadFunc should be more flexible. I am trying to fix this > > at this moment. > > > > > > > > Thanks, > > > > Richard > > > >> From: zjf...@gmail.com > >> To: pig-dev@hadoop.apache.org > >> Subject: RE: Custom Loadfunc problem! > >> Date: Tue, 27 Oct 2009 23:40:43 -0800 > >> > >> > >> I mean hadoop's local mode not pig's own local mode > >> > >> > >> -Original Message- > >> From: Dmitriy Ryaboy [mailto:dvrya...@gmail.com] > >> Sent: 2009年10月26日 6:33 > >> To: pig-dev@hadoop.apache.org; pig-dev@hadoop.apache.org > >> Subject: RE: Custom Loadfunc problem! > >> > >> Jeff, > >> Slicers dont work in local mode, there is an ancient ticket for that on the > >> Jira. > >> > >> Richard -- hard to say whats going on without more code. Think you can come > >> up with a simplified version of your loadfunc that fails in a similar > >> manner, and share it? > >> > >> > >> > >> -Original Message- > >> From: "zjffdu" > >> To: pig-dev@hadoop.apache.org > >> Sent: 10/27/2009 1:45 AM > >> Subject: RE: Custom Loadfunc problem! > >> > >> Illustrate will not execute the job, while dump and store will execute it. > >> So I think there must be something wrong with your custom slicer. I suggest > >> you set breakpoint in your slicer and debug it in map reduce mode locally > >> > >> > >> > >> -Original Message- > >> From: RichardGUO Fei [mailto:gladiato...@hotmail.com] > >> Sent: 20091026 0:43 > >> To: pig-dev@hadoop.apache.org > >> Subject: RE: Custom Loadfunc problem! > >> > >> > >> Hi, > >> > >> > >> > >> Btw, my program works with ILLUSTRATE but n > >> > >> [truncated by sender] > >> > > > > _ > > MSN十周年庆典,查看MSN注册时间,赢取神秘大奖 > > http://10.msn.com.cn _ 约会说不清地方?来试试微软地图最新msn互动功能! http://ditu.live.com/?form=TL&swm=1
[jira] Updated: (PIG-1037) better memory layout and spill for sorted and distinct bags
[ https://issues.apache.org/jira/browse/PIG-1037?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alan Gates updated PIG-1037: Resolution: Fixed Fix Version/s: 0.6.0 Status: Resolved (was: Patch Available) Patch checked in. Thanks Ying. > better memory layout and spill for sorted and distinct bags > --- > > Key: PIG-1037 > URL: https://issues.apache.org/jira/browse/PIG-1037 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich >Assignee: Ying He > Fix For: 0.6.0 > > Attachments: PIG-1037.patch, PIG-1037.patch2, PIG-1037.patch3 > > -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-970) Support of HBase 0.20.0
[ https://issues.apache.org/jira/browse/PIG-970?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12770974#action_12770974 ] Alan Gates commented on PIG-970: I haven't been able to get the unit test to pass in my environment. > Support of HBase 0.20.0 > --- > > Key: PIG-970 > URL: https://issues.apache.org/jira/browse/PIG-970 > Project: Pig > Issue Type: Improvement > Components: impl >Affects Versions: 0.3.0 >Reporter: Vincent BARAT > Attachments: build.xml.path, pig-hbase-0.20.0-support.patch, > pig-hbase-20-v2.patch, TEST-org.apache.pig.test.TestHBaseStorage.txt, > TEST-org.apache.pig.test.TestHBaseStorage.txt > > > The support of HBase is currently very limited and restricted to HBase 0.18.0. > Because the next releases of PIG will support Hadoop 0.20.0, they should also > support HBase 0.20.0. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
Re: Custom Loadfunc problem!
Richard, I would strongly advise you against trying to mess with PigInputFormat and JobControlCompiler. We know that slicers work (after all, people manage to write hbase loaders and such). Plus the whole input/output/slice/slicer thing is being revised right now, there is a proposal in PIG-966 you can look at if you would like to comment. In fact I just took your code and substituted some placeholders for various methods and classes (like HSSchema, etc), and I can see Slice methods getting called and everything. What do your location strings look like? Pig tries to be helpful by transforming locations into HDFS paths if the locations are not qualified by a scheme ( something:/ ). This makes file-based LoadFuncs easier to write, but it also means that you have to specify a scheme if you are doing something that's not file-based. Try changing your code to expect the location string to start with some scheme, such as abcd:/ (or perhaps something more meaningful) and things should work. -D 2009/10/28 RichardGUO Fei : > > It seems that the real problem is JobControlCompiler.getJob, because this > method removes POLoad and sets PigInputFormat as the input format but the > path passed to my storage is actually a table name rather than a file path! I > think that the LoadFunc should be more flexible. I am trying to fix this at > this moment. > > > > Thanks, > > Richard > >> From: zjf...@gmail.com >> To: pig-dev@hadoop.apache.org >> Subject: RE: Custom Loadfunc problem! >> Date: Tue, 27 Oct 2009 23:40:43 -0800 >> >> >> I mean hadoop's local mode not pig's own local mode >> >> >> -Original Message- >> From: Dmitriy Ryaboy [mailto:dvrya...@gmail.com] >> Sent: 2009年10月26日 6:33 >> To: pig-dev@hadoop.apache.org; pig-dev@hadoop.apache.org >> Subject: RE: Custom Loadfunc problem! >> >> Jeff, >> Slicers dont work in local mode, there is an ancient ticket for that on the >> Jira. >> >> Richard -- hard to say whats going on without more code. Think you can come >> up with a simplified version of your loadfunc that fails in a similar >> manner, and share it? >> >> >> >> -Original Message- >> From: "zjffdu" >> To: pig-dev@hadoop.apache.org >> Sent: 10/27/2009 1:45 AM >> Subject: RE: Custom Loadfunc problem! >> >> Illustrate will not execute the job, while dump and store will execute it. >> So I think there must be something wrong with your custom slicer. I suggest >> you set breakpoint in your slicer and debug it in map reduce mode locally >> >> >> >> -Original Message- >> From: RichardGUO Fei [mailto:gladiato...@hotmail.com] >> Sent: 20091026 0:43 >> To: pig-dev@hadoop.apache.org >> Subject: RE: Custom Loadfunc problem! >> >> >> Hi, >> >> >> >> Btw, my program works with ILLUSTRATE but n >> >> [truncated by sender] >> > > _ > MSN十周年庆典,查看MSN注册时间,赢取神秘大奖 > http://10.msn.com.cn
[jira] Commented: (PIG-790) Error message should indicate in which line number in the Pig script the error occured (debugging BinCond)
[ https://issues.apache.org/jira/browse/PIG-790?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12770959#action_12770959 ] Hadoop QA commented on PIG-790: --- -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12423074/PIG-790-2.patch against trunk revision 830335. +1 @author. The patch does not contain any @author tags. -1 tests included. The patch doesn't appear to include any new or modified tests. Please justify why no tests are needed for this patch. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. +1 core tests. The patch passed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/121/testReport/ Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/121/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/121/console This message is automatically generated. > Error message should indicate in which line number in the Pig script the > error occured (debugging BinCond) > -- > > Key: PIG-790 > URL: https://issues.apache.org/jira/browse/PIG-790 > Project: Pig > Issue Type: Bug >Affects Versions: 0.4.0 >Reporter: Viraj Bhat >Assignee: Daniel Dai >Priority: Minor > Fix For: 0.6.0 > > Attachments: error_rerport.pig, myerrordata.txt, PIG-790-1.patch, > PIG-790-2.patch, pig_1240972895275.log > > > I have a simple Pig script which loads integer data and does a Bincond, where > it compares, (col1 eq ''). There is an error message that is generated in > this case, but it does not specify the line number in the script. > {code} > MYDATA = load '/user/viraj/myerrordata.txt' using PigStorage() as (col1:int, > col2:int); > MYDATA_PROJECT = FOREACH MYDATA GENERATE ((col1 eq '') ? 1 : 0) as newcol1, > ((col1 neq '') ? col1 - col2 : > 16) > as time_diff; > dump MYDATA_PROJECT; > {code} > == > 2009-04-29 02:33:07,182 [main] INFO > org.apache.pig.backend.hadoop.executionengine.HExecutionEngine - Connecting > to hadoop file system at: hdfs://localhost:9000 > 2009-04-29 02:33:08,584 [main] INFO > org.apache.pig.backend.hadoop.executionengine.HExecutionEngine - Connecting > to map-reduce job tracker at: localhost:9001 > 2009-04-29 02:33:08,836 [main] INFO org.apache.pig.PigServer - Create a new > graph. > 2009-04-29 02:33:10,040 [main] ERROR org.apache.pig.tools.grunt.Grunt - ERROR > 1039: Incompatible types in EqualTo Operator left hand side:int right hand > side:chararray > Details at logfile: /home/viraj/pig-svn/trunk/pig_1240972386081.log > == > It would be good if the error message has a line number and a copy of the > line in the script which is causing the problem. > Attaching data, script and log file. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-747) Logical to Physical Plan Translation fails when temporary alias are created within foreach
[ https://issues.apache.org/jira/browse/PIG-747?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12770901#action_12770901 ] Hadoop QA commented on PIG-747: --- -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12422758/PIG-747-1.patch against trunk revision 830335. +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 3 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/120/testReport/ Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/120/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/120/console This message is automatically generated. > Logical to Physical Plan Translation fails when temporary alias are created > within foreach > -- > > Key: PIG-747 > URL: https://issues.apache.org/jira/browse/PIG-747 > Project: Pig > Issue Type: Bug >Affects Versions: 0.4.0 >Reporter: Viraj Bhat >Assignee: Daniel Dai > Fix For: 0.6.0 > > Attachments: physicalplan.txt, physicalplanprob.pig, PIG-747-1.patch > > > Consider a the pig script which calculates a new column F inside the foreach > as: > {code} > A = load 'physicalplan.txt' as (col1,col2,col3); > B = foreach A { >D = col1/col2; >E = col3/col2; >F = E - (D*D); >generate >F as newcol; > }; > dump B; > {code} > This gives the following error: > === > Caused by: > org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogicalToPhysicalTranslatorException: > ERROR 2015: Invalid physical operators in the physical plan > at > org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTranslationVisitor.visit(LogToPhyTranslationVisitor.java:377) > at > org.apache.pig.impl.logicalLayer.LOMultiply.visit(LOMultiply.java:63) > at > org.apache.pig.impl.logicalLayer.LOMultiply.visit(LOMultiply.java:29) > at > org.apache.pig.impl.plan.DependencyOrderWalkerWOSeenChk.walk(DependencyOrderWalkerWOSeenChk.java:68) > at > org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTranslationVisitor.visit(LogToPhyTranslationVisitor.java:908) > at > org.apache.pig.impl.logicalLayer.LOForEach.visit(LOForEach.java:122) > at org.apache.pig.impl.logicalLayer.LOForEach.visit(LOForEach.java:41) > at > org.apache.pig.impl.plan.DependencyOrderWalker.walk(DependencyOrderWalker.java:68) > at org.apache.pig.impl.plan.PlanVisitor.visit(PlanVisitor.java:51) > at > org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.compile(HExecutionEngine.java:246) > ... 10 more > Caused by: org.apache.pig.impl.plan.PlanException: ERROR 0: Attempt to give > operator of type > org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.Divide > multiple outputs. This operator does not support multiple outputs. > at > org.apache.pig.impl.plan.OperatorPlan.connect(OperatorPlan.java:158) > at > org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan.connect(PhysicalPlan.java:89) > at > org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTranslationVisitor.visit(LogToPhyTranslationVisitor.java:373) > ... 19 more > === -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1029) HBaseStorage is way too slow to be usable
[ https://issues.apache.org/jira/browse/PIG-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12770875#action_12770875 ] Jeff Zhang commented on PIG-1029: - Vincent, what environment do you use to get the performance comparison ? > HBaseStorage is way too slow to be usable > - > > Key: PIG-1029 > URL: https://issues.apache.org/jira/browse/PIG-1029 > Project: Pig > Issue Type: Bug >Affects Versions: 0.4.0 >Reporter: Vincent BARAT > > I have performed a set of benchmarks on HBaseStorage loader, using PIG 0.4.0 > and HBase 0.20.0 (using the patch referred in > https://issues.apache.org/jira/browse/PIG-970) and Hadoop 0.20.0. > The HBaseStorage loader is basically 10x slower than the PigStorage loader. > To bypass this limitation, I had to read my HBase tables, write them to a > Hadoop file and then use this file as input for my subsequent computations. > I report this bug for the track, I will try to sse if I can optimise this a > bit. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-970) Support of HBase 0.20.0
[ https://issues.apache.org/jira/browse/PIG-970?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12770866#action_12770866 ] Jeff Zhang commented on PIG-970: Any progress on this issue ? It looks like it's still opened > Support of HBase 0.20.0 > --- > > Key: PIG-970 > URL: https://issues.apache.org/jira/browse/PIG-970 > Project: Pig > Issue Type: Improvement > Components: impl >Affects Versions: 0.3.0 >Reporter: Vincent BARAT > Attachments: build.xml.path, pig-hbase-0.20.0-support.patch, > pig-hbase-20-v2.patch, TEST-org.apache.pig.test.TestHBaseStorage.txt, > TEST-org.apache.pig.test.TestHBaseStorage.txt > > > The support of HBase is currently very limited and restricted to HBase 0.18.0. > Because the next releases of PIG will support Hadoop 0.20.0, they should also > support HBase 0.20.0. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
RE: Custom Loadfunc problem!
It seems that the real problem is JobControlCompiler.getJob, because this method removes POLoad and sets PigInputFormat as the input format but the path passed to my storage is actually a table name rather than a file path! I think that the LoadFunc should be more flexible. I am trying to fix this at this moment. Thanks, Richard > From: zjf...@gmail.com > To: pig-dev@hadoop.apache.org > Subject: RE: Custom Loadfunc problem! > Date: Tue, 27 Oct 2009 23:40:43 -0800 > > > I mean hadoop's local mode not pig's own local mode > > > -Original Message- > From: Dmitriy Ryaboy [mailto:dvrya...@gmail.com] > Sent: 2009年10月26日 6:33 > To: pig-dev@hadoop.apache.org; pig-dev@hadoop.apache.org > Subject: RE: Custom Loadfunc problem! > > Jeff, > Slicers dont work in local mode, there is an ancient ticket for that on the > Jira. > > Richard -- hard to say whats going on without more code. Think you can come > up with a simplified version of your loadfunc that fails in a similar > manner, and share it? > > > > -Original Message- > From: "zjffdu" > To: pig-dev@hadoop.apache.org > Sent: 10/27/2009 1:45 AM > Subject: RE: Custom Loadfunc problem! > > Illustrate will not execute the job, while dump and store will execute it. > So I think there must be something wrong with your custom slicer. I suggest > you set breakpoint in your slicer and debug it in map reduce mode locally > > > > -Original Message- > From: RichardGUO Fei [mailto:gladiato...@hotmail.com] > Sent: 20091026 0:43 > To: pig-dev@hadoop.apache.org > Subject: RE: Custom Loadfunc problem! > > > Hi, > > > > Btw, my program works with ILLUSTRATE but n > > [truncated by sender] > _ MSN十周年庆典,查看MSN注册时间,赢取神秘大奖 http://10.msn.com.cn
[jira] Commented: (PIG-1022) optimizer pushes filter before the foreach that generates column used by filter
[ https://issues.apache.org/jira/browse/PIG-1022?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12770831#action_12770831 ] Hadoop QA commented on PIG-1022: -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12422754/PIG-1022-1.patch against trunk revision 830335. +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 3 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/119/testReport/ Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/119/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-h7.grid.sp2.yahoo.net/119/console This message is automatically generated. > optimizer pushes filter before the foreach that generates column used by > filter > --- > > Key: PIG-1022 > URL: https://issues.apache.org/jira/browse/PIG-1022 > Project: Pig > Issue Type: Bug > Components: impl >Affects Versions: 0.4.0 >Reporter: Thejas M Nair >Assignee: Daniel Dai > Fix For: 0.6.0 > > Attachments: PIG-1022-1.patch > > > grunt> l = load 'students.txt' using PigStorage() as (name:chararray, > gender:chararray, age:chararray, score:chararray); > grunt> f = foreach l generate name, gender, age,score, '200' as > gid:chararray; > grunt> g = group f by (name, gid); > grunt> f2 = foreach g generate group.name as name: chararray, group.gid as > gid: chararray; > grunt> filt = filter f2 by gid == '200'; > grunt> explain filt; > In the plan generated filt is pushed up after the load and before the first > foreach, even though the filter is on gid which is generated in first foreach. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.