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

Daniel Dai updated PIG-3979:
----------------------------
    Attachment: PIG-3979-3.patch

Here is a summary of changes I read:
1. Create a strong reference list spillablesSR for sorting: +1, this is to fix 
comparison contract exception. The list itself could be big and exacerbate the 
memory stress, but I cannot think of a better solution
2. Avoid invoking System.gc: I am not familiar with the gc detail, but the 
explanation sounds fine to me
3. POPartialAgg sample size should use memory percentage instead of numTuple : 
+1
4. LOG.info => LOG.debug: +1

I also make some minor comments in RB: https://reviews.apache.org/r/26527/. 
Many of the issues I have already addressed in PIG-3979-3.patch. I leave the 
change history in RB for your reference.

One more thing, this does not seems to completely solve [~mrflip]'s infinite 
loop issue. To solve that, we shall do an actually spill inside 
POPartialAgg.spill instead of putting a doContingentSpill flag. Otherwise, 
during the actual spill in getNextTuple, we could enter into handleNotification 
constantly. If we do actual spill inside POPartialAgg.spill, since this is 
inside handleNotification, the spill process will not be interrupted by another 
handleNotification. Isn't it?

> group all performance, garbage collection, and incremental aggregation
> ----------------------------------------------------------------------
>
>                 Key: PIG-3979
>                 URL: https://issues.apache.org/jira/browse/PIG-3979
>             Project: Pig
>          Issue Type: Improvement
>          Components: impl
>    Affects Versions: 0.12.0, 0.11.1
>            Reporter: David Dreyfus
>            Assignee: David Dreyfus
>             Fix For: 0.15.0
>
>         Attachments: PIG-3979-3.patch, PIG-3979-v1.patch, 
> POPartialAgg.java.patch, SpillableMemoryManager.java.patch
>
>
> I have a PIG statement similar to:
> summary = foreach (group data ALL) generate 
> COUNT(data.col1), SUM(data.col2), SUM(data.col2)
> , Moments(col3)
> , Moments(data.col4)
> There are a couple of hundred columns.
> I set the following:
> SET pig.exec.mapPartAgg true;
> SET pig.exec.mapPartAgg.minReduction 3;
> SET pig.cachedbag.memusage 0.05;
> I found that when I ran this on a JVM with insufficient memory, the process 
> eventually timed out because of an infinite garbage collection loop.
> The problem was invariant to the memusage setting.
> I solved the problem by making changes to:
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperator.POPartialAgg.java
> Rather than reading in 10000 records to establish an estimate of the 
> reduction, I make an estimate after reading in enough tuples to fill 
> pig.cachedbag.memusage percent of Runtime.getRuntime().maxMemory().
> I also made a change to guarantee at least one record allowed in second tier 
> storage. In the current implementation, if the reduction is very high 1000:1, 
> space in second tier storage is zero.
> With these changes, I can summarize large data sets with small JVMs. I also 
> find that setting pig.cachedbag.memusage to a small number such as 0.05 
> results in much better garbage collection performance without reducing 
> throughput. I suppose tuning GC would also solve a problem with excessive 
> garbage collection.
> The performance is sweet. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to