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

ASF GitHub Bot commented on DRILL-5457:
---------------------------------------

Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119748791
  
    --- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
 ---
    @@ -306,14 +685,29 @@ public AggOutcome doWork() {
     
               while (true) {
                 // Cleanup the previous batch since we are done processing it.
    +            long pre = allocator.getAllocatedMemory();
                 for (VectorWrapper<?> v : incoming) {
                   v.getValueVector().clear();
                 }
    +            long beforeAlloc = allocator.getAllocatedMemory();
    +
    +            // Get the next RecordBatch from the incoming
                 IterOutcome out = outgoing.next(0, incoming);
    +
    +            // If incoming batch is bigger than our estimate - adjust the 
estimate
    +            long afterAlloc = allocator.getAllocatedMemory();
    +            long incomingBatchSize = afterAlloc - beforeAlloc;
    +            if ( /* ! handlingSpills && */ estMaxBatchSize < 
incomingBatchSize ) {
    +               logger.trace("Found a bigger incoming batch: {} , prior 
estimate was: {}",incomingBatchSize,estMaxBatchSize);
    +               estMaxBatchSize = incomingBatchSize;
    +            }
    +
                 if (EXTRA_DEBUG_1) {
                   logger.debug("Received IterOutcome of {}", out);
                 }
                 switch (out) {
    +              case RESTART:
    +                logger.warn("HASH AGG: doWork got a RESTART...");
    --- End diff --
    
    This code (and the RESTART state) was eliminated ....


> Support Spill to Disk for the Hash Aggregate Operator
> -----------------------------------------------------
>
>                 Key: DRILL-5457
>                 URL: https://issues.apache.org/jira/browse/DRILL-5457
>             Project: Apache Drill
>          Issue Type: Improvement
>          Components: Execution - Relational Operators
>    Affects Versions: 1.10.0
>            Reporter: Boaz Ben-Zvi
>            Assignee: Boaz Ben-Zvi
>             Fix For: 1.11.0
>
>
> Support gradual spilling memory to disk as the available memory gets too 
> small to allow in memory work for the Hash Aggregate Operator.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to