[ 
https://issues.apache.org/jira/browse/PIG-3979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14168695#comment-14168695
 ] 

David Dreyfus commented on PIG-3979:
------------------------------------

Comment at line 279:
The second tier should at least allow one tuple before it tries to aggregate.
This code retains the total number of tuples in the buffer while guaranteeing 
the second tier has at least one tuple.

The difference between doSpill and doContingentSpill:
The doSpill flag is set when spilling is running or needs to run.
It is set by POPartialAgg when its buffers are full after having run 
aggregations.
The doContingentSpill flag is set when the SpillableMemoryManager is notified
by GC that the runtime is low on memory and the SpillableMemoryManager 
identifies
the particular buffer as a good spill candidate because it is large. The 
contingent spill logic tries 
to satisfy the memory manager's request for freeing memory by aggregating data
rather than just spilling records to disk. 

> 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.14.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