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

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

Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813651
  
    --- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
 ---
    @@ -285,8 +648,18 @@ public AggOutcome doWork() {
           // In the future HashAggregate may also need to perform some actions 
conditionally
           // in the outer try block.
     
    +      assert ! handlingSpills || currentIndex < Integer.MAX_VALUE;
    +
           outside:
           while (true) {
    +
    +        // This would be called only once - after actual data arrives on 
incoming
    +        if ( schema == null && incoming.getRecordCount() > 0 ) {
    --- End diff --
    
    This kind of work is often done in response to the status codes from the 
upstream operator. Have to handle OK_NEW_SCHEMA, OK. The schema is defined on 
the first batch, with OK_NEW_SCHEMA, typically with a row count of 0. Must also 
handle (and probably fail) for OK_NEW_SCHEMA on subsequent batches.
    
    By putting the code here, rather in the code that calls the upstream 
`next()` it is necessary to reconcile here with that other code when doing a 
review.


> 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