You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pig.apache.org by "Olga Natkovich (JIRA)" <ji...@apache.org> on 2010/07/08 23:14:51 UTC

[jira] Updated: (PIG-1442) java.lang.OutOfMemoryError: Java heap space (Reopen of PIG-766)

     [ https://issues.apache.org/jira/browse/PIG-1442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Olga Natkovich updated PIG-1442:
--------------------------------

         Assignee: Thejas M Nair
    Fix Version/s: 0.8.0

> java.lang.OutOfMemoryError: Java heap space (Reopen of PIG-766)
> ---------------------------------------------------------------
>
>                 Key: PIG-1442
>                 URL: https://issues.apache.org/jira/browse/PIG-1442
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>    Affects Versions: 0.2.0, 0.7.0
>         Environment: Apache-Hadoop 0.20.2 + Pig 0.7.0 and also for 0.8.0-dev (18/may)
> Hadoop-0.18.3 (cloudera RPMs) + PIG 0.2.0
>            Reporter: Dirk Schmid
>            Assignee: Thejas M Nair
>             Fix For: 0.8.0
>
>
> As mentioned by Ashutosh this is a reopen of https://issues.apache.org/jira/browse/PIG-766 because there is still a problem which causes that PIG scales only by memory.
> For convenience here comes the last entry of the PIG-766-Jira-Ticket:
> {quote}1. Are you getting the exact same stack trace as mentioned in the jira?{quote} Yes the same and some similar traces:
> {noformat}
> java.lang.OutOfMemoryError: Java heap space
> 	at java.util.Arrays.copyOf(Arrays.java:2786)
> 	at java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:94)
> 	at java.io.DataOutputStream.write(DataOutputStream.java:90)
> 	at java.io.FilterOutputStream.write(FilterOutputStream.java:80)
> 	at org.apache.pig.data.DataReaderWriter.writeDatum(DataReaderWriter.java:279)
> 	at org.apache.pig.data.DefaultTuple.write(DefaultTuple.java:264)
> 	at org.apache.pig.data.DefaultAbstractBag.write(DefaultAbstractBag.java:249)
> 	at org.apache.pig.data.DataReaderWriter.writeDatum(DataReaderWriter.java:214)
> 	at org.apache.pig.data.DefaultTuple.write(DefaultTuple.java:264)
> 	at org.apache.pig.data.DataReaderWriter.writeDatum(DataReaderWriter.java:209)
> 	at org.apache.pig.data.DefaultTuple.write(DefaultTuple.java:264)
> 	at org.apache.pig.impl.io.PigNullableWritable.write(PigNullableWritable.java:123)
> 	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.IFile$Writer.append(IFile.java:179)
> 	at org.apache.hadoop.mapred.Task$CombineOutputCollector.collect(Task.java:880)
> 	at org.apache.hadoop.mapred.Task$NewCombinerRunner$OutputConverter.write(Task.java:1201)
> 	at org.apache.hadoop.mapreduce.TaskInputOutputContext.write(TaskInputOutputContext.java:80)
> 	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner$Combine.processOnePackageOutput(PigCombiner.java:199)
> 	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner$Combine.reduce(PigCombiner.java:161)
> 	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner$Combine.reduce(PigCombiner.java:51)
> 	at org.apache.hadoop.mapreduce.Reducer.run(Reducer.java:176)
> 	at org.apache.hadoop.mapred.Task$NewCombinerRunner.combine(Task.java:1222)
> 	at org.apache.hadoop.mapred.ReduceTask$ReduceCopier$InMemFSMergeThread.doInMemMerge(ReduceTask.java:2563)
> 	at org.apache.hadoop.mapred.ReduceTask$ReduceCopier$InMemFSMergeThread.run(ReduceTask.java:2501)
> java.lang.OutOfMemoryError: Java heap space
> 	at org.apache.pig.data.DefaultTuple.(DefaultTuple.java:58)
> 	at org.apache.pig.data.DefaultTupleFactory.newTuple(DefaultTupleFactory.java:35)
> 	at org.apache.pig.data.DataReaderWriter.bytesToTuple(DataReaderWriter.java:61)
> 	at org.apache.pig.data.DataReaderWriter.readDatum(DataReaderWriter.java:142)
> 	at org.apache.pig.data.DataReaderWriter.readDatum(DataReaderWriter.java:136)
> 	at org.apache.pig.data.DefaultAbstractBag.readFields(DefaultAbstractBag.java:263)
> 	at org.apache.pig.data.DataReaderWriter.bytesToBag(DataReaderWriter.java:71)
> 	at org.apache.pig.data.DataReaderWriter.readDatum(DataReaderWriter.java:145)
> 	at org.apache.pig.data.DataReaderWriter.readDatum(DataReaderWriter.java:136)
> 	at org.apache.pig.data.DataReaderWriter.bytesToTuple(DataReaderWriter.java:63)
> 	at org.apache.pig.data.DataReaderWriter.readDatum(DataReaderWriter.java:142)
> 	at org.apache.pig.data.DataReaderWriter.readDatum(DataReaderWriter.java:136)
> 	at org.apache.pig.data.DefaultTuple.readFields(DefaultTuple.java:284)
> 	at org.apache.pig.impl.io.PigNullableWritable.readFields(PigNullableWritable.java:114)
> 	at org.apache.hadoop.io.serializer.WritableSerialization$WritableDeserializer.deserialize(WritableSerialization.java:67)
> 	at org.apache.hadoop.io.serializer.WritableSerialization$WritableDeserializer.deserialize(WritableSerialization.java:40)
> 	at org.apache.hadoop.mapreduce.ReduceContext.nextKeyValue(ReduceContext.java:116)
> 	at org.apache.hadoop.mapreduce.ReduceContext$ValueIterator.next(ReduceContext.java:163)
> 	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POCombinerPackage.getNext(POCombinerPackage.java:155)
> 	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMultiQueryPackage.getNext(POMultiQueryPackage.java:242)
> 	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner$Combine.processOnePackageOutput(PigCombiner.java:170)
> 	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner$Combine.reduce(PigCombiner.java:161)
> 	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner$Combine.reduce(PigCombiner.java:51)
> 	at org.apache.hadoop.mapreduce.Reducer.run(Reducer.java:176)
> 	at org.apache.hadoop.mapred.Task$NewCombinerRunner.combine(Task.java:1222)
> 	at org.apache.hadoop.mapred.ReduceTask$ReduceCopier$InMemFSMergeThread.doInMemMerge(ReduceTask.java:2563)
> 	at org.apache.hadoop.mapred.ReduceTask$ReduceCopier$InMemFSMergeThread.run(ReduceTask.java:2501)
> java.lang.OutOfMemoryError: Java heap space
> 	at java.util.ArrayList.(ArrayList.java:112)
> 	at org.apache.pig.data.DefaultTuple.(DefaultTuple.java:58)
> 	at org.apache.pig.data.DefaultTupleFactory.newTuple(DefaultTupleFactory.java:35)
> 	at org.apache.pig.data.DataReaderWriter.bytesToTuple(DataReaderWriter.java:61)
> 	at org.apache.pig.data.DataReaderWriter.readDatum(DataReaderWriter.java:142)
> 	at org.apache.pig.data.DataReaderWriter.readDatum(DataReaderWriter.java:136)
> 	at org.apache.pig.data.DefaultAbstractBag.readFields(DefaultAbstractBag.java:263)
> 	at org.apache.pig.data.DataReaderWriter.bytesToBag(DataReaderWriter.java:71)
> 	at org.apache.pig.data.DataReaderWriter.readDatum(DataReaderWriter.java:145)
> 	at org.apache.pig.data.DataReaderWriter.readDatum(DataReaderWriter.java:136)
> 	at org.apache.pig.data.DefaultTuple.readFields(DefaultTuple.java:284)
> 	at org.apache.pig.data.InternalCachedBag$CachedBagIterator.hasNext(InternalCachedBag.java:221)
> 	at org.apache.pig.builtin.Distinct.getDistinctFromNestedBags(Distinct.java:138)
> 	at org.apache.pig.builtin.Distinct.access$200(Distinct.java:40)
> 	at org.apache.pig.builtin.Distinct$Intermediate.exec(Distinct.java:103)
> 	at org.apache.pig.builtin.Distinct$Intermediate.exec(Distinct.java:96)
> 	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc.getNext(POUserFunc.java:209)
> 	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc.getNext(POUserFunc.java:250)
> 	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.processPlan(POForEach.java:341)
> 	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.getNext(POForEach.java:289)
> 	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:276)
> 	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange.getNext(POLocalRearrange.java:259)
> 	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.PODemux.runPipeline(PODemux.java:217)
> 	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.PODemux.getNext(PODemux.java:207)
> 	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner$Combine.processOnePackageOutput(PigCombiner.java:183)
> 	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner$Combine.reduce(PigCombiner.java:161)
> 	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigCombiner$Combine.reduce(PigCombiner.java:51)
> 	at org.apache.hadoop.mapreduce.Reducer.run(Reducer.java:176)
> 	at org.apache.hadoop.mapred.Task$NewCombinerRunner.combine(Task.java:1222)
> 	at org.apache.hadoop.mapred.ReduceTask$ReduceCopier$InMemFSMergeThread.doInMemMerge(ReduceTask.java:2563)
> 	at org.apache.hadoop.mapred.ReduceTask$ReduceCopier$InMemFSMergeThread.run(ReduceTask.java:2501)
> {noformat}
> {quote}
> 2. Which operations are you doing in your query - join, group-by, any other ?
> 3. What load/store func are you using to read and write data? PigStorage or your own ?
> 4. What is your data size and memory available to your tasks?
> 5. Do you have very large records in your dataset, like hundreds of MB for one record ?
> It would be great if you can paste here the script from which you get this exception.
> {quote}
> As we started to test the transformation (see below) the OutOfMemory-Error first occured at input-datasets of about 150MB.
> Increasing the Memory for the child-vms by setting {{mapred.child.java.opts}} to {{600m}} fixed it for a while.
> When using larger input-dataset the problem reappears.
> *Input-Data:*
> A CSV-File, ~14GB Dataset, ~100,000,000 Records per Dataset, ~145 Byte per Record
> *Example:*
> {noformat} 
>   USER_ID                       REQUEST_DATE    SESSION                                 COMPANY SERVICENAME  SECTION_1  SECTION_2  SECTION_3  SECTION_4  SECTION_5  SECTION_6     SECTION SECTION_NEW
>   ac14263e-22082-2263455080-9   2010-03-02      ac14263e-22082-2263455080-9-1273015305  ABC     (NULL)       main       (NULL)     (NULL)     (NULL)     (NULL)     abc/main/mail /main/mail
>   ...
>   ...
> {noformat} 
> *The Pig-Script*
> {code}
> A = LOAD 'full_load' USING PigStorage('\t');
> B = FOREACH A GENERATE $4 AS servicename, $3 AS company, $2 AS session, $0 as user_id
>                        , $5 AS section_1, $6 AS section_2, $7 AS section_3, $8 as section_4
>                        , $9 as section_5, $10 as section_6, $11 AS section;
>                         
> /* 1st aggregation */
> S0 = GROUP B BY (servicename, company);
> S0_A = FOREACH S0 {
>                     unique_clients = DISTINCT B.user_id;
>                     visits = DISTINCT B.session;
>                     GENERATE FLATTEN(group), COUNT(B) AS pi_count, COUNT(unique_clients) AS unique_clients_count, COUNT(visits) AS visit_count;
>                   }
> S0_B = FOREACH S0_A GENERATE servicename, company, '' as section_1, '' as section_2, '' as section_3, '' as section_4
>                            , '' as section_5, '' as section_6, '' as section, pi_count, unique_clients_count
>                            , visit_count, 0 as level;
> /* 2nd aggregation */
> S1 = GROUP B BY (servicename, company, section_1); S1_A = FOREACH S1 {
>                     unique_clients = DISTINCT B.user_id;
>                     visits = DISTINCT B.session;
>                     GENERATE FLATTEN(group), COUNT(B) AS pi_count, COUNT(unique_clients) AS unique_clients_count, COUNT(visits) AS visit_count;
>                   }
> S1_B = FOREACH S1_A GENERATE servicename, company, section_1, '' as section_2, '' as section_3, '' as section_4
>                              , '' as section_5, '' as section_6, '' as section, pi_count, unique_clients_count
>                              , visit_count, 1 as level;
> /* 3rd - 7th aggregation may follow here */
> /* build result*/
> X = UNION S0_B, S1_B;
> STORE X INTO 'result' USING PigStorage ('\t'); {code} 

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.